-
Notifications
You must be signed in to change notification settings - Fork 3.8k
CASSANDRA-21129: Offline TCM dump tool #4581
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?
Conversation
When a Cassandra node fails to start due to Transactional Cluster Metadata (TCM/CEP-21) corruption or issues, operators need a way to inspect the cluster metadata state offline without starting the node. The existing tools (nodetool, cqlsh) require a running node, leaving operators blind when debugging startup failures. With CEP-21 (Transactional Cluster Metadata), cluster metadata is stored in system tables: * system.local_metadata_log - Contains transformation entries (epoch -> transformation) * system.metadata_snapshots - Contains periodic snapshots of ClusterMetadata When a node fails to start due to TCM corruption or inconsistencies, operators have no way to inspect the metadata state without a running node. This tool fills that gap by reading directly from SSTables.
krummas
left a comment
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.
So this is an emergency recovery tool, hopefully extremely rarely used by an operator, I think we can slim it down a lot, these are the features I think we need here:
- dump metadata to current (or user provided) epoch
- serialized binary format
- metadata.toString, to avoid locking us in to any formats
- dump log (with start/end epoch), just toString each entry
- maybe add option to dump
system_clustermetadata.distributed_metadata_logif this is run on a CMS node
issues;
- shell script should live in tools/bin/ directory
- tool name - this does not dump sstable metadata, it dumps cluster metadata from sstables, sstable metadata is something different (see tools/bin/sstablemetadata)
- it copies the sstables to $CASSANDRA_HOME/data (or, if that is unset, in to the current directory) - we should create a temporary directory for import and clean that directory up after dumping the metadata, we need something like
Path p = Files.createTempDirectory("dumptcmlog");
DatabaseDescriptor.getRawConfig().data_file_directories = new String[] {p.resolve("data").toString()};
DatabaseDescriptor.getRawConfig().commitlog_directory = p.resolve("commitlog").toString();
DatabaseDescriptor.getRawConfig().accord.journal_directory = p.resolve("accord_journal").toString();
DatabaseDescriptor.getRawConfig().hints_directory = p.resolve("hints").toString();
DatabaseDescriptor.getRawConfig().saved_caches_directory = p.resolve("saved_caches").toString();
to make sure we only touch the tmp directory
krummas
left a comment
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.
looks good in general, just a few minor things inline
| import static com.google.common.base.Throwables.getStackTraceAsString; | ||
|
|
||
| /** | ||
| * Standalone tool to dump Transactional Cluster Metadata (TCM) from local SSTables. |
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.
"Standalone tool to dump cluster metadata", don't think we need to mention "TCM" or "transactional" at all in this file, what we're dumping is the cluster metadata, nothing transactional about it
| // Create temporary directory for SSTable import | ||
| setupTempDirectory(); | ||
|
|
||
| DatabaseDescriptor.setPartitioner(partitioner); |
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.
I don't think we need this - all the tables we read use MetaStrategy.partitioner
| mixinStandardHelpOptions = true, | ||
| description = "Dump Transactional Cluster Metadata from local SSTables", | ||
| subcommands = { TCMDump.DumpMetadata.class }) | ||
| public class TCMDump implements Runnable |
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.
maybe add "offline" to the name here somehow, to indicate that this shouldn't be used on running clusters
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.
oh and I'm not sure why we have the dump subcommand here, is it to be able to add more features here later? If so we should remove dump from the tool name, otherwise we can remove the subcommand. Or maybe have more subcommands (for dumping log/metadata/dist log) instead of the boolean switches
| CommandLine.usage(this, output.out); | ||
| } | ||
|
|
||
| @Command(name = "dump", description = "Dump cluster metadata from SSTables") |
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.
mention in the description that this is only for offline use
| DatabaseDescriptor.getRawConfig().data_file_directories = new String[]{ tempDir.resolve("data").toString() }; | ||
| DatabaseDescriptor.getRawConfig().commitlog_directory = tempDir.resolve("commitlog").toString(); | ||
| DatabaseDescriptor.getRawConfig().hints_directory = tempDir.resolve("hints").toString(); | ||
| DatabaseDescriptor.getRawConfig().saved_caches_directory = tempDir.resolve("saved_caches").toString(); |
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.
I think we also need to set accord.journal_directory here (see DatabaseDescriptor.createAllDirectories())
| Path outputPath = outputFile != null ? Path.of(outputFile) : Files.createTempFile("clustermetadata", ".dump"); | ||
| try (FileOutputStreamPlus out = new FileOutputStreamPlus(outputPath)) | ||
| { | ||
| VerboseMetadataSerializer.serialize(ClusterMetadata.serializer, metadata, out, NodeVersion.CURRENT.serializationVersion()); |
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.
we might want to add a parameter to specify which serialization version to use here, use case could be something like that we need to downgrade and need to rewrite in an older version
| SystemKeyspaceStorage storage = new SystemKeyspaceStorage(() -> snapshotManager); | ||
| LogState logState = storage.getPersistedLogState(); | ||
|
|
||
| for (Entry entry : logState.entries) |
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.
looks like we could use dumpLogEntries(...) here
When a Cassandra node fails to start due to Transactional Cluster Metadata (TCM/CEP-21) corruption or issues, operators need a way to inspect the cluster metadata state offline without starting the node. The existing tools (nodetool, cqlsh) require a running node, leaving operators blind when debugging startup failures.
With CEP-21 (Transactional Cluster Metadata), cluster metadata is stored in system tables:
When a node fails to start due to TCM corruption or inconsistencies, operators have no way to inspect the metadata state without a running node. This tool fills that gap by reading directly from SSTables.
Thanks for sending a pull request! Here are some tips if you're new here:
Commit messages should follow the following format:
The Cassandra Jira