-
Notifications
You must be signed in to change notification settings - Fork 3.9k
CASSANDRA-21133 Make bin/sstableupgrade functionally on par with nodetool upgradesstables #4825
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
base: trunk
Are you sure you want to change the base?
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -1025,12 +1025,31 @@ public Descriptor newSSTableDescriptor(File directory, SSTableFormat<?, ?> forma | |
|
|
||
| public Descriptor newSSTableDescriptor(File directory, Version version) | ||
| { | ||
| Descriptor newDescriptor = new Descriptor(version, | ||
| directory, | ||
| getKeyspaceName(), | ||
| name, | ||
| sstableIdGenerator.get()); | ||
| assert !newDescriptor.fileFor(Components.DATA).exists(); | ||
| Descriptor newDescriptor; | ||
|
|
||
| /* | ||
| * Note that thread safety is managed by the underlying sstableIdGenerator (e.g., via AtomicInteger), | ||
| * guaranteeing unique IDs for concurrent callers within the process. | ||
| * * This loop acts as a collision-resolution mechanism for offline vs. online operations. | ||
| * Offline tools (such as bin/sstableupgrade) may generate new SSTables on disk, bypassing | ||
| * the in-memory ID generator. If the live process encounters a collision due to such | ||
| * out-of-band generation, this loop ensures the process advances the generator until a | ||
| * free ID is found. | ||
| */ | ||
| while (true) | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think that thread safety is just delegated to how @arvindKandpal-ksolves could you put a comment on
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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. |
||
| { | ||
| newDescriptor = new Descriptor(version, | ||
| directory, | ||
| getKeyspaceName(), | ||
| name, | ||
| sstableIdGenerator.get()); | ||
|
|
||
| if (!newDescriptor.fileFor(Components.DATA).exists()) | ||
| break; | ||
|
|
||
| logger.debug("Generated SSTable id {} collides with existing file; advancing.", newDescriptor.id); | ||
| } | ||
|
|
||
| return newDescriptor; | ||
| } | ||
|
|
||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -48,6 +48,7 @@ | |
| import org.apache.cassandra.SchemaLoader; | ||
| import org.apache.cassandra.UpdateBuilder; | ||
| import org.apache.cassandra.Util; | ||
| import org.apache.cassandra.config.DatabaseDescriptor; | ||
| import org.apache.cassandra.cql3.ColumnIdentifier; | ||
| import org.apache.cassandra.cql3.Operator; | ||
| import org.apache.cassandra.cql3.statements.schema.IndexTarget; | ||
|
|
@@ -71,8 +72,11 @@ | |
| import org.apache.cassandra.index.transactions.UpdateTransaction; | ||
| import org.apache.cassandra.io.sstable.Component; | ||
| import org.apache.cassandra.io.sstable.Descriptor; | ||
| import org.apache.cassandra.io.sstable.SSTableIdFactory; | ||
| import org.apache.cassandra.io.sstable.SSTableId; | ||
| import org.apache.cassandra.io.sstable.SSTableReadsListener; | ||
| import org.apache.cassandra.io.sstable.ScrubTest; | ||
| import org.apache.cassandra.io.sstable.format.Version; | ||
| import org.apache.cassandra.io.sstable.format.SSTableFormat.Components; | ||
| import org.apache.cassandra.io.sstable.format.SSTableReader; | ||
| import org.apache.cassandra.io.util.File; | ||
|
|
@@ -839,6 +843,52 @@ public static long getSnapshotManifestAndSchemaFileSizes(TableSnapshot snapshot) | |
| return schemaAndManifestFileSizes; | ||
| } | ||
|
|
||
| @Test | ||
| public void testNewSSTableDescriptorCollision() throws Exception | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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. |
||
| { | ||
| ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF_STANDARD1); | ||
| File dir = cfs.getDirectories().getDirectoryForNewSSTables(); | ||
| Version version = DatabaseDescriptor.getSelectedSSTableFormat().getLatestVersion(); | ||
|
|
||
| Descriptor probe = cfs.newSSTableDescriptor(dir, version); | ||
|
|
||
| int currentId; | ||
| try { | ||
| currentId = Integer.parseInt(probe.id.toString()); | ||
| } catch (NumberFormatException e) { | ||
| return; | ||
| } | ||
|
|
||
| SSTableId blockedId1 = SSTableIdFactory.instance.fromString(String.valueOf(currentId + 1)); | ||
| SSTableId blockedId2 = SSTableIdFactory.instance.fromString(String.valueOf(currentId + 2)); | ||
|
|
||
| Descriptor blockedDesc1 = new Descriptor(version, dir, cfs.getKeyspaceName(), cfs.name, blockedId1); | ||
| Descriptor blockedDesc2 = new Descriptor(version, dir, cfs.getKeyspaceName(), cfs.name, blockedId2); | ||
|
|
||
| File blockedDataFile1 = blockedDesc1.fileFor(Components.DATA); | ||
| File blockedDataFile2 = blockedDesc2.fileFor(Components.DATA); | ||
|
|
||
| try | ||
| { | ||
| blockedDataFile1.createFileIfNotExists(); | ||
| blockedDataFile2.createFileIfNotExists(); | ||
|
|
||
| Descriptor nextDesc = cfs.newSSTableDescriptor(dir, version); | ||
|
|
||
| Assert.assertNotEquals("The loop should have skipped the first blocked ID", | ||
| blockedDesc1.id, nextDesc.id); | ||
| Assert.assertNotEquals("The loop should have skipped the second blocked ID", | ||
| blockedDesc2.id, nextDesc.id); | ||
| Assert.assertFalse("The returned descriptor must not already exist on disk", | ||
| nextDesc.fileFor(Components.DATA).exists()); | ||
| } | ||
| finally | ||
| { | ||
| blockedDataFile1.deleteIfExists(); | ||
| blockedDataFile2.deleteIfExists(); | ||
| } | ||
| } | ||
|
|
||
| private Memtable fakeMemTableWithMinTS(ColumnFamilyStore cfs, long minTS) | ||
| { | ||
| return new AbstractMemtable(cfs.metadata, minTS) | ||
|
|
||
There was a problem hiding this comment.
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)
There was a problem hiding this comment.
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.