diff --git a/bin/sstablemetadatadump b/bin/sstablemetadatadump new file mode 100755 index 000000000000..bcf9b998a8f0 --- /dev/null +++ b/bin/sstablemetadatadump @@ -0,0 +1,49 @@ +#!/bin/sh + +# 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. + +if [ "x$CASSANDRA_INCLUDE" = "x" ]; then + # Locations (in order) to use when searching for an include file. + for include in "`dirname "$0"`/cassandra.in.sh" \ + "$HOME/.cassandra.in.sh" \ + /usr/share/cassandra/cassandra.in.sh \ + /usr/local/share/cassandra/cassandra.in.sh \ + /opt/cassandra/cassandra.in.sh; do + if [ -r "$include" ]; then + . "$include" + break + fi + done +elif [ -r "$CASSANDRA_INCLUDE" ]; then + . "$CASSANDRA_INCLUDE" +fi + +if [ -z "$CLASSPATH" ]; then + echo "You must set the CLASSPATH var" >&2 + exit 1 +fi + +if [ "x$MAX_HEAP_SIZE" = "x" ]; then + MAX_HEAP_SIZE="256M" +fi + +"$JAVA" $JAVA_AGENT -ea -cp "$CLASSPATH" $JVM_OPTS -Xmx$MAX_HEAP_SIZE \ + -Dcassandra.storagedir="$cassandra_storagedir" \ + -Dlogback.configurationFile=logback-tools.xml \ + org.apache.cassandra.tools.SSTableMetadataDump "$@" + +# vi:ai sw=4 ts=4 tw=0 et diff --git a/src/java/org/apache/cassandra/tools/SSTableMetadataDump.java b/src/java/org/apache/cassandra/tools/SSTableMetadataDump.java new file mode 100644 index 000000000000..bba922aefc0c --- /dev/null +++ b/src/java/org/apache/cassandra/tools/SSTableMetadataDump.java @@ -0,0 +1,502 @@ +/* + * 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.tools; + +import java.io.IOException; +import java.io.PrintStream; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.Keyspace; +import org.apache.cassandra.db.SystemKeyspace; +import org.apache.cassandra.dht.Token; +import org.apache.cassandra.io.util.FileOutputStreamPlus; +import org.apache.cassandra.schema.Keyspaces; +import org.apache.cassandra.schema.Schema; +import org.apache.cassandra.schema.SchemaConstants; +import org.apache.cassandra.tcm.ClusterMetadata; +import org.apache.cassandra.tcm.ClusterMetadataService; +import org.apache.cassandra.tcm.Epoch; +import org.apache.cassandra.tcm.MetadataSnapshots; +import org.apache.cassandra.tcm.log.Entry; +import org.apache.cassandra.tcm.log.LogState; +import org.apache.cassandra.tcm.log.SystemKeyspaceStorage; +import org.apache.cassandra.tcm.membership.NodeVersion; +import org.apache.cassandra.tcm.serialization.VerboseMetadataSerializer; + +import picocli.CommandLine; +import picocli.CommandLine.Command; +import picocli.CommandLine.Option; + +import static com.google.common.base.Throwables.getStackTraceAsString; + +/** + * Standalone tool to dump Transactional Cluster Metadata (TCM) from local SSTables. + *

+ * Useful for debugging when a Cassandra instance cannot start due to TCM issues. + * This tool reads the local_metadata_log and metadata_snapshots tables from the + * system keyspace to reconstruct and display the cluster metadata state. + *

+ * Usage: + * sstablemetadatadump dump --data-dir /path/to/data --all + * sstablemetadatadump dump --data-dir /path/to/data --to-string + * sstablemetadatadump dump --sstables /path/to/system/local_metadata_log-xxx/ --epochs + */ +@Command(name = "sstablemetadatadump", +mixinStandardHelpOptions = true, +description = "Dump Transactional Cluster Metadata from local SSTables", +subcommands = { SSTableMetadataDump.DumpMetadata.class }) +public class SSTableMetadataDump implements Runnable +{ + private static final Output output = Output.CONSOLE; + + public static void main(String... args) + { + // Minimal initialization + Util.initDatabaseDescriptor(); + + CommandLine cli = new CommandLine(SSTableMetadataDump.class).setExecutionExceptionHandler((ex, cmd, parseResult) -> { + err(ex); + return 2; + }); + int status = cli.execute(args); + System.exit(status); + } + + protected static void err(Throwable e) + { + output.err.println("error: " + e.getMessage()); + output.err.println("-- StackTrace --"); + output.err.println(getStackTraceAsString(e)); + } + + @Override + public void run() + { + CommandLine.usage(this, output.out); + } + + /** + * Container for all collected dump data. Data is collected once and used + * for text output to avoid duplicate queries. + *

+ * Contents: + * - metadata: Final reconstructed ClusterMetadata (latest snapshot + applied transformations) + * - logState: Contains base snapshot + list of transformation entries for --epochs display + * - snapshotEpochs: List of epoch numbers where snapshots exist for --snapshots display + */ + private static class DumpData + { + final ClusterMetadata metadata; + final LogState logState; + final List snapshotEpochs; + + DumpData(ClusterMetadata metadata, LogState logState, List snapshotEpochs) + { + this.metadata = metadata; + this.logState = logState; + this.snapshotEpochs = snapshotEpochs; + } + } + + @Command(name = "dump", description = "Dump cluster metadata from SSTables") + public static class DumpMetadata implements Runnable + { + @Option(names = { "-d", "--data-dir" }, description = "Data directory containing system keyspace") + public String dataDir; + + @Option(names = { "-s", "--sstables" }, description = "Path to SSTable directory for metadata tables") + public String sstables; + + @Option(names = { "-p", "--partitioner" }, description = "Partitioner class name", + defaultValue = "org.apache.cassandra.dht.Murmur3Partitioner") + public String partitioner; + + @Option(names = { "-o", "--output" }, description = "Output file path for binary dump (default: temp file)") + public String outputFile; + + // Output mode flags + @Option(names = { "--to-string" }, description = "Print ClusterMetadata.toString() to stdout") + public boolean toStringOutput; + + @Option(names = { "--text" }, description = "Print detailed text output to stdout") + public boolean textOutput; + + // Scope flags (used with --text) + @Option(names = { "--epochs" }, description = "List all epochs and transformation kinds") + public boolean epochs; + + @Option(names = { "--schema" }, description = "Dump schema information") + public boolean schema; + + @Option(names = { "--directory" }, description = "Dump node membership/directory") + public boolean directory; + + @Option(names = { "--tokens" }, description = "Dump token map") + public boolean tokens; + + @Option(names = { "--snapshots" }, description = "List available snapshots") + public boolean snapshots; + + @Option(names = { "--transformations" }, description = "Dump full transformation details") + public boolean transformations; + + @Option(names = { "--all" }, description = "Include everything") + public boolean all; + + // Filters + @Option(names = { "--epoch" }, description = "Show state at specific epoch") + public Long targetEpoch; + + @Option(names = { "--from-epoch" }, description = "Filter from this epoch") + public Long fromEpoch; + + @Option(names = { "--to-epoch" }, description = "Filter to this epoch") + public Long toEpoch; + + // Debug + @Option(names = { "-v", "--verbose" }, description = "Verbose output") + public boolean verbose; + + @Option(names = { "--debug" }, description = "Show stack traces on errors") + public boolean debug; + + @Override + public void run() + { + try + { + DatabaseDescriptor.setPartitioner(partitioner); + + // Set up minimal schema for system keyspace + ClusterMetadataService.empty(Keyspaces.of(SystemKeyspace.metadata())); + Keyspace.setInitialized(); + + importSSTables(); + + DumpData data = collectData(); + + if (data.metadata == null) + { + output.out.println("No metadata available"); + return; + } + + // Handle output modes + if (toStringOutput) + { + // Print ClusterMetadata.toString() to stdout + output.out.println(data.metadata); + } + else if (textOutput) + { + // Print detailed text output + if (all) + { + epochs = schema = directory = tokens = snapshots = transformations = true; + } + outputText(data); + } + else + { + // Default: binary dump like ClusterMetadataService.dumpClusterMetadata() + dumpBinary(data.metadata); + } + } + catch (Exception e) + { + if (debug) + { + e.printStackTrace(output.err); + } + else + { + output.err.println("Error: " + e.getMessage()); + } + System.exit(1); + } + } + + /** + * Dumps ClusterMetadata to a binary file using VerboseMetadataSerializer. + * This is the same format used by ClusterMetadataService.dumpClusterMetadata(). + */ + private void dumpBinary(ClusterMetadata metadata) throws IOException + { + 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()); + } + output.out.println("Dumped cluster metadata to " + outputPath); + } + + /** + * Collects all data needed for output. This is done once to avoid + * creating multiple SystemKeyspaceStorage instances and re-querying. + *

+ * Data flow: + * 1. getPersistedLogState() returns LogState containing: + * - baseState: latest snapshot from metadata_snapshots + * - entries: all transformations after that snapshot + * 2. flatten() applies transformations to get final ClusterMetadata + * 3. listSnapshotsSince(EMPTY) returns all snapshot epoch numbers for display + */ + private DumpData collectData() + { + MetadataSnapshots snapshotManager = new MetadataSnapshots.SystemKeyspaceMetadataSnapshots(); + SystemKeyspaceStorage storage = new SystemKeyspaceStorage(() -> snapshotManager); + + LogState logState = storage.getPersistedLogState(); + + ClusterMetadata metadata = null; + if (!logState.isEmpty()) + { + if (targetEpoch != null) + { + logState = LogState.getForRecovery(Epoch.create(targetEpoch)); + } + + metadata = logState.flatten().baseState; + } + + List snapshotEpochs = snapshotManager.listSnapshotsSince(Epoch.EMPTY); + + return new DumpData(metadata, logState, snapshotEpochs); + } + + private void importSSTables() throws IOException + { + Keyspace ks = Schema.instance.getKeyspaceInstance(SchemaConstants.SYSTEM_KEYSPACE_NAME); + + // Find and import SSTables for local_metadata_log + String logTablePath = findTablePath(SystemKeyspace.METADATA_LOG); + if (logTablePath != null) + { + ColumnFamilyStore logCfs = ks.getColumnFamilyStore(SystemKeyspace.METADATA_LOG); + logCfs.importNewSSTables(Collections.singleton(logTablePath), false, false, false, false, false, false, true); + if (verbose) + { + output.out.println("Imported SSTables from: " + logTablePath); + } + } + + // Find and import SSTables for metadata_snapshots + String snapshotTablePath = findTablePath(SystemKeyspace.SNAPSHOT_TABLE_NAME); + if (snapshotTablePath != null) + { + ColumnFamilyStore snapshotCfs = ks.getColumnFamilyStore(SystemKeyspace.SNAPSHOT_TABLE_NAME); + snapshotCfs.importNewSSTables(Collections.singleton(snapshotTablePath), false, false, false, false, false, false, true); + if (verbose) + { + output.out.println("Imported SSTables from: " + snapshotTablePath); + } + } + } + + private String findTablePath(String tableName) throws IOException + { + if (sstables != null) + { + // User provided explicit path + if (sstables.contains(tableName)) + return sstables; + // Check if it's a parent directory containing the table + Path tableDir = Path.of(sstables, tableName); + if (Files.exists(tableDir)) + return tableDir.toString(); + } + + if (dataDir != null) + { + // Discover from data directory + String matches = findTablePath(tableName, dataDir); + if (matches != null) return matches; + } + + // Try default data directories from cassandra.yaml + String[] dataDirs = DatabaseDescriptor.getAllDataFileLocations(); + for (String dir : dataDirs) + { + String matches = findTablePath(tableName, dir); + if (matches != null) return matches; + } + + return null; + } + + private String findTablePath(String tableName, String dataDir) throws IOException + { + Path systemDir = Path.of(dataDir, "system"); + if (Files.exists(systemDir)) + { + try (Stream paths = Files.list(systemDir)) + { + List matches = paths.filter(p -> p.getFileName().toString().startsWith(tableName + "-")) + .collect(Collectors.toList()); + if (!matches.isEmpty()) + return matches.get(0).toString(); + } + } + return null; + } + + /** + * Filters log entries based on --from-epoch and --to-epoch flags. + */ + private List filterEntries(DumpData data) + { + Epoch from = fromEpoch != null ? Epoch.create(fromEpoch) : Epoch.EMPTY; + Epoch to = toEpoch != null ? Epoch.create(toEpoch) : Epoch.create(Long.MAX_VALUE); + + return data.logState.entries.stream() + .filter(e -> !e.epoch.isBefore(from) && !e.epoch.isAfter(to)) + .collect(Collectors.toList()); + } + + private void outputText(DumpData data) + { + PrintStream out = output.out; + + out.println("=== Cluster Metadata Dump ==="); + + if (data.metadata == null) + { + out.println("No metadata available"); + return; + } + + out.println("Current Epoch: " + data.metadata.epoch); + out.println("Partitioner: " + data.metadata.partitioner.getClass().getName()); + out.println(); + + if (epochs || transformations) + { + outputEpochsText(data); + } + + if (directory) + { + outputDirectoryText(data.metadata); + } + + if (tokens) + { + outputTokensText(data.metadata); + } + + if (schema) + { + outputSchemaText(data.metadata); + } + + if (snapshots) + { + outputSnapshotsText(data); + } + } + + private void outputEpochsText(DumpData data) + { + PrintStream out = output.out; + out.println("--- Epochs and Transformations ---"); + + for (Entry entry : filterEntries(data)) + { + out.printf("Epoch %d: %s%n", entry.epoch.getEpoch(), entry.transform.kind()); + + if (transformations && verbose) + { + out.printf(" Entry ID: %d%n", entry.id.entryId); + out.printf(" Transform: %s%n", entry.transform); + } + } + out.println(); + } + + private void outputDirectoryText(ClusterMetadata metadata) + { + PrintStream out = output.out; + out.println("--- Directory (Node Membership) ---"); + + metadata.directory.peerIds().forEach(nodeId -> { + out.printf("NodeId: %s%n", nodeId); + out.printf(" Address: %s%n", metadata.directory.endpoint(nodeId)); + out.printf(" State: %s%n", metadata.directory.peerState(nodeId)); + out.printf(" DC/Rack: %s/%s%n", + metadata.directory.location(nodeId).datacenter, + metadata.directory.location(nodeId).rack); + out.printf(" Host ID: %s%n", metadata.directory.hostId(nodeId)); + out.println(); + }); + } + + private void outputTokensText(ClusterMetadata metadata) + { + PrintStream out = output.out; + out.println("--- Token Map ---"); + + metadata.directory.peerIds().forEach(nodeId -> { + List nodeTokens = metadata.tokenMap.tokens(nodeId); + out.printf("NodeId %s: %d tokens%n", nodeId, nodeTokens.size()); + if (verbose) + { + nodeTokens.forEach(token -> out.printf(" %s%n", token)); + } + }); + out.println(); + } + + private void outputSchemaText(ClusterMetadata metadata) + { + PrintStream out = output.out; + out.println("--- Schema ---"); + + metadata.schema.getKeyspaces().forEach(ks -> { + out.printf("Keyspace: %s%n", ks.name); + ks.tables.forEach(table -> { + out.printf(" Table: %s (id: %s)%n", table.name, table.id); + }); + out.println(); + }); + } + + private void outputSnapshotsText(DumpData data) + { + PrintStream out = output.out; + out.println("--- Available Snapshots ---"); + + if (data.snapshotEpochs.isEmpty()) + { + out.println("No snapshots found"); + } + else + { + data.snapshotEpochs.forEach(epoch -> out.printf("Snapshot at epoch: %d%n", epoch.getEpoch())); + } + out.println(); + } + } +} diff --git a/test/unit/org/apache/cassandra/tools/SSTableMetadataDumpTest.java b/test/unit/org/apache/cassandra/tools/SSTableMetadataDumpTest.java new file mode 100644 index 000000000000..2bb926495538 --- /dev/null +++ b/test/unit/org/apache/cassandra/tools/SSTableMetadataDumpTest.java @@ -0,0 +1,202 @@ +/* + * 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.tools; + +import org.assertj.core.api.Assertions; +import org.hamcrest.CoreMatchers; +import org.junit.Test; + +import org.apache.cassandra.tools.ToolRunner.ToolResult; + +import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertTrue; + +/** + * Tests for SSTableMetadataDump tool. + *

+ * Note: This tool requires some initialization (DatabaseDescriptor, Schema) even for help, + * similar to StandaloneJournalUtil and other TCM-related tools. + */ +public class SSTableMetadataDumpTest extends OfflineToolUtils +{ + @Test + public void testMainHelpOption() + { + // Main command help shows subcommands + ToolResult tool = ToolRunner.invokeClass(SSTableMetadataDump.class, "-h"); + String output = tool.getStdout() + tool.getStderr(); + assertThat("Help should show usage", output, CoreMatchers.containsStringIgnoringCase("Usage:")); + assertThat("Help should mention dump subcommand", output, CoreMatchers.containsStringIgnoringCase("dump")); + } + + @Test + public void testDumpSubcommandHelpOption() + { + // Dump subcommand help shows all the options + ToolResult tool = ToolRunner.invokeClass(SSTableMetadataDump.class, "dump", "-h"); + String output = tool.getStdout() + tool.getStderr(); + + assertThat("Help should show usage", output, CoreMatchers.containsStringIgnoringCase("Usage:")); + // Check for key options + Assertions.assertThat(output).containsIgnoringCase("--epochs"); + Assertions.assertThat(output).containsIgnoringCase("--schema"); + Assertions.assertThat(output).containsIgnoringCase("--directory"); + Assertions.assertThat(output).containsIgnoringCase("--tokens"); + Assertions.assertThat(output).containsIgnoringCase("--all"); + } + + @Test + public void testMaybeChangeDocs() + { + // If you added, modified options or help, please update docs if necessary + ToolResult tool = ToolRunner.invokeClass(SSTableMetadataDump.class, "dump", "-h"); + String output = tool.getStdout() + tool.getStderr(); + + // Verify key options are documented + Assertions.assertThat(output).containsIgnoringCase("--data-dir"); + Assertions.assertThat(output).containsIgnoringCase("--output"); + Assertions.assertThat(output).containsIgnoringCase("--to-string"); + Assertions.assertThat(output).containsIgnoringCase("--text"); + Assertions.assertThat(output).containsIgnoringCase("--epochs"); + Assertions.assertThat(output).containsIgnoringCase("--schema"); + Assertions.assertThat(output).containsIgnoringCase("--directory"); + Assertions.assertThat(output).containsIgnoringCase("--tokens"); + Assertions.assertThat(output).containsIgnoringCase("--snapshots"); + Assertions.assertThat(output).containsIgnoringCase("--transformations"); + Assertions.assertThat(output).containsIgnoringCase("--all"); + Assertions.assertThat(output).containsIgnoringCase("--epoch"); + Assertions.assertThat(output).containsIgnoringCase("--from-epoch"); + Assertions.assertThat(output).containsIgnoringCase("--to-epoch"); + Assertions.assertThat(output).containsIgnoringCase("--verbose"); + Assertions.assertThat(output).containsIgnoringCase("--debug"); + } + + @Test + public void testWrongArgFailsAndPrintsHelp() + { + ToolResult tool = ToolRunner.invokeClass(SSTableMetadataDump.class, "dump", "--invalid-option"); + String output = tool.getStdout() + tool.getStderr(); + assertThat("Should mention unknown option", output, CoreMatchers.containsStringIgnoringCase("Unknown")); + assertTrue("Expected non-zero exit code", tool.getExitCode() != 0); + } + + @Test + public void testNonExistentDataDirectory() + { + // When running with a non-existent directory, should fail gracefully + ToolResult tool = ToolRunner.invokeClass(SSTableMetadataDump.class, "dump", + "--data-dir", "/nonexistent/path/to/data", + "--all"); + String output = tool.getStdout() + tool.getStderr(); + // Tool should fail gracefully when directory doesn't exist or no SSTables found + assertTrue("Expected error or no sstables message", + tool.getExitCode() != 0 || + output.toLowerCase().contains("no sstables") || + output.toLowerCase().contains("not found") || + output.toLowerCase().contains("does not exist") || + output.toLowerCase().contains("error")); + } + + @Test + public void testOutputModeFlags() + { + // Test that --to-string flag is recognized + ToolResult toStringFlag = ToolRunner.invokeClass(SSTableMetadataDump.class, "dump", "--to-string", "-h"); + String toStringOutput = toStringFlag.getStdout() + toStringFlag.getStderr(); + assertThat("Should show help with --to-string", toStringOutput, CoreMatchers.containsStringIgnoringCase("Usage:")); + + // Test that --text flag is recognized + ToolResult textFlag = ToolRunner.invokeClass(SSTableMetadataDump.class, "dump", "--text", "-h"); + String textOutput = textFlag.getStdout() + textFlag.getStderr(); + assertThat("Should show help with --text", textOutput, CoreMatchers.containsStringIgnoringCase("Usage:")); + + // Test that -o/--output flag is recognized + ToolResult outputFlag = ToolRunner.invokeClass(SSTableMetadataDump.class, "dump", "-o", "/tmp/test.dump", "-h"); + String outputOutput = outputFlag.getStdout() + outputFlag.getStderr(); + assertThat("Should show help with -o", outputOutput, CoreMatchers.containsStringIgnoringCase("Usage:")); + + ToolResult outputLongFlag = ToolRunner.invokeClass(SSTableMetadataDump.class, "dump", "--output", "/tmp/test.dump", "-h"); + String outputLongOutput = outputLongFlag.getStdout() + outputLongFlag.getStderr(); + assertThat("Should show help with --output", outputLongOutput, CoreMatchers.containsStringIgnoringCase("Usage:")); + } + + @Test + public void testScopeFlagsRecognized() + { + // Test that all scope flags are recognized (combined with -h to avoid needing real data) + String[] scopeFlags = {"--epochs", "--schema", "--directory", "--tokens", "--snapshots", "--transformations", "--all"}; + + for (String flag : scopeFlags) + { + ToolResult tool = ToolRunner.invokeClass(SSTableMetadataDump.class, "dump", flag, "-h"); + String output = tool.getStdout() + tool.getStderr(); + assertThat("Flag " + flag + " should be recognized", output, CoreMatchers.containsStringIgnoringCase("Usage:")); + } + } + + @Test + public void testEpochFilterFlags() + { + // Test that epoch filter flags are recognized + ToolResult epochTool = ToolRunner.invokeClass(SSTableMetadataDump.class, "dump", "--epoch", "100", "-h"); + String epochOutput = epochTool.getStdout() + epochTool.getStderr(); + assertThat("--epoch flag should be recognized", epochOutput, CoreMatchers.containsStringIgnoringCase("Usage:")); + + ToolResult fromTool = ToolRunner.invokeClass(SSTableMetadataDump.class, "dump", "--from-epoch", "50", "-h"); + String fromOutput = fromTool.getStdout() + fromTool.getStderr(); + assertThat("--from-epoch flag should be recognized", fromOutput, CoreMatchers.containsStringIgnoringCase("Usage:")); + + ToolResult toTool = ToolRunner.invokeClass(SSTableMetadataDump.class, "dump", "--to-epoch", "150", "-h"); + String toOutput = toTool.getStdout() + toTool.getStderr(); + assertThat("--to-epoch flag should be recognized", toOutput, CoreMatchers.containsStringIgnoringCase("Usage:")); + } + + @Test + public void testVerboseAndDebugFlags() + { + // Test verbose flags + ToolResult verboseShort = ToolRunner.invokeClass(SSTableMetadataDump.class, "dump", "-v", "-h"); + String verboseShortOutput = verboseShort.getStdout() + verboseShort.getStderr(); + assertThat("-v flag should be recognized", verboseShortOutput, CoreMatchers.containsStringIgnoringCase("Usage:")); + + ToolResult verboseLong = ToolRunner.invokeClass(SSTableMetadataDump.class, "dump", "--verbose", "-h"); + String verboseLongOutput = verboseLong.getStdout() + verboseLong.getStderr(); + assertThat("--verbose flag should be recognized", verboseLongOutput, CoreMatchers.containsStringIgnoringCase("Usage:")); + + // Test debug flag + ToolResult debug = ToolRunner.invokeClass(SSTableMetadataDump.class, "dump", "--debug", "-h"); + String debugOutput = debug.getStdout() + debug.getStderr(); + assertThat("--debug flag should be recognized", debugOutput, CoreMatchers.containsStringIgnoringCase("Usage:")); + } + + @Test + public void testPartitionerFlag() + { + // Test partitioner flags + ToolResult shortFlag = ToolRunner.invokeClass(SSTableMetadataDump.class, "dump", + "-p", "org.apache.cassandra.dht.Murmur3Partitioner", "-h"); + String shortOutput = shortFlag.getStdout() + shortFlag.getStderr(); + assertThat("-p flag should be recognized", shortOutput, CoreMatchers.containsStringIgnoringCase("Usage:")); + + ToolResult longFlag = ToolRunner.invokeClass(SSTableMetadataDump.class, "dump", + "--partitioner", "org.apache.cassandra.dht.Murmur3Partitioner", "-h"); + String longOutput = longFlag.getStdout() + longFlag.getStderr(); + assertThat("--partitioner flag should be recognized", longOutput, CoreMatchers.containsStringIgnoringCase("Usage:")); + } +}