You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by br...@apache.org on 2020/11/17 20:02:51 UTC

[cassandra] branch trunk updated: sstablescrub unit test hardening and docs

This is an automated email from the ASF dual-hosted git repository.

brandonwilliams pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/cassandra.git


The following commit(s) were added to refs/heads/trunk by this push:
     new a04ccf3  sstablescrub unit test hardening and docs
a04ccf3 is described below

commit a04ccf3297839febed68c314704db4d920b64413
Author: Brandon Williams <br...@apache.org>
AuthorDate: Tue Nov 17 14:01:28 2020 -0600

    sstablescrub unit test hardening and docs
    
    Patch by Berenguer Blasi, reviewed by brandonwilliams for
    CASSANDRA-16013
---
 doc/source/tools/sstable/sstablescrub.rst          |   7 +
 .../apache/cassandra/tools/StandaloneScrubber.java |  40 +++--
 src/java/org/apache/cassandra/tools/Util.java      |   2 +-
 .../cql3/validation/operations/TTLTest.java        | 101 ++++++++----
 test/unit/org/apache/cassandra/db/ScrubTest.java   | 176 ++++++++++++++++++++-
 .../cassandra/tools/StandaloneScrubberTest.java    |   6 +
 6 files changed, 284 insertions(+), 48 deletions(-)

diff --git a/doc/source/tools/sstable/sstablescrub.rst b/doc/source/tools/sstable/sstablescrub.rst
index 0bbda9f..a8529e3 100644
--- a/doc/source/tools/sstable/sstablescrub.rst
+++ b/doc/source/tools/sstable/sstablescrub.rst
@@ -29,6 +29,13 @@ sstablescrub <options> <keyspace> <table>
 
 ===================================     ================================================================================
 --debug                                 display stack traces
+-e,--header-fix <arg>                   Option whether and how to perform a check of the sstable serialization-headers and fix , fixable issues.
+                                        Possible argument values:
+                                         - validate-only: validate the serialization-headers, but do not fix those. Do not continue with scrub - i.e. only validate the header (dry-run of fix-only).
+                                         - validate: (default) validate the serialization-headers, but do not fix those and only continue with scrub if no error were detected.
+                                         - fix-only: validate and fix the serialization-headers, don't continue with scrub.
+                                         - fix: validate and fix the serialization-headers, do not fix and do not continue with scrub if the serialization-header check encountered errors.
+                                         - off: don't perform the serialization-header checks.
 -h,--help                               display this help message
 -m,--manifest-check                     only check and repair the leveled manifest, without actually scrubbing the sstables
 -n,--no-validate                        do not validate columns using column validator
diff --git a/src/java/org/apache/cassandra/tools/StandaloneScrubber.java b/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
index e2fedad..bd71c64 100644
--- a/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
+++ b/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
@@ -20,29 +20,41 @@ package org.apache.cassandra.tools;
 
 import java.io.File;
 import java.nio.file.Paths;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
 import java.util.concurrent.TimeUnit;
 
-import com.google.common.base.Predicate;
-import com.google.common.base.Predicates;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Lists;
-import org.apache.commons.cli.*;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.CommandLineParser;
+import org.apache.commons.cli.GnuParser;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.ParseException;
 
-import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Directories;
 import org.apache.cassandra.db.Keyspace;
-import org.apache.cassandra.db.compaction.*;
+import org.apache.cassandra.db.compaction.AbstractStrategyHolder;
+import org.apache.cassandra.db.compaction.CompactionManager;
+import org.apache.cassandra.db.compaction.CompactionStrategyManager;
+import org.apache.cassandra.db.compaction.LeveledCompactionStrategy;
+import org.apache.cassandra.db.compaction.LeveledManifest;
+import org.apache.cassandra.db.compaction.OperationType;
+import org.apache.cassandra.db.compaction.Scrubber;
 import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.SSTableHeaderFix;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
-import org.apache.cassandra.io.sstable.*;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.tools.BulkLoader.CmdLineOptions;
 import org.apache.cassandra.utils.JVMStabilityInspector;
 import org.apache.cassandra.utils.OutputHandler;
 import org.apache.cassandra.utils.Pair;
 
-import static org.apache.cassandra.tools.BulkLoader.CmdLineOptions;
-
 public class StandaloneScrubber
 {
     public static final String REINSERT_OVERFLOWED_TTL_OPTION_DESCRIPTION = "Rewrites rows with overflowed expiration date affected by CASSANDRA-14092 with " +
@@ -64,7 +76,11 @@ public class StandaloneScrubber
     public static void main(String args[])
     {
         Options options = Options.parseArgs(args);
-        Util.initDatabaseDescriptor();
+
+        if (Boolean.getBoolean(Util.ALLOW_TOOL_REINIT_FOR_TEST))
+            DatabaseDescriptor.toolInitialization(false); //Necessary for testing
+        else
+            Util.initDatabaseDescriptor();
 
         try
         {
diff --git a/src/java/org/apache/cassandra/tools/Util.java b/src/java/org/apache/cassandra/tools/Util.java
index 7411858..3757754 100644
--- a/src/java/org/apache/cassandra/tools/Util.java
+++ b/src/java/org/apache/cassandra/tools/Util.java
@@ -54,7 +54,7 @@ import com.google.common.collect.Lists;
 @SuppressWarnings("serial")
 public final class Util
 {
-    static final String ALLOW_TOOL_REINIT_FOR_TEST = Util.class.getName() + "ALLOW_TOOL_REINIT_FOR_TEST"; // Necessary for testing
+    public static final String ALLOW_TOOL_REINIT_FOR_TEST = Util.class.getName() + "ALLOW_TOOL_REINIT_FOR_TEST"; // Necessary for testing
     static final String RESET = "\u001B[0m";
     static final String BLUE = "\u001B[34m";
     static final String CYAN = "\u001B[36m";
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/TTLTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/TTLTest.java
index 85503d0..6f73e83 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/TTLTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/TTLTest.java
@@ -23,10 +23,12 @@ import java.io.FileInputStream;
 import java.io.FileOutputStream;
 import java.io.IOException;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
 
+import org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.config.Config;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.cql3.Attributes;
@@ -37,12 +39,17 @@ import org.apache.cassandra.db.ExpirationDateOverflowHandling;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.rows.AbstractCell;
 import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.tools.StandaloneScrubber;
+import org.apache.cassandra.tools.ToolRunner;
+import org.apache.cassandra.tools.ToolRunner.ToolResult;
 import org.apache.cassandra.utils.FBUtilities;
+import org.assertj.core.api.Assertions;
 
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
+@RunWith(OrderedJUnit4ClassRunner.class)
 public class TTLTest extends CQLTester
 {
     public static String NEGATIVE_LOCAL_EXPIRATION_TEST_DIR = "test/data/negative-local-expiration-test/%s";
@@ -54,6 +61,7 @@ public class TTLTest extends CQLTester
     public static final String COMPLEX_NOCLUSTERING = "table3";
     public static final String COMPLEX_CLUSTERING = "table4";
     private Config.CorruptedTombstoneStrategy corruptTombstoneStrategy;
+
     @Before
     public void before()
     {
@@ -203,9 +211,20 @@ public class TTLTest extends CQLTester
     {
         // this tests writes corrupt tombstones on purpose, disable the strategy:
         DatabaseDescriptor.setCorruptedTombstoneStrategy(Config.CorruptedTombstoneStrategy.disabled);
-        baseTestRecoverOverflowedExpiration(false, false);
-        baseTestRecoverOverflowedExpiration(true, false);
-        baseTestRecoverOverflowedExpiration(true, true);
+        baseTestRecoverOverflowedExpiration(false, false, false);
+        baseTestRecoverOverflowedExpiration(true, false, false);
+        baseTestRecoverOverflowedExpiration(true, false, true);
+        // we reset the corrupted ts strategy after each test in @After above
+    }
+
+    @Test
+    public void testRecoverOverflowedExpirationWithSSTableScrub() throws Throwable
+    {
+        // this tests writes corrupt tombstones on purpose, disable the strategy:
+        DatabaseDescriptor.setCorruptedTombstoneStrategy(Config.CorruptedTombstoneStrategy.disabled);
+        baseTestRecoverOverflowedExpiration(false, false, false);
+        baseTestRecoverOverflowedExpiration(false, true, false);
+        baseTestRecoverOverflowedExpiration(false, true, true);
         // we reset the corrupted ts strategy after each test in @After above
     }
 
@@ -276,16 +295,16 @@ public class TTLTest extends CQLTester
         }
     }
 
-    public void baseTestRecoverOverflowedExpiration(boolean runScrub, boolean reinsertOverflowedTTL) throws Throwable
+    public void baseTestRecoverOverflowedExpiration(boolean runScrub, boolean runSStableScrub, boolean reinsertOverflowedTTL) throws Throwable
     {
         // simple column, clustering
-        testRecoverOverflowedExpirationWithScrub(true, true, runScrub, reinsertOverflowedTTL);
+        testRecoverOverflowedExpirationWithScrub(true, true, runScrub, runSStableScrub, reinsertOverflowedTTL);
         // simple column, noclustering
-        testRecoverOverflowedExpirationWithScrub(true, false, runScrub, reinsertOverflowedTTL);
+        testRecoverOverflowedExpirationWithScrub(true, false, runScrub, runSStableScrub, reinsertOverflowedTTL);
         // complex column, clustering
-        testRecoverOverflowedExpirationWithScrub(false, true, runScrub, reinsertOverflowedTTL);
+        testRecoverOverflowedExpirationWithScrub(false, true, runScrub, runSStableScrub, reinsertOverflowedTTL);
         // complex column, noclustering
-        testRecoverOverflowedExpirationWithScrub(false, false, runScrub, reinsertOverflowedTTL);
+        testRecoverOverflowedExpirationWithScrub(false, false, runScrub, runSStableScrub, reinsertOverflowedTTL);
     }
 
     private void createTable(boolean simple, boolean clustering)
@@ -350,11 +369,11 @@ public class TTLTest extends CQLTester
         return AbstractCell.MAX_DELETION_TIME - nowInSecs;
     }
 
-    public void testRecoverOverflowedExpirationWithScrub(boolean simple, boolean clustering, boolean runScrub, boolean reinsertOverflowedTTL) throws Throwable
+    public void testRecoverOverflowedExpirationWithScrub(boolean simple, boolean clustering, boolean runScrub, boolean runSStableScrub,  boolean reinsertOverflowedTTL) throws Throwable
     {
         if (reinsertOverflowedTTL)
         {
-            assert runScrub;
+            assert runScrub || runSStableScrub;
         }
 
         createTable(simple, clustering);
@@ -371,25 +390,49 @@ public class TTLTest extends CQLTester
         if (runScrub)
         {
             cfs.scrub(true, false, true, reinsertOverflowedTTL, 1);
-        }
 
-        if (reinsertOverflowedTTL)
-        {
-            if (simple)
-                assertRows(execute("SELECT * from %s"), row(1, 1, 1), row(2, 2, null));
-            else
-                assertRows(execute("SELECT * from %s"), row(1, 1, set("v11", "v12", "v13", "v14")), row(2, 2, set("v21", "v22", "v23", "v24")));
+            if (reinsertOverflowedTTL)
+            {
+                if (simple)
+                    assertRows(execute("SELECT * from %s"), row(1, 1, 1), row(2, 2, null));
+                else
+                    assertRows(execute("SELECT * from %s"), row(1, 1, set("v11", "v12", "v13", "v14")), row(2, 2, set("v21", "v22", "v23", "v24")));
 
-            cfs.forceMajorCompaction();
+                cfs.forceMajorCompaction();
 
-            if (simple)
-                assertRows(execute("SELECT * from %s"), row(1, 1, 1), row(2, 2, null));
+                if (simple)
+                    assertRows(execute("SELECT * from %s"), row(1, 1, 1), row(2, 2, null));
+                else
+                    assertRows(execute("SELECT * from %s"), row(1, 1, set("v11", "v12", "v13", "v14")), row(2, 2, set("v21", "v22", "v23", "v24")));
+            }
             else
-                assertRows(execute("SELECT * from %s"), row(1, 1, set("v11", "v12", "v13", "v14")), row(2, 2, set("v21", "v22", "v23", "v24")));
+            {
+                assertEmpty(execute("SELECT * from %s"));
+            }
         }
-        else
+        if (runSStableScrub)
         {
-            assertEmpty(execute("SELECT * from %s"));
+            System.setProperty(org.apache.cassandra.tools.Util.ALLOW_TOOL_REINIT_FOR_TEST, "true"); // Necessary for testing
+
+            try
+            {
+                ToolResult tool;
+                if (reinsertOverflowedTTL)
+                    tool = ToolRunner.invokeClass(StandaloneScrubber.class, "-r", KEYSPACE, cfs.name);
+                else
+                    tool = ToolRunner.invokeClass(StandaloneScrubber.class, KEYSPACE, cfs.name);
+
+                Assertions.assertThat(tool.getStdout()).contains("Pre-scrub sstables snapshotted into");
+                if (reinsertOverflowedTTL)
+                    Assertions.assertThat(tool.getStdout()).contains("Fixed 2 rows with overflowed local deletion time.");
+                else
+                    Assertions.assertThat(tool.getStdout()).contains("Unable to recover 2 rows that were skipped.");
+                tool.assertOnCleanExit();
+            }
+            finally
+            {
+                System.clearProperty(org.apache.cassandra.tools.Util.ALLOW_TOOL_REINIT_FOR_TEST);
+            }
         }
     }
 
diff --git a/test/unit/org/apache/cassandra/db/ScrubTest.java b/test/unit/org/apache/cassandra/db/ScrubTest.java
index 533b490..6ecaded 100644
--- a/test/unit/org/apache/cassandra/db/ScrubTest.java
+++ b/test/unit/org/apache/cassandra/db/ScrubTest.java
@@ -18,21 +18,34 @@
  */
 package org.apache.cassandra.db;
 
-import java.io.*;
+import java.io.File;
+import java.io.IOError;
+import java.io.IOException;
+import java.io.RandomAccessFile;
 import java.nio.ByteBuffer;
 import java.nio.file.Files;
 import java.nio.file.Path;
 import java.nio.file.Paths;
-import java.util.*;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
 import java.util.concurrent.ExecutionException;
 
-import org.apache.cassandra.db.lifecycle.LifecycleNewTracker;
 import org.apache.commons.lang3.StringUtils;
+
+import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 
-import org.apache.cassandra.*;
+import org.apache.cassandra.OrderedJUnit4ClassRunner;
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.UpdateBuilder;
+import org.apache.cassandra.Util;
 import org.apache.cassandra.cache.ChunkCache;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.cql3.Operator;
@@ -41,18 +54,27 @@ import org.apache.cassandra.cql3.UntypedResultSet;
 import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.db.compaction.OperationType;
 import org.apache.cassandra.db.compaction.Scrubber;
+import org.apache.cassandra.db.lifecycle.LifecycleNewTracker;
 import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 import org.apache.cassandra.db.marshal.LongType;
 import org.apache.cassandra.db.marshal.UUIDType;
 import org.apache.cassandra.db.partitions.Partition;
 import org.apache.cassandra.db.partitions.PartitionUpdate;
 import org.apache.cassandra.db.rows.EncodingStats;
-import org.apache.cassandra.dht.*;
+import org.apache.cassandra.dht.ByteOrderedPartitioner;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.dht.Murmur3Partitioner;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.RequestExecutionException;
 import org.apache.cassandra.exceptions.WriteTimeoutException;
 import org.apache.cassandra.io.compress.CompressionMetadata;
-import org.apache.cassandra.io.sstable.*;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.sstable.SSTableMultiWriter;
+import org.apache.cassandra.io.sstable.SSTableRewriter;
+import org.apache.cassandra.io.sstable.SSTableTxnWriter;
+import org.apache.cassandra.io.sstable.SimpleSSTableMultiWriter;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.sstable.format.SSTableWriter;
 import org.apache.cassandra.io.sstable.format.big.BigTableWriter;
@@ -60,7 +82,11 @@ import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.schema.KeyspaceParams;
 import org.apache.cassandra.schema.TableMetadataRef;
+import org.apache.cassandra.tools.StandaloneScrubber;
+import org.apache.cassandra.tools.ToolRunner;
+import org.apache.cassandra.tools.ToolRunner.ToolResult;
 import org.apache.cassandra.utils.ByteBufferUtil;
+import org.assertj.core.api.Assertions;
 
 import static org.apache.cassandra.SchemaLoader.counterCFMD;
 import static org.apache.cassandra.SchemaLoader.createKeyspace;
@@ -79,9 +105,11 @@ public class ScrubTest
     public static final String INVALID_LEGACY_SSTABLE_ROOT_PROP = "invalid-legacy-sstable-root";
 
     public static final String KEYSPACE = "Keyspace1";
+    public static final String KEYSPACE2 = "Keyspace2";
     public static final String CF = "Standard1";
     public static final String CF2 = "Standard2";
     public static final String CF3 = "Standard3";
+    public static final String CF4 = "Standard4";
     public static final String COUNTER_CF = "Counter1";
     public static final String CF_UUID = "UUIDKeys";
     public static final String CF_INDEX1 = "Indexed1";
@@ -110,6 +138,12 @@ public class ScrubTest
                        SchemaLoader.compositeIndexCFMD(KEYSPACE, CF_INDEX2_BYTEORDERED, true).partitioner(ByteOrderedPartitioner.instance));
     }
 
+    @AfterClass
+    public static void clearClassEnv()
+    {
+        System.clearProperty(org.apache.cassandra.tools.Util.ALLOW_TOOL_REINIT_FOR_TEST);
+    }
+
     @Test
     public void testScrubOneRow() throws ExecutionException, InterruptedException
     {
@@ -712,4 +746,134 @@ public class ScrubTest
         rs = QueryProcessor.executeInternal(String.format("SELECT * FROM \"%s\".cf_with_duplicates_3_0", KEYSPACE));
         assertEquals(0, rs.size());
     }
+
+    @Test
+    public void testToolTestingEnvSetup()
+    {
+        createKeyspace(KEYSPACE2,
+                       KeyspaceParams.simple(1),
+                       standardCFMD(KEYSPACE2, CF),
+                       standardCFMD(KEYSPACE2, CF2),
+                       standardCFMD(KEYSPACE2, CF3),
+                       standardCFMD(KEYSPACE2, CF4),
+                       counterCFMD(KEYSPACE2, COUNTER_CF));
+
+        System.setProperty(org.apache.cassandra.tools.Util.ALLOW_TOOL_REINIT_FOR_TEST, "true"); // Necessary for testing
+    }
+
+    @Test
+    public void testScrubOneRowWithTool()
+    {
+        CompactionManager.instance.disableAutoCompaction();
+        Keyspace keyspace = Keyspace.open(KEYSPACE2);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF);
+        cfs.clearUnsafe();
+
+        // insert data and verify we get it back w/ range query
+        fillCF(cfs, 1);
+        assertOrderedAll(cfs, 1);
+
+        ToolResult tool = ToolRunner.invokeClass(StandaloneScrubber.class, KEYSPACE2, CF);
+        Assertions.assertThat(tool.getStdout()).contains("Pre-scrub sstables snapshotted into");
+        Assertions.assertThat(tool.getStdout()).contains("1 rows in new sstable and 0 empty");
+        tool.assertOnCleanExit();
+
+        // check data is still there
+        assertOrderedAll(cfs, 1);
+    }
+
+    @Test
+    public void testSkipScrubCorruptedCounterRowWithTool() throws IOException, WriteTimeoutException
+    {
+        int numPartitions = 1000;
+
+        CompactionManager.instance.disableAutoCompaction();
+        Keyspace keyspace = Keyspace.open(KEYSPACE2);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(COUNTER_CF);
+        cfs.clearUnsafe();
+
+        fillCounterCF(cfs, numPartitions);
+        assertOrderedAll(cfs, numPartitions);
+        assertEquals(1, cfs.getLiveSSTables().size());
+        SSTableReader sstable = cfs.getLiveSSTables().iterator().next();
+
+        overrideWithGarbage(sstable, ByteBufferUtil.bytes("0"), ByteBufferUtil.bytes("1"));
+
+        // with skipCorrupted == false, the scrub is expected to fail
+        try
+        {
+            ToolRunner.invokeClass(StandaloneScrubber.class, KEYSPACE2, COUNTER_CF);
+            fail("Expected a CorruptSSTableException to be thrown");
+        }
+        catch (IOError err) {}
+
+        // with skipCorrupted == true, the corrupt rows will be skipped
+        ToolResult tool = ToolRunner.invokeClass(StandaloneScrubber.class, "-s", KEYSPACE2, COUNTER_CF);
+        Assertions.assertThat(tool.getStdout()).contains("0 empty");
+        Assertions.assertThat(tool.getStdout()).contains("rows that were skipped");
+        tool.assertOnCleanExit();
+
+        assertEquals(1, cfs.getLiveSSTables().size());
+    }
+
+    @Test
+    public void testNoCheckScrubMultiRowWithTool()
+    {
+        CompactionManager.instance.disableAutoCompaction();
+        Keyspace keyspace = Keyspace.open(KEYSPACE2);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF2);
+        cfs.clearUnsafe();
+
+        // insert data and verify we get it back w/ range query
+        fillCF(cfs, 10);
+        assertOrderedAll(cfs, 10);
+
+        ToolResult tool = ToolRunner.invokeClass(StandaloneScrubber.class, "-n", KEYSPACE2, CF2);
+        Assertions.assertThat(tool.getStdout()).contains("Pre-scrub sstables snapshotted into");
+        Assertions.assertThat(tool.getStdout()).contains("10 rows in new sstable and 0 empty");
+        tool.assertOnCleanExit();
+
+        // check data is still there
+        assertOrderedAll(cfs, 10);
+    }
+
+    @Test
+    public void testHeaderFixWithTool()
+    {
+        CompactionManager.instance.disableAutoCompaction();
+        Keyspace keyspace = Keyspace.open(KEYSPACE2);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF3);
+        cfs.clearUnsafe();
+
+        fillCF(cfs, 1);
+        assertOrderedAll(cfs, 1);
+
+        ToolResult tool = ToolRunner.invokeClass(StandaloneScrubber.class, "-e", "validate_only", KEYSPACE2, CF3);
+        Assertions.assertThat(tool.getStdout()).contains("Not continuing with scrub, since '--header-fix validate-only' was specified.");
+        tool.assertOnCleanExit();
+        assertOrderedAll(cfs, 1);
+
+        tool = ToolRunner.invokeClass(StandaloneScrubber.class, "-e", "validate", KEYSPACE2, CF3);
+        Assertions.assertThat(tool.getStdout()).contains("Pre-scrub sstables snapshotted into");
+        Assertions.assertThat(tool.getStdout()).contains("1 rows in new sstable and 0 empty");
+        tool.assertOnCleanExit();
+        assertOrderedAll(cfs, 1);
+
+        tool = ToolRunner.invokeClass(StandaloneScrubber.class, "-e", "fix-only", KEYSPACE2, CF3);
+        Assertions.assertThat(tool.getStdout()).contains("Not continuing with scrub, since '--header-fix fix-only' was specified.");
+        tool.assertOnCleanExit();
+        assertOrderedAll(cfs, 1);
+
+        tool = ToolRunner.invokeClass(StandaloneScrubber.class, "-e", "fix", KEYSPACE2, CF3);
+        Assertions.assertThat(tool.getStdout()).contains("Pre-scrub sstables snapshotted into");
+        Assertions.assertThat(tool.getStdout()).contains("1 rows in new sstable and 0 empty");
+        tool.assertOnCleanExit();
+        assertOrderedAll(cfs, 1);
+
+        tool = ToolRunner.invokeClass(StandaloneScrubber.class, "-e", "off", KEYSPACE2, CF3);
+        Assertions.assertThat(tool.getStdout()).contains("Pre-scrub sstables snapshotted into");
+        Assertions.assertThat(tool.getStdout()).contains("1 rows in new sstable and 0 empty");
+        tool.assertOnCleanExit();
+        assertOrderedAll(cfs, 1);
+    }
 }
diff --git a/test/unit/org/apache/cassandra/tools/StandaloneScrubberTest.java b/test/unit/org/apache/cassandra/tools/StandaloneScrubberTest.java
index f0b996f..ea1499e 100644
--- a/test/unit/org/apache/cassandra/tools/StandaloneScrubberTest.java
+++ b/test/unit/org/apache/cassandra/tools/StandaloneScrubberTest.java
@@ -34,6 +34,12 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 
+/*
+ * We are testing cmd line params here.
+ * For sstable scrubbing tests look at {@link ScrubTest}
+ * For TTL sstable scrubbing tests look at {@link TTLTest}
+ */
+
 @RunWith(OrderedJUnit4ClassRunner.class)
 public class StandaloneScrubberTest extends OfflineToolUtils
 {


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org