You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by ma...@apache.org on 2021/09/20 12:25:49 UTC

[cassandra] branch cassandra-4.0 updated: Remove OrderedJUnit4ClassRunner

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

marcuse pushed a commit to branch cassandra-4.0
in repository https://gitbox.apache.org/repos/asf/cassandra.git


The following commit(s) were added to refs/heads/cassandra-4.0 by this push:
     new 9e9dffb  Remove OrderedJUnit4ClassRunner
9e9dffb is described below

commit 9e9dffb70439e3e09e3da5515b7687b449b5ea76
Author: Marcus Eriksson <ma...@apache.org>
AuthorDate: Thu Sep 9 19:55:22 2021 +0200

    Remove OrderedJUnit4ClassRunner
    
    Patch by marcuse; reviewed by Caleb Rackliffe for CASSANDRA-16942
---
 ide/idea/workspace.xml                             |   1 +
 .../db/compaction/CompactionAllocationTest.java    |   4 -
 .../apache/cassandra/OrderedJUnit4ClassRunner.java |  55 -----
 .../cassandra/audit/AuditLoggerAuthTest.java       |   4 +-
 .../cassandra/config/DatabaseDescriptorTest.java   |   3 -
 .../cql3/validation/operations/AlterNTSTest.java   | 103 +++++++++
 .../cql3/validation/operations/AlterTest.java      |  77 -------
 .../cql3/validation/operations/TTLTest.java        |  12 +-
 .../apache/cassandra/db/ColumnFamilyStoreTest.java |   4 +-
 test/unit/org/apache/cassandra/db/ScrubTest.java   | 238 ++++----------------
 .../org/apache/cassandra/db/ScrubToolTest.java     | 249 +++++++++++++++++++++
 test/unit/org/apache/cassandra/db/VerifyTest.java  |   6 -
 .../cassandra/db/compaction/CompactionsTest.java   |   4 -
 .../compaction/LeveledCompactionStrategyTest.java  |   3 -
 .../cassandra/db/compaction/TTLExpiryTest.java     |   3 -
 .../org/apache/cassandra/dht/BootStrapperTest.java |   3 -
 .../cassandra/diag/DiagnosticEventServiceTest.java |   5 +-
 .../io/sstable/IndexSummaryManagerTest.java        |   6 -
 .../cassandra/io/sstable/LargePartitionsTest.java  |   3 -
 .../cassandra/locator/TokenMetadataTest.java       |   3 -
 .../apache/cassandra/metrics/BatchMetricsTest.java |   3 -
 .../cassandra/metrics/BufferPoolMetricsTest.java   |   3 -
 .../apache/cassandra/metrics/CQLMetricsTest.java   |   3 -
 .../apache/cassandra/metrics/CacheMetricsTest.java |   3 -
 .../apache/cassandra/metrics/TableMetricsTest.java |   5 +-
 .../schema/MigrationManagerDropKSTest.java         | 106 +++++++++
 .../cassandra/schema/MigrationManagerTest.java     |  53 -----
 .../apache/cassandra/service/GCInspectorTest.java  |   3 -
 .../apache/cassandra/service/QueryPagerTest.java   |   3 -
 .../service/StorageServiceServerM3PTest.java       |  68 ++++++
 .../service/StorageServiceServerTest.java          |  25 +--
 .../cassandra/streaming/StreamingTransferTest.java |   4 +-
 .../org/apache/cassandra/tools/BulkLoaderTest.java |   6 +-
 .../cassandra/tools/CompactionStressTest.java      |   3 -
 .../org/apache/cassandra/tools/GetVersionTest.java |   3 -
 .../cassandra/tools/NodeToolGossipInfoTest.java    |   3 -
 .../cassandra/tools/NodeToolTPStatsTest.java       |   3 -
 .../cassandra/tools/NodetoolNetStatsTest.java      |   3 -
 .../apache/cassandra/tools/OfflineToolUtils.java   |   1 +
 .../tools/SSTableExpiredBlockersTest.java          |   3 -
 .../apache/cassandra/tools/SSTableExportTest.java  |   4 -
 .../cassandra/tools/SSTableLevelResetterTest.java  |   3 -
 .../cassandra/tools/SSTableMetadataViewerTest.java |   3 -
 .../cassandra/tools/SSTableOfflineRelevelTest.java |   3 -
 .../tools/SSTableRepairedAtSetterTest.java         |   4 -
 .../cassandra/tools/StandaloneSSTableUtilTest.java |  18 --
 .../cassandra/tools/StandaloneScrubberTest.java    |  18 --
 .../cassandra/tools/StandaloneSplitterTest.java    |  18 --
 .../tools/StandaloneSplitterWithCQLTesterTest.java |  70 +-----
 .../tools/StandaloneUpgraderOnSStablesTest.java    |   8 +-
 .../cassandra/tools/StandaloneUpgraderTest.java    |  18 --
 .../cassandra/tools/StandaloneVerifierTest.java    |  19 --
 .../cassandra/tools/ToolsSchemaLoadingTest.java    | 104 +++++++++
 .../apache/cassandra/tools/nodetool/RingTest.java  |   3 -
 .../cassandra/tools/nodetool/StatusTest.java       |   3 -
 .../nodetool/stats/NodetoolTableStatsTest.java     |   3 -
 .../cassandra/transport/CQLUserAuditTest.java      |   3 -
 57 files changed, 693 insertions(+), 699 deletions(-)

diff --git a/ide/idea/workspace.xml b/ide/idea/workspace.xml
index 446f6dc..41645f5 100644
--- a/ide/idea/workspace.xml
+++ b/ide/idea/workspace.xml
@@ -169,6 +169,7 @@
       <option name="TEST_OBJECT" value="class" />
       <option name="VM_PARAMETERS" value="-Dcassandra.config=file://$PROJECT_DIR$/test/conf/cassandra.yaml -Dlogback.configurationFile=file://$PROJECT_DIR$/test/conf/logback-test.xml -Dcassandra.logdir=$PROJECT_DIR$/build/test/logs -Djava.library.path=$PROJECT_DIR$/lib/sigar-bin -Dlegacy-sstable-root=$PROJECT_DIR$/test/data/legacy-sstables -Dinvalid-legacy-sstable-root=$PROJECT_DIR$/test/data/invalid-legacy-sstables -Dcassandra.ring_delay_ms=1000 -Dcassandra.skip_sync=true -ea -XX:MaxMet [...]
       <option name="PARAMETERS" value="" />
+      <fork_mode value="class" />
       <option name="WORKING_DIRECTORY" value="" />
       <option name="ENV_VARIABLES" />
       <option name="PASS_PARENT_ENVS" value="true" />
diff --git a/test/memory/org/apache/cassandra/db/compaction/CompactionAllocationTest.java b/test/memory/org/apache/cassandra/db/compaction/CompactionAllocationTest.java
index a58303d..4398b3d 100644
--- a/test/memory/org/apache/cassandra/db/compaction/CompactionAllocationTest.java
+++ b/test/memory/org/apache/cassandra/db/compaction/CompactionAllocationTest.java
@@ -36,14 +36,12 @@ import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
-import org.junit.runner.RunWith;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import com.google.monitoring.runtime.instrumentation.AllocationRecorder;
 import com.google.monitoring.runtime.instrumentation.Sampler;
 import com.sun.management.ThreadMXBean;
-import org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.cql3.QueryOptions;
 import org.apache.cassandra.cql3.QueryProcessor;
@@ -56,7 +54,6 @@ import org.apache.cassandra.db.filter.ColumnFilter;
 import org.apache.cassandra.db.partitions.UnfilteredPartitionIterator;
 import org.apache.cassandra.db.partitions.UnfilteredPartitionIterators;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
-import org.apache.cassandra.io.util.DataOutputBuffer;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.io.util.UnbufferedDataOutputStreamPlus;
 import org.apache.cassandra.net.MessagingService;
@@ -68,7 +65,6 @@ import org.apache.cassandra.service.QueryState;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.ObjectSizes;
 
-@RunWith(OrderedJUnit4ClassRunner.class)
 public class CompactionAllocationTest
 {
     private static final Logger logger = LoggerFactory.getLogger(CompactionAllocationTest.class);
diff --git a/test/unit/org/apache/cassandra/OrderedJUnit4ClassRunner.java b/test/unit/org/apache/cassandra/OrderedJUnit4ClassRunner.java
deleted file mode 100644
index d0dec24..0000000
--- a/test/unit/org/apache/cassandra/OrderedJUnit4ClassRunner.java
+++ /dev/null
@@ -1,55 +0,0 @@
-package org.apache.cassandra;
-/*
- * 
- * 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.
- * 
- */
-
-
-import org.junit.runners.BlockJUnit4ClassRunner;
-import org.junit.runners.model.FrameworkMethod;
-import org.junit.runners.model.InitializationError;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-
-public class OrderedJUnit4ClassRunner extends BlockJUnit4ClassRunner
-{
-
-    public OrderedJUnit4ClassRunner(Class aClass) throws InitializationError
-    {
-        super(aClass);
-    }
-
-    @Override
-    protected List<FrameworkMethod> computeTestMethods()
-    {
-        final List<FrameworkMethod> list = super.computeTestMethods();
-        try
-        {
-            final List<FrameworkMethod> copy = new ArrayList<FrameworkMethod>(list);
-            Collections.sort(copy, MethodComparator.getFrameworkMethodComparatorForJUnit4());
-            return copy;
-        }
-        catch (Throwable throwable)
-        {
-            return list;
-        }
-    }
-}
diff --git a/test/unit/org/apache/cassandra/audit/AuditLoggerAuthTest.java b/test/unit/org/apache/cassandra/audit/AuditLoggerAuthTest.java
index c64323f..a873b6e 100644
--- a/test/unit/org/apache/cassandra/audit/AuditLoggerAuthTest.java
+++ b/test/unit/org/apache/cassandra/audit/AuditLoggerAuthTest.java
@@ -27,14 +27,13 @@ import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
-import org.junit.runner.RunWith;
 
 import com.datastax.driver.core.Cluster;
 import com.datastax.driver.core.Session;
 import com.datastax.driver.core.exceptions.AuthenticationException;
 import com.datastax.driver.core.exceptions.SyntaxError;
 import com.datastax.driver.core.exceptions.UnauthorizedException;
-import org.apache.cassandra.OrderedJUnit4ClassRunner;
+
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.OverrideConfigurationLoader;
 import org.apache.cassandra.config.ParameterizedClass;
@@ -54,7 +53,6 @@ import static org.junit.Assert.assertTrue;
  * Non authenticated tests are covered in {@link AuditLoggerTest}
  */
 
-@RunWith(OrderedJUnit4ClassRunner.class)
 public class AuditLoggerAuthTest
 {
     private static EmbeddedCassandraService embedded;
diff --git a/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java b/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
index 8a1bf05..0038938 100644
--- a/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
+++ b/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
@@ -31,9 +31,7 @@ import com.google.common.base.Throwables;
 import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
-import org.junit.runner.RunWith;
 
-import org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.exceptions.ConfigurationException;
 
@@ -42,7 +40,6 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
-@RunWith(OrderedJUnit4ClassRunner.class)
 public class DatabaseDescriptorTest
 {
     @BeforeClass
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/AlterNTSTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/AlterNTSTest.java
new file mode 100644
index 0000000..4cc95e1
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/AlterNTSTest.java
@@ -0,0 +1,103 @@
+/*
+ * 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.cql3.validation.operations;
+
+import java.util.List;
+
+import org.junit.Test;
+
+import com.datastax.driver.core.PreparedStatement;
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.service.ClientWarn;
+import org.assertj.core.api.Assertions;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
+public class AlterNTSTest extends CQLTester
+{
+    @Test
+    public void testDropColumnAsPreparedStatement() throws Throwable
+    {
+        String table = createTable("CREATE TABLE %s (key int PRIMARY KEY, value int);");
+
+        PreparedStatement prepared = sessionNet().prepare("ALTER TABLE " + KEYSPACE + "." + table + " DROP value;");
+
+        executeNet("INSERT INTO %s (key, value) VALUES (1, 1)");
+        assertRowsNet(executeNet("SELECT * FROM %s"), row(1, 1));
+
+        sessionNet().execute(prepared.bind());
+
+        executeNet("ALTER TABLE %s ADD value int");
+
+        assertRows(execute("SELECT * FROM %s"), row(1, null));
+    }
+
+    @Test
+    public void testCreateAlterKeyspacesRFWarnings() throws Throwable
+    {
+        requireNetwork();
+
+        // NTS
+        ClientWarn.instance.captureWarnings();
+        String ks = createKeyspace("CREATE KEYSPACE %s WITH replication = {'class' : 'NetworkTopologyStrategy', '" + DATA_CENTER + "' : 3 }");
+        List<String> warnings = ClientWarn.instance.getWarnings();
+        assertEquals(1, warnings.size());
+        Assertions.assertThat(warnings.get(0)).contains("Your replication factor 3 for keyspace " + ks + " is higher than the number of nodes 1 for datacenter " + DATA_CENTER);
+
+        ClientWarn.instance.captureWarnings();
+        execute("CREATE TABLE " + ks + ".t (k int PRIMARY KEY, v int)");
+        warnings = ClientWarn.instance.getWarnings();
+        assertNull(warnings);
+
+        ClientWarn.instance.captureWarnings();
+        execute("ALTER KEYSPACE " + ks + " WITH replication = {'class' : 'NetworkTopologyStrategy', '" + DATA_CENTER + "' : 2 }");
+        warnings = ClientWarn.instance.getWarnings();
+        assertEquals(1, warnings.size());
+        Assertions.assertThat(warnings.get(0)).contains("Your replication factor 2 for keyspace " + ks + " is higher than the number of nodes 1 for datacenter " + DATA_CENTER);
+
+        ClientWarn.instance.captureWarnings();
+        execute("ALTER KEYSPACE " + ks + " WITH replication = {'class' : 'NetworkTopologyStrategy', '" + DATA_CENTER + "' : 1 }");
+        warnings = ClientWarn.instance.getWarnings();
+        assertNull(warnings);
+
+        // SimpleStrategy
+        ClientWarn.instance.captureWarnings();
+        ks = createKeyspace("CREATE KEYSPACE %s WITH replication = { 'class' : 'SimpleStrategy', 'replication_factor' : 3 }");
+        warnings = ClientWarn.instance.getWarnings();
+        assertEquals(1, warnings.size());
+        Assertions.assertThat(warnings.get(0)).contains("Your replication factor 3 for keyspace " + ks + " is higher than the number of nodes 1");
+
+        ClientWarn.instance.captureWarnings();
+        execute("CREATE TABLE " + ks + ".t (k int PRIMARY KEY, v int)");
+        warnings = ClientWarn.instance.getWarnings();
+        assertNull(warnings);
+
+        ClientWarn.instance.captureWarnings();
+        execute("ALTER KEYSPACE " + ks + " WITH replication = { 'class' : 'SimpleStrategy', 'replication_factor' : 2 }");
+        warnings = ClientWarn.instance.getWarnings();
+        assertEquals(1, warnings.size());
+        Assertions.assertThat(warnings.get(0)).contains("Your replication factor 2 for keyspace " + ks + " is higher than the number of nodes 1");
+
+        ClientWarn.instance.captureWarnings();
+        execute("ALTER KEYSPACE " + ks + " WITH replication = { 'class' : 'SimpleStrategy', 'replication_factor' : 1 }");
+        warnings = ClientWarn.instance.getWarnings();
+        assertNull(warnings);
+    }
+}
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java
index 5297a41..7209150 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/AlterTest.java
@@ -17,14 +17,10 @@
  */
 package org.apache.cassandra.cql3.validation.operations;
 
-import java.util.List;
 import java.util.UUID;
 
 import org.junit.Test;
-import org.junit.runner.RunWith;
 
-import com.datastax.driver.core.PreparedStatement;
-import org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.cql3.CQLTester;
 import org.apache.cassandra.db.ColumnFamilyStore;
@@ -36,38 +32,17 @@ import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.locator.TokenMetadata;
 import org.apache.cassandra.schema.SchemaConstants;
 import org.apache.cassandra.schema.SchemaKeyspace;
-import org.apache.cassandra.service.ClientWarn;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.FBUtilities;
-import org.assertj.core.api.Assertions;
 
 import static java.lang.String.format;
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
-@RunWith(OrderedJUnit4ClassRunner.class)
 public class AlterTest extends CQLTester
 {
     @Test
-    public void testDropColumnAsPreparedStatement() throws Throwable
-    {
-        String table = createTable("CREATE TABLE %s (key int PRIMARY KEY, value int);");
-
-        PreparedStatement prepared = sessionNet().prepare("ALTER TABLE " + KEYSPACE + "." + table + " DROP value;");
-
-        executeNet("INSERT INTO %s (key, value) VALUES (1, 1)");
-        assertRowsNet(executeNet("SELECT * FROM %s"), row(1, 1));
-
-        sessionNet().execute(prepared.bind());
-
-        executeNet("ALTER TABLE %s ADD value int");
-
-        assertRows(execute("SELECT * FROM %s"), row(1, null));
-    }
-
-    @Test
     public void testAddList() throws Throwable
     {
         createTable("CREATE TABLE %s (id text PRIMARY KEY, content text);");
@@ -278,58 +253,6 @@ public class AlterTest extends CQLTester
     }
 
     @Test
-    public void testCreateAlterKeyspacesRFWarnings() throws Throwable
-    {
-        requireNetwork();
-
-        // NTS
-        ClientWarn.instance.captureWarnings();
-        String ks = createKeyspace("CREATE KEYSPACE %s WITH replication = {'class' : 'NetworkTopologyStrategy', '" + DATA_CENTER + "' : 3 }");
-        List<String> warnings = ClientWarn.instance.getWarnings();
-        assertEquals(1, warnings.size());
-        Assertions.assertThat(warnings.get(0)).contains("Your replication factor 3 for keyspace " + ks + " is higher than the number of nodes 1 for datacenter " + DATA_CENTER);
-
-        ClientWarn.instance.captureWarnings();
-        execute("CREATE TABLE " + ks + ".t (k int PRIMARY KEY, v int)");
-        warnings = ClientWarn.instance.getWarnings();
-        assertNull(warnings);
-
-        ClientWarn.instance.captureWarnings();
-        execute("ALTER KEYSPACE " + ks + " WITH replication = {'class' : 'NetworkTopologyStrategy', '" + DATA_CENTER + "' : 2 }");
-        warnings = ClientWarn.instance.getWarnings();
-        assertEquals(1, warnings.size());
-        Assertions.assertThat(warnings.get(0)).contains("Your replication factor 2 for keyspace " + ks + " is higher than the number of nodes 1 for datacenter " + DATA_CENTER);
-
-        ClientWarn.instance.captureWarnings();
-        execute("ALTER KEYSPACE " + ks + " WITH replication = {'class' : 'NetworkTopologyStrategy', '" + DATA_CENTER + "' : 1 }");
-        warnings = ClientWarn.instance.getWarnings();
-        assertNull(warnings);
-
-        // SimpleStrategy
-        ClientWarn.instance.captureWarnings();
-        ks = createKeyspace("CREATE KEYSPACE %s WITH replication = { 'class' : 'SimpleStrategy', 'replication_factor' : 3 }");
-        warnings = ClientWarn.instance.getWarnings();
-        assertEquals(1, warnings.size());
-        Assertions.assertThat(warnings.get(0)).contains("Your replication factor 3 for keyspace " + ks + " is higher than the number of nodes 1");
-
-        ClientWarn.instance.captureWarnings();
-        execute("CREATE TABLE " + ks + ".t (k int PRIMARY KEY, v int)");
-        warnings = ClientWarn.instance.getWarnings();
-        assertNull(warnings);
-
-        ClientWarn.instance.captureWarnings();
-        execute("ALTER KEYSPACE " + ks + " WITH replication = { 'class' : 'SimpleStrategy', 'replication_factor' : 2 }");
-        warnings = ClientWarn.instance.getWarnings();
-        assertEquals(1, warnings.size());
-        Assertions.assertThat(warnings.get(0)).contains("Your replication factor 2 for keyspace " + ks + " is higher than the number of nodes 1");
-
-        ClientWarn.instance.captureWarnings();
-        execute("ALTER KEYSPACE " + ks + " WITH replication = { 'class' : 'SimpleStrategy', 'replication_factor' : 1 }");
-        warnings = ClientWarn.instance.getWarnings();
-        assertNull(warnings);
-    }
-
-    @Test
     public void testCreateAlterNetworkTopologyWithDefaults() throws Throwable
     {
         TokenMetadata metadata = StorageService.instance.getTokenMetadata();
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 1c1f116..5c24432 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/TTLTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/TTLTest.java
@@ -26,9 +26,7 @@ import java.io.IOException;
 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;
@@ -49,7 +47,6 @@ 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";
@@ -214,15 +211,7 @@ public class TTLTest extends CQLTester
         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
@@ -437,6 +426,7 @@ public class TTLTest extends CQLTester
 
         try
         {
+            cfs.truncateBlocking();
             dropTable("DROP TABLE %s");
         }
         catch (Throwable e)
diff --git a/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java b/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
index 48ef580..df88374 100644
--- a/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
+++ b/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
@@ -31,7 +31,6 @@ import org.junit.Before;
 import org.junit.Assume;
 import org.junit.BeforeClass;
 import org.junit.Test;
-import org.junit.runner.RunWith;
 
 import org.json.simple.JSONArray;
 import org.json.simple.JSONObject;
@@ -59,7 +58,6 @@ import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.WrappedRunnable;
 import static junit.framework.Assert.assertNotNull;
 
-@RunWith(OrderedJUnit4ClassRunner.class)
 public class ColumnFamilyStoreTest
 {
     public static final String KEYSPACE1 = "ColumnFamilyStoreTest1";
@@ -121,6 +119,7 @@ public class ColumnFamilyStoreTest
     {
         Keyspace keyspace = Keyspace.open(KEYSPACE1);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF_STANDARD1);
+        keyspace.getColumnFamilyStores().forEach(ColumnFamilyStore::truncateBlocking);
 
         List<Mutation> rms = new LinkedList<>();
         rms.add(new RowUpdateBuilder(cfs.metadata(), 0, "key1")
@@ -503,5 +502,6 @@ public class ColumnFamilyStoreTest
         List<File> ssTableFiles = new Directories(cfs.metadata()).sstableLister(Directories.OnTxnErr.THROW).listFiles();
         assertNotNull(ssTableFiles);
         assertEquals(0, ssTableFiles.size());
+        cfs.clearUnsafe();
     }
 }
diff --git a/test/unit/org/apache/cassandra/db/ScrubTest.java b/test/unit/org/apache/cassandra/db/ScrubTest.java
index 7c24d7c..9f73702 100644
--- a/test/unit/org/apache/cassandra/db/ScrubTest.java
+++ b/test/unit/org/apache/cassandra/db/ScrubTest.java
@@ -42,9 +42,7 @@ import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
-import org.junit.runner.RunWith;
 
-import org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.UpdateBuilder;
 import org.apache.cassandra.Util;
@@ -84,11 +82,7 @@ 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;
@@ -101,7 +95,6 @@ import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 import static org.junit.Assume.assumeTrue;
 
-@RunWith(OrderedJUnit4ClassRunner.class)
 public class ScrubTest
 {
     public static final String INVALID_LEGACY_SSTABLE_ROOT_PROP = "invalid-legacy-sstable-root";
@@ -120,8 +113,8 @@ public class ScrubTest
 
     private static final AtomicInteger seq = new AtomicInteger();
 
-    private String ksName;
-    private Keyspace keyspace;
+    String ksName;
+    Keyspace keyspace;
 
     @BeforeClass
     public static void defineSchema() throws ConfigurationException
@@ -179,7 +172,7 @@ public class ScrubTest
 
         CompactionManager.instance.disableAutoCompaction();
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(COUNTER_CF);
-
+        cfs.truncateBlocking();
         fillCounterCF(cfs, numPartitions);
 
         assertOrderedAll(cfs, numPartitions);
@@ -418,7 +411,7 @@ public class ScrubTest
         }
     }
 
-    private void overrideWithGarbage(SSTableReader sstable, ByteBuffer key1, ByteBuffer key2) throws IOException
+    static void overrideWithGarbage(SSTableReader sstable, ByteBuffer key1, ByteBuffer key2) throws IOException
     {
         boolean compression = Boolean.parseBoolean(System.getProperty("cassandra.test.compression", "false"));
         long startPosition, endPosition;
@@ -448,7 +441,7 @@ public class ScrubTest
         overrideWithGarbage(sstable, startPosition, endPosition);
     }
 
-    private void overrideWithGarbage(SSTableReader sstable, long startPosition, long endPosition) throws IOException
+    private static void overrideWithGarbage(SSTableReader sstable, long startPosition, long endPosition) throws IOException
     {
         try (RandomAccessFile file = new RandomAccessFile(sstable.getFilename(), "rw"))
         {
@@ -459,7 +452,7 @@ public class ScrubTest
             ChunkCache.instance.invalidateFile(sstable.getFilename());
     }
 
-    private static void assertOrderedAll(ColumnFamilyStore cfs, int expectedSize)
+    static void assertOrderedAll(ColumnFamilyStore cfs, int expectedSize)
     {
         assertOrdered(Util.cmd(cfs).build(), expectedSize);
     }
@@ -478,7 +471,7 @@ public class ScrubTest
         assertEquals(expectedSize, size);
     }
 
-    protected void fillCF(ColumnFamilyStore cfs, int partitionsPerSSTable)
+    protected static void fillCF(ColumnFamilyStore cfs, int partitionsPerSSTable)
     {
         for (int i = 0; i < partitionsPerSSTable; i++)
         {
@@ -517,7 +510,7 @@ public class ScrubTest
         cfs.forceBlockingFlush();
     }
 
-    protected void fillCounterCF(ColumnFamilyStore cfs, int partitionsPerSSTable) throws WriteTimeoutException
+    protected static void fillCounterCF(ColumnFamilyStore cfs, int partitionsPerSSTable) throws WriteTimeoutException
     {
         for (int i = 0; i < partitionsPerSSTable; i++)
         {
@@ -709,193 +702,46 @@ public class ScrubTest
     @Test
     public void testFilterOutDuplicates() throws Exception
     {
-        DatabaseDescriptor.setPartitionerUnsafe(Murmur3Partitioner.instance);
-        QueryProcessor.process(String.format("CREATE TABLE \"%s\".cf_with_duplicates_3_0 (a int, b int, c int, PRIMARY KEY (a, b))", ksName), ConsistencyLevel.ONE);
-
-        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("cf_with_duplicates_3_0");
-
-        Path legacySSTableRoot = Paths.get(System.getProperty(INVALID_LEGACY_SSTABLE_ROOT_PROP),
-                                           "Keyspace1",
-                                           "cf_with_duplicates_3_0");
-
-        for (String filename : new String[]{ "mb-3-big-CompressionInfo.db",
-                                             "mb-3-big-Digest.crc32",
-                                             "mb-3-big-Index.db",
-                                             "mb-3-big-Summary.db",
-                                             "mb-3-big-Data.db",
-                                             "mb-3-big-Filter.db",
-                                             "mb-3-big-Statistics.db",
-                                             "mb-3-big-TOC.txt" })
+        IPartitioner oldPart = DatabaseDescriptor.getPartitioner();
+        try
         {
-            Files.copy(Paths.get(legacySSTableRoot.toString(), filename), cfs.getDirectories().getDirectoryForNewSSTables().toPath().resolve(filename));
-        }
-
-        cfs.loadNewSSTables();
-
-        cfs.scrub(true, true, false, false, false, 1);
-
-        UntypedResultSet rs = QueryProcessor.executeInternal(String.format("SELECT * FROM \"%s\".cf_with_duplicates_3_0", ksName));
-        assertNotNull(rs);
-        assertEquals(1, rs.size());
-
-        QueryProcessor.executeInternal(String.format("DELETE FROM \"%s\".cf_with_duplicates_3_0 WHERE a=1 AND b =2", ksName));
-        rs = QueryProcessor.executeInternal(String.format("SELECT * FROM \"%s\".cf_with_duplicates_3_0", ksName));
-        assertNotNull(rs);
-        assertEquals(0, rs.size());
-    }
-
-    @Test
-    public void testScrubOnePartitionWithTool()
-    {
-        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF);
-
-        // insert data and verify we get it back w/ range query
-        fillCF(cfs, 1);
-        assertOrderedAll(cfs, 1);
-
-        ToolResult tool = ToolRunner.invokeClass(StandaloneScrubber.class, ksName, CF);
-        Assertions.assertThat(tool.getStdout()).contains("Pre-scrub sstables snapshotted into");
-        Assertions.assertThat(tool.getStdout()).contains("1 partitions in new sstable and 0 empty");
-        tool.assertOnCleanExit();
-
-        // check data is still there
-        assertOrderedAll(cfs, 1);
-    }
-
-    @Test
-    public void testSkipScrubCorruptedCounterPartitionWithTool() throws IOException, WriteTimeoutException
-    {
-        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(COUNTER_CF);
-        int numPartitions = 1000;
-
-        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 == true, the corrupt rows will be skipped
-        ToolResult tool = ToolRunner.invokeClass(StandaloneScrubber.class, "-s", ksName, COUNTER_CF);
-        Assertions.assertThat(tool.getStdout()).contains("0 empty");
-        Assertions.assertThat(tool.getStdout()).contains("partitions that were skipped");
-        tool.assertOnCleanExit();
-
-        assertEquals(1, cfs.getLiveSSTables().size());
-    }
+            DatabaseDescriptor.setPartitionerUnsafe(Murmur3Partitioner.instance);
+            QueryProcessor.process(String.format("CREATE TABLE \"%s\".cf_with_duplicates_3_0 (a int, b int, c int, PRIMARY KEY (a, b))", ksName), ConsistencyLevel.ONE);
+
+            ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("cf_with_duplicates_3_0");
+
+            Path legacySSTableRoot = Paths.get(System.getProperty(INVALID_LEGACY_SSTABLE_ROOT_PROP),
+                                               "Keyspace1",
+                                               "cf_with_duplicates_3_0");
+
+            for (String filename : new String[]{ "mb-3-big-CompressionInfo.db",
+                                                 "mb-3-big-Digest.crc32",
+                                                 "mb-3-big-Index.db",
+                                                 "mb-3-big-Summary.db",
+                                                 "mb-3-big-Data.db",
+                                                 "mb-3-big-Filter.db",
+                                                 "mb-3-big-Statistics.db",
+                                                 "mb-3-big-TOC.txt" })
+            {
+                Files.copy(Paths.get(legacySSTableRoot.toString(), filename), cfs.getDirectories().getDirectoryForNewSSTables().toPath().resolve(filename));
+            }
 
-    @Test
-    public void testNoSkipScrubCorruptedCounterPartitionWithTool() throws IOException, WriteTimeoutException
-    {
-        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(COUNTER_CF);
-        int numPartitions = 1000;
+            cfs.loadNewSSTables();
 
-        fillCounterCF(cfs, numPartitions);
-        assertOrderedAll(cfs, numPartitions);
-        assertEquals(1, cfs.getLiveSSTables().size());
-        SSTableReader sstable = cfs.getLiveSSTables().iterator().next();
+            cfs.scrub(true, true, false, false, false, 1);
 
-        overrideWithGarbage(sstable, ByteBufferUtil.bytes("0"), ByteBufferUtil.bytes("1"));
+            UntypedResultSet rs = QueryProcessor.executeInternal(String.format("SELECT * FROM \"%s\".cf_with_duplicates_3_0", ksName));
+            assertNotNull(rs);
+            assertEquals(1, rs.size());
 
-        // with skipCorrupted == false, the scrub is expected to fail
-        try
-        {
-            ToolRunner.invokeClass(StandaloneScrubber.class, ksName, COUNTER_CF);
-            fail("Expected a CorruptSSTableException to be thrown");
+            QueryProcessor.executeInternal(String.format("DELETE FROM \"%s\".cf_with_duplicates_3_0 WHERE a=1 AND b =2", ksName));
+            rs = QueryProcessor.executeInternal(String.format("SELECT * FROM \"%s\".cf_with_duplicates_3_0", ksName));
+            assertNotNull(rs);
+            assertEquals(0, rs.size());
         }
-        catch (IOError err) {
-            assertTrue(err.getCause() instanceof CorruptSSTableException);
+        finally
+        {
+            DatabaseDescriptor.setPartitionerUnsafe(oldPart);
         }
     }
-
-    @Test
-    public void testNoCheckScrubMultiPartitionWithTool()
-    {
-        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF);
-
-        // insert data and verify we get it back w/ range query
-        fillCF(cfs, 10);
-        assertOrderedAll(cfs, 10);
-
-        ToolResult tool = ToolRunner.invokeClass(StandaloneScrubber.class, "-n", ksName, CF);
-        Assertions.assertThat(tool.getStdout()).contains("Pre-scrub sstables snapshotted into");
-        Assertions.assertThat(tool.getStdout()).contains("10 partitions in new sstable and 0 empty");
-        tool.assertOnCleanExit();
-
-        // check data is still there
-        assertOrderedAll(cfs, 10);
-    }
-
-    @Test
-    public void testHeaderFixValidateOnlyWithTool()
-    {
-        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF);
-
-        fillCF(cfs, 1);
-        assertOrderedAll(cfs, 1);
-
-        ToolResult tool = ToolRunner.invokeClass(StandaloneScrubber.class, "-e", "validate_only", ksName, CF);
-        Assertions.assertThat(tool.getStdout()).contains("Not continuing with scrub, since '--header-fix validate-only' was specified.");
-        tool.assertOnCleanExit();
-        assertOrderedAll(cfs, 1);
-    }
-
-    @Test
-    public void testHeaderFixValidateWithTool()
-    {
-        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF);
-
-        fillCF(cfs, 1);
-        assertOrderedAll(cfs, 1);
-
-        ToolResult tool = ToolRunner.invokeClass(StandaloneScrubber.class, "-e", "validate", ksName, CF);
-        Assertions.assertThat(tool.getStdout()).contains("Pre-scrub sstables snapshotted into");
-        Assertions.assertThat(tool.getStdout()).contains("1 partitions in new sstable and 0 empty");
-        tool.assertOnCleanExit();
-        assertOrderedAll(cfs, 1);
-    }
-
-    @Test
-    public void testHeaderFixFixOnlyWithTool()
-    {
-        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF);
-
-        fillCF(cfs, 1);
-        assertOrderedAll(cfs, 1);
-
-        ToolResult tool = ToolRunner.invokeClass(StandaloneScrubber.class, "-e", "fix-only", ksName, CF);
-        Assertions.assertThat(tool.getStdout()).contains("Not continuing with scrub, since '--header-fix fix-only' was specified.");
-        tool.assertOnCleanExit();
-        assertOrderedAll(cfs, 1);
-    }
-
-    @Test
-    public void testHeaderFixWithTool()
-    {
-        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF);
-
-        fillCF(cfs, 1);
-        assertOrderedAll(cfs, 1);
-
-        ToolResult tool = ToolRunner.invokeClass(StandaloneScrubber.class, "-e", "fix", ksName, CF);
-        Assertions.assertThat(tool.getStdout()).contains("Pre-scrub sstables snapshotted into");
-        Assertions.assertThat(tool.getStdout()).contains("1 partitions in new sstable and 0 empty");
-        tool.assertOnCleanExit();
-        assertOrderedAll(cfs, 1);
-    }
-
-    @Test
-    public void testHeaderFixNoChecksWithTool()
-    {
-        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF);
-
-        fillCF(cfs, 1);
-        assertOrderedAll(cfs, 1);
-
-        ToolResult tool = ToolRunner.invokeClass(StandaloneScrubber.class, "-e", "off", ksName, CF);
-        Assertions.assertThat(tool.getStdout()).contains("Pre-scrub sstables snapshotted into");
-        Assertions.assertThat(tool.getStdout()).contains("1 partitions in new sstable and 0 empty");
-        tool.assertOnCleanExit();
-        assertOrderedAll(cfs, 1);
-    }
 }
diff --git a/test/unit/org/apache/cassandra/db/ScrubToolTest.java b/test/unit/org/apache/cassandra/db/ScrubToolTest.java
new file mode 100644
index 0000000..280810c
--- /dev/null
+++ b/test/unit/org/apache/cassandra/db/ScrubToolTest.java
@@ -0,0 +1,249 @@
+/*
+ * 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.db;
+
+import java.io.IOError;
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.db.compaction.CompactionManager;
+import org.apache.cassandra.db.marshal.UUIDType;
+import org.apache.cassandra.dht.ByteOrderedPartitioner;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.exceptions.WriteTimeoutException;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.schema.KeyspaceParams;
+import org.apache.cassandra.tools.StandaloneScrubber;
+import org.apache.cassandra.tools.ToolRunner;
+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;
+import static org.apache.cassandra.SchemaLoader.getCompressionParameters;
+import static org.apache.cassandra.SchemaLoader.loadSchema;
+import static org.apache.cassandra.SchemaLoader.standardCFMD;
+import static org.apache.cassandra.db.ScrubTest.CF_INDEX1;
+import static org.apache.cassandra.db.ScrubTest.CF_INDEX1_BYTEORDERED;
+import static org.apache.cassandra.db.ScrubTest.CF_INDEX2;
+import static org.apache.cassandra.db.ScrubTest.CF_INDEX2_BYTEORDERED;
+import static org.apache.cassandra.db.ScrubTest.CF_UUID;
+import static org.apache.cassandra.db.ScrubTest.COMPRESSION_CHUNK_LENGTH;
+import static org.apache.cassandra.db.ScrubTest.COUNTER_CF;
+import static org.apache.cassandra.db.ScrubTest.assertOrderedAll;
+import static org.apache.cassandra.db.ScrubTest.fillCF;
+import static org.apache.cassandra.db.ScrubTest.fillCounterCF;
+import static org.apache.cassandra.db.ScrubTest.overrideWithGarbage;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public class ScrubToolTest
+{
+    private static final String CF = "scrub_tool_test";
+    private static final AtomicInteger seq = new AtomicInteger();
+
+    String ksName;
+    Keyspace keyspace;
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        loadSchema();
+    }
+
+    @Before
+    public void setup()
+    {
+        ksName = "scrub_test_" + seq.incrementAndGet();
+        createKeyspace(ksName,
+                       KeyspaceParams.simple(1),
+                       standardCFMD(ksName, CF),
+                       counterCFMD(ksName, COUNTER_CF).compression(getCompressionParameters(COMPRESSION_CHUNK_LENGTH)),
+                       standardCFMD(ksName, CF_UUID, 0, UUIDType.instance),
+                       SchemaLoader.keysIndexCFMD(ksName, CF_INDEX1, true),
+                       SchemaLoader.compositeIndexCFMD(ksName, CF_INDEX2, true),
+                       SchemaLoader.keysIndexCFMD(ksName, CF_INDEX1_BYTEORDERED, true).partitioner(ByteOrderedPartitioner.instance),
+                       SchemaLoader.compositeIndexCFMD(ksName, CF_INDEX2_BYTEORDERED, true).partitioner(ByteOrderedPartitioner.instance));
+        keyspace = Keyspace.open(ksName);
+
+        CompactionManager.instance.disableAutoCompaction();
+        System.setProperty(org.apache.cassandra.tools.Util.ALLOW_TOOL_REINIT_FOR_TEST, "true"); // Necessary for testing
+    }
+
+    @Test
+    public void testScrubOnePartitionWithTool()
+    {
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF);
+
+        // insert data and verify we get it back w/ range query
+        fillCF(cfs, 1);
+        assertOrderedAll(cfs, 1);
+
+        ToolRunner.ToolResult tool = ToolRunner.invokeClass(StandaloneScrubber.class, ksName, CF);
+        Assertions.assertThat(tool.getStdout()).contains("Pre-scrub sstables snapshotted into");
+        Assertions.assertThat(tool.getStdout()).contains("1 partitions in new sstable and 0 empty");
+        tool.assertOnCleanExit();
+
+        // check data is still there
+        assertOrderedAll(cfs, 1);
+    }
+
+    @Test
+    public void testSkipScrubCorruptedCounterPartitionWithTool() throws IOException, WriteTimeoutException
+    {
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(COUNTER_CF);
+        int numPartitions = 1000;
+
+        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 == true, the corrupt rows will be skipped
+        ToolRunner.ToolResult tool = ToolRunner.invokeClass(StandaloneScrubber.class, "-s", ksName, COUNTER_CF);
+        Assertions.assertThat(tool.getStdout()).contains("0 empty");
+        Assertions.assertThat(tool.getStdout()).contains("partitions that were skipped");
+        tool.assertOnCleanExit();
+
+        assertEquals(1, cfs.getLiveSSTables().size());
+    }
+
+    @Test
+    public void testNoSkipScrubCorruptedCounterPartitionWithTool() throws IOException, WriteTimeoutException
+    {
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(COUNTER_CF);
+        int numPartitions = 1000;
+
+        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, ksName, COUNTER_CF);
+            fail("Expected a CorruptSSTableException to be thrown");
+        }
+        catch (IOError err) {
+            assertTrue(err.getCause() instanceof CorruptSSTableException);
+        }
+    }
+
+    @Test
+    public void testNoCheckScrubMultiPartitionWithTool()
+    {
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF);
+
+        // insert data and verify we get it back w/ range query
+        fillCF(cfs, 10);
+        assertOrderedAll(cfs, 10);
+
+        ToolRunner.ToolResult tool = ToolRunner.invokeClass(StandaloneScrubber.class, "-n", ksName, CF);
+        Assertions.assertThat(tool.getStdout()).contains("Pre-scrub sstables snapshotted into");
+        Assertions.assertThat(tool.getStdout()).contains("10 partitions in new sstable and 0 empty");
+        tool.assertOnCleanExit();
+
+        // check data is still there
+        assertOrderedAll(cfs, 10);
+    }
+
+    @Test
+    public void testHeaderFixValidateOnlyWithTool()
+    {
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF);
+
+        fillCF(cfs, 1);
+        assertOrderedAll(cfs, 1);
+
+        ToolRunner.ToolResult tool = ToolRunner.invokeClass(StandaloneScrubber.class, "-e", "validate_only", ksName, CF);
+        Assertions.assertThat(tool.getStdout()).contains("Not continuing with scrub, since '--header-fix validate-only' was specified.");
+        tool.assertOnCleanExit();
+        assertOrderedAll(cfs, 1);
+    }
+
+    @Test
+    public void testHeaderFixValidateWithTool()
+    {
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF);
+
+        fillCF(cfs, 1);
+        assertOrderedAll(cfs, 1);
+
+        ToolRunner.ToolResult tool = ToolRunner.invokeClass(StandaloneScrubber.class, "-e", "validate", ksName, CF);
+        Assertions.assertThat(tool.getStdout()).contains("Pre-scrub sstables snapshotted into");
+        Assertions.assertThat(tool.getStdout()).contains("1 partitions in new sstable and 0 empty");
+        tool.assertOnCleanExit();
+        assertOrderedAll(cfs, 1);
+    }
+
+    @Test
+    public void testHeaderFixFixOnlyWithTool()
+    {
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF);
+
+        fillCF(cfs, 1);
+        assertOrderedAll(cfs, 1);
+
+        ToolRunner.ToolResult tool = ToolRunner.invokeClass(StandaloneScrubber.class, "-e", "fix-only", ksName, CF);
+        Assertions.assertThat(tool.getStdout()).contains("Not continuing with scrub, since '--header-fix fix-only' was specified.");
+        tool.assertOnCleanExit();
+        assertOrderedAll(cfs, 1);
+    }
+
+    @Test
+    public void testHeaderFixWithTool()
+    {
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF);
+
+        fillCF(cfs, 1);
+        assertOrderedAll(cfs, 1);
+
+        ToolRunner.ToolResult tool = ToolRunner.invokeClass(StandaloneScrubber.class, "-e", "fix", ksName, CF);
+        Assertions.assertThat(tool.getStdout()).contains("Pre-scrub sstables snapshotted into");
+        Assertions.assertThat(tool.getStdout()).contains("1 partitions in new sstable and 0 empty");
+        tool.assertOnCleanExit();
+        assertOrderedAll(cfs, 1);
+    }
+
+    @Test
+    public void testHeaderFixNoChecksWithTool()
+    {
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF);
+
+        fillCF(cfs, 1);
+        assertOrderedAll(cfs, 1);
+
+        ToolRunner.ToolResult tool = ToolRunner.invokeClass(StandaloneScrubber.class, "-e", "off", ksName, CF);
+        Assertions.assertThat(tool.getStdout()).contains("Pre-scrub sstables snapshotted into");
+        Assertions.assertThat(tool.getStdout()).contains("1 partitions in new sstable and 0 empty");
+        tool.assertOnCleanExit();
+        assertOrderedAll(cfs, 1);
+    }
+}
diff --git a/test/unit/org/apache/cassandra/db/VerifyTest.java b/test/unit/org/apache/cassandra/db/VerifyTest.java
index 4b73f27..8e3fbe4 100644
--- a/test/unit/org/apache/cassandra/db/VerifyTest.java
+++ b/test/unit/org/apache/cassandra/db/VerifyTest.java
@@ -20,13 +20,11 @@ package org.apache.cassandra.db;
 
 import com.google.common.base.Charsets;
 
-import org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.Util;
 import org.apache.cassandra.batchlog.Batch;
 import org.apache.cassandra.batchlog.BatchlogManager;
 import org.apache.cassandra.cache.ChunkCache;
 import org.apache.cassandra.UpdateBuilder;
-import org.apache.cassandra.db.compaction.AbstractCompactionStrategy;
 import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.db.compaction.Verifier;
 import org.apache.cassandra.db.marshal.UUIDType;
@@ -37,7 +35,6 @@ import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.WriteTimeoutException;
 import org.apache.cassandra.io.FSWriteError;
-import org.apache.cassandra.io.compress.CorruptBlockException;
 import org.apache.cassandra.io.sstable.Component;
 import org.apache.cassandra.io.sstable.CorruptSSTableException;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
@@ -51,10 +48,8 @@ import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.commons.lang3.StringUtils;
 import org.junit.BeforeClass;
 import org.junit.Test;
-import org.junit.runner.RunWith;
 
 import java.io.*;
-import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.nio.file.Files;
 import java.util.ArrayList;
@@ -74,7 +69,6 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
-@RunWith(OrderedJUnit4ClassRunner.class)
 public class VerifyTest
 {
     public static final String KEYSPACE = "Keyspace1";
diff --git a/test/unit/org/apache/cassandra/db/compaction/CompactionsTest.java b/test/unit/org/apache/cassandra/db/compaction/CompactionsTest.java
index a78c7d4..dd51f61 100644
--- a/test/unit/org/apache/cassandra/db/compaction/CompactionsTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/CompactionsTest.java
@@ -29,9 +29,7 @@ import java.util.concurrent.TimeUnit;
 import org.junit.BeforeClass;
 import org.junit.Ignore;
 import org.junit.Test;
-import org.junit.runner.RunWith;
 
-import org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
 import org.apache.cassandra.cql3.ColumnIdentifier;
@@ -77,10 +75,8 @@ import org.apache.cassandra.utils.FBUtilities;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotEquals;
 import static org.junit.Assert.assertTrue;
 
-@RunWith(OrderedJUnit4ClassRunner.class)
 public class CompactionsTest
 {
     private static final String KEYSPACE1 = "Keyspace1";
diff --git a/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java b/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java
index 2d6835b..41039d5 100644
--- a/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java
@@ -37,12 +37,10 @@ import org.junit.After;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
-import org.junit.runner.RunWith;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
 import org.apache.cassandra.UpdateBuilder;
@@ -72,7 +70,6 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
-@RunWith(OrderedJUnit4ClassRunner.class)
 public class LeveledCompactionStrategyTest
 {
     private static final Logger logger = LoggerFactory.getLogger(LeveledCompactionStrategyTest.class);
diff --git a/test/unit/org/apache/cassandra/db/compaction/TTLExpiryTest.java b/test/unit/org/apache/cassandra/db/compaction/TTLExpiryTest.java
index a2352fc..c20316a 100644
--- a/test/unit/org/apache/cassandra/db/compaction/TTLExpiryTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/TTLExpiryTest.java
@@ -26,9 +26,7 @@ import com.google.common.collect.Multimap;
 import com.google.common.collect.Sets;
 import org.junit.BeforeClass;
 import org.junit.Test;
-import org.junit.runner.RunWith;
 
-import org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
 import org.apache.cassandra.schema.TableMetadata;
@@ -49,7 +47,6 @@ import org.apache.cassandra.utils.ByteBufferUtil;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
-@RunWith(OrderedJUnit4ClassRunner.class)
 public class TTLExpiryTest
 {
     public static final String KEYSPACE1 = "TTLExpiryTest";
diff --git a/test/unit/org/apache/cassandra/dht/BootStrapperTest.java b/test/unit/org/apache/cassandra/dht/BootStrapperTest.java
index 44ea5a9..05d42cf 100644
--- a/test/unit/org/apache/cassandra/dht/BootStrapperTest.java
+++ b/test/unit/org/apache/cassandra/dht/BootStrapperTest.java
@@ -28,9 +28,7 @@ import com.google.common.collect.Multimap;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
-import org.junit.runner.RunWith;
 
-import org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.Keyspace;
@@ -49,7 +47,6 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 
 
-@RunWith(OrderedJUnit4ClassRunner.class)
 public class BootStrapperTest
 {
     static IPartitioner oldPartitioner;
diff --git a/test/unit/org/apache/cassandra/diag/DiagnosticEventServiceTest.java b/test/unit/org/apache/cassandra/diag/DiagnosticEventServiceTest.java
index a645c03..548ce62 100644
--- a/test/unit/org/apache/cassandra/diag/DiagnosticEventServiceTest.java
+++ b/test/unit/org/apache/cassandra/diag/DiagnosticEventServiceTest.java
@@ -28,9 +28,7 @@ import com.google.common.collect.ImmutableList;
 import org.junit.After;
 import org.junit.BeforeClass;
 import org.junit.Test;
-import org.junit.runner.RunWith;
 
-import org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.config.DatabaseDescriptor;
 
 import static org.junit.Assert.assertEquals;
@@ -38,7 +36,6 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
-@RunWith(OrderedJUnit4ClassRunner.class)
 public class DiagnosticEventServiceTest
 {
 
@@ -240,5 +237,5 @@ public class DiagnosticEventServiceTest
         }
     }
 
-    public enum TestEventType { TEST1, TEST2, TEST3 };
+    public enum TestEventType { TEST1, TEST2, TEST3 }
 }
diff --git a/test/unit/org/apache/cassandra/io/sstable/IndexSummaryManagerTest.java b/test/unit/org/apache/cassandra/io/sstable/IndexSummaryManagerTest.java
index 9bae923..08bd67c 100644
--- a/test/unit/org/apache/cassandra/io/sstable/IndexSummaryManagerTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/IndexSummaryManagerTest.java
@@ -23,7 +23,6 @@ import java.util.*;
 import java.util.concurrent.*;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.function.Consumer;
-import java.util.stream.Collectors;
 
 import com.google.common.base.Joiner;
 import com.google.common.collect.Sets;
@@ -31,18 +30,15 @@ import org.junit.After;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
-import org.junit.runner.RunWith;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
 import org.apache.cassandra.concurrent.NamedThreadFactory;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.RowUpdateBuilder;
-import org.apache.cassandra.db.compaction.AntiCompactionTest;
 import org.apache.cassandra.db.compaction.CompactionInfo;
 import org.apache.cassandra.db.compaction.CompactionInterruptedException;
 import org.apache.cassandra.db.compaction.CompactionManager;
@@ -51,7 +47,6 @@ import org.apache.cassandra.db.compaction.OperationType;
 import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
-import org.apache.cassandra.metrics.CompactionMetrics;
 import org.apache.cassandra.metrics.RestorableMeter;
 import org.apache.cassandra.schema.CachingParams;
 import org.apache.cassandra.schema.KeyspaceParams;
@@ -71,7 +66,6 @@ import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
-@RunWith(OrderedJUnit4ClassRunner.class)
 public class IndexSummaryManagerTest
 {
     private static final Logger logger = LoggerFactory.getLogger(IndexSummaryManagerTest.class);
diff --git a/test/unit/org/apache/cassandra/io/sstable/LargePartitionsTest.java b/test/unit/org/apache/cassandra/io/sstable/LargePartitionsTest.java
index 45fd712..7ff7845 100644
--- a/test/unit/org/apache/cassandra/io/sstable/LargePartitionsTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/LargePartitionsTest.java
@@ -22,9 +22,7 @@ import java.util.concurrent.ThreadLocalRandom;
 
 import org.junit.Ignore;
 import org.junit.Test;
-import org.junit.runner.RunWith;
 
-import org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.cql3.CQLTester;
 import org.apache.cassandra.cql3.UntypedResultSet;
 import org.apache.cassandra.metrics.CacheMetrics;
@@ -34,7 +32,6 @@ import org.apache.cassandra.service.CacheService;
  * Test intended to manually measure GC pressure to write and read partitions of different size
  * for CASSANDRA-11206.
  */
-@RunWith(OrderedJUnit4ClassRunner.class)
 @Ignore // all these tests take very, very long - so only run them manually
 public class LargePartitionsTest extends CQLTester
 {
diff --git a/test/unit/org/apache/cassandra/locator/TokenMetadataTest.java b/test/unit/org/apache/cassandra/locator/TokenMetadataTest.java
index 20eb7a0..9e34b93 100644
--- a/test/unit/org/apache/cassandra/locator/TokenMetadataTest.java
+++ b/test/unit/org/apache/cassandra/locator/TokenMetadataTest.java
@@ -31,9 +31,7 @@ import com.google.common.collect.Multimap;
 
 import org.junit.BeforeClass;
 import org.junit.Test;
-import org.junit.runner.RunWith;
 
-import org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.service.StorageService;
@@ -46,7 +44,6 @@ import static org.junit.Assert.assertTrue;
 import static org.apache.cassandra.Util.token;
 
 
-@RunWith(OrderedJUnit4ClassRunner.class)
 public class TokenMetadataTest
 {
     public final static String ONE = "1";
diff --git a/test/unit/org/apache/cassandra/metrics/BatchMetricsTest.java b/test/unit/org/apache/cassandra/metrics/BatchMetricsTest.java
index c3bf794..439ed73 100644
--- a/test/unit/org/apache/cassandra/metrics/BatchMetricsTest.java
+++ b/test/unit/org/apache/cassandra/metrics/BatchMetricsTest.java
@@ -23,13 +23,11 @@ import java.util.concurrent.TimeUnit;
 
 import org.junit.BeforeClass;
 import org.junit.Test;
-import org.junit.runner.RunWith;
 
 import com.datastax.driver.core.BatchStatement;
 import com.datastax.driver.core.Cluster;
 import com.datastax.driver.core.PreparedStatement;
 import com.datastax.driver.core.Session;
-import org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.exceptions.ConfigurationException;
@@ -44,7 +42,6 @@ import static org.quicktheories.QuickTheory.qt;
 import static org.quicktheories.generators.Generate.intArrays;
 import static org.quicktheories.generators.SourceDSL.integers;
 
-@RunWith(OrderedJUnit4ClassRunner.class)
 public class BatchMetricsTest extends SchemaLoader
 {
     private static final int MAX_ROUNDS_TO_PERFORM = 3;
diff --git a/test/unit/org/apache/cassandra/metrics/BufferPoolMetricsTest.java b/test/unit/org/apache/cassandra/metrics/BufferPoolMetricsTest.java
index 5e0286f..8db86d8 100644
--- a/test/unit/org/apache/cassandra/metrics/BufferPoolMetricsTest.java
+++ b/test/unit/org/apache/cassandra/metrics/BufferPoolMetricsTest.java
@@ -23,9 +23,7 @@ import java.util.Random;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
-import org.junit.runner.RunWith;
 
-import org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.compress.BufferType;
@@ -35,7 +33,6 @@ import static org.assertj.core.api.Assertions.assertThat;
 import static org.assertj.core.api.Assertions.assertThatExceptionOfType;
 import static org.junit.Assert.assertEquals;
 
-@RunWith(OrderedJUnit4ClassRunner.class)
 public class BufferPoolMetricsTest
 {
     private BufferPool bufferPool;
diff --git a/test/unit/org/apache/cassandra/metrics/CQLMetricsTest.java b/test/unit/org/apache/cassandra/metrics/CQLMetricsTest.java
index e186998..fbd825e 100644
--- a/test/unit/org/apache/cassandra/metrics/CQLMetricsTest.java
+++ b/test/unit/org/apache/cassandra/metrics/CQLMetricsTest.java
@@ -22,12 +22,10 @@ import java.io.IOException;
 
 import org.junit.BeforeClass;
 import org.junit.Test;
-import org.junit.runner.RunWith;
 
 import com.datastax.driver.core.Cluster;
 import com.datastax.driver.core.PreparedStatement;
 import com.datastax.driver.core.Session;
-import org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.schema.Schema;
@@ -37,7 +35,6 @@ import org.apache.cassandra.service.EmbeddedCassandraService;
 
 import static junit.framework.Assert.assertEquals;
 
-@RunWith(OrderedJUnit4ClassRunner.class)
 public class CQLMetricsTest extends SchemaLoader
 {
     private static EmbeddedCassandraService cassandra;
diff --git a/test/unit/org/apache/cassandra/metrics/CacheMetricsTest.java b/test/unit/org/apache/cassandra/metrics/CacheMetricsTest.java
index b2bf6af..32a38c9 100644
--- a/test/unit/org/apache/cassandra/metrics/CacheMetricsTest.java
+++ b/test/unit/org/apache/cassandra/metrics/CacheMetricsTest.java
@@ -23,16 +23,13 @@ import java.util.Iterator;
 import java.util.Map;
 
 import org.junit.Test;
-import org.junit.runner.RunWith;
 
-import org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.cache.CacheSize;
 import org.apache.cassandra.cache.ICache;
 import org.apache.cassandra.cache.InstrumentingCache;
 
 import static org.junit.Assert.assertEquals;
 
-@RunWith(OrderedJUnit4ClassRunner.class)
 public class CacheMetricsTest
 {
     private static final long capacity = 65536;
diff --git a/test/unit/org/apache/cassandra/metrics/TableMetricsTest.java b/test/unit/org/apache/cassandra/metrics/TableMetricsTest.java
index 3ce219b..1e8175e 100644
--- a/test/unit/org/apache/cassandra/metrics/TableMetricsTest.java
+++ b/test/unit/org/apache/cassandra/metrics/TableMetricsTest.java
@@ -26,13 +26,11 @@ import java.util.stream.Stream;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
-import org.junit.runner.RunWith;
 
 import com.datastax.driver.core.BatchStatement;
 import com.datastax.driver.core.Cluster;
 import com.datastax.driver.core.PreparedStatement;
 import com.datastax.driver.core.Session;
-import org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.ColumnFamilyStore;
@@ -43,7 +41,6 @@ import org.apache.cassandra.service.EmbeddedCassandraService;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
-@RunWith(OrderedJUnit4ClassRunner.class)
 public class TableMetricsTest extends SchemaLoader
 {
     private static Session session;
@@ -261,7 +258,7 @@ public class TableMetricsTest extends SchemaLoader
     @Test
     public void testViewMetricsCleanupOnDrop()
     {
-        String tableName = TABLE + "_metrics_cleanup";
+        String tableName = TABLE + "_2_metrics_cleanup";
         String viewName = TABLE + "_materialized_view_cleanup";
         CassandraMetricsRegistry registry = CassandraMetricsRegistry.Metrics;
         Supplier<Stream<String>> metrics = () -> registry.getNames().stream().filter(m -> m.contains(viewName));
diff --git a/test/unit/org/apache/cassandra/schema/MigrationManagerDropKSTest.java b/test/unit/org/apache/cassandra/schema/MigrationManagerDropKSTest.java
new file mode 100644
index 0000000..e28a0df
--- /dev/null
+++ b/test/unit/org/apache/cassandra/schema/MigrationManagerDropKSTest.java
@@ -0,0 +1,106 @@
+/*
+ * 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.schema;
+
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.cql3.QueryProcessor;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Directories;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.exceptions.ConfigurationException;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+public class MigrationManagerDropKSTest
+{
+    private static final String KEYSPACE1 = "keyspace1";
+    private static final String TABLE1 = "standard1";
+    private static final String TABLE2 = "standard2";
+
+    @Rule
+    public ExpectedException thrown = ExpectedException.none();
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.startGossiper();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    KeyspaceParams.simple(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, TABLE1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, TABLE2));
+    }
+    @Test
+    public void dropKS() throws ConfigurationException
+    {
+        // sanity
+        final KeyspaceMetadata ks = Schema.instance.getKeyspaceMetadata(KEYSPACE1);
+        assertNotNull(ks);
+        final TableMetadata cfm = ks.tables.getNullable(TABLE2);
+        assertNotNull(cfm);
+
+        // write some data, force a flush, then verify that files exist on disk.
+        for (int i = 0; i < 100; i++)
+            QueryProcessor.executeInternal(String.format("INSERT INTO %s.%s (key, name, val) VALUES (?, ?, ?)",
+                                                         KEYSPACE1, TABLE2),
+                                           "dropKs", "col" + i, "anyvalue");
+        ColumnFamilyStore cfs = Keyspace.open(cfm.keyspace).getColumnFamilyStore(cfm.name);
+        assertNotNull(cfs);
+        cfs.forceBlockingFlush();
+        assertTrue(!cfs.getDirectories().sstableLister(Directories.OnTxnErr.THROW).list().isEmpty());
+
+        MigrationManager.announceKeyspaceDrop(ks.name);
+
+        assertNull(Schema.instance.getKeyspaceMetadata(ks.name));
+
+        // write should fail.
+        boolean success = true;
+        try
+        {
+            QueryProcessor.executeInternal(String.format("INSERT INTO %s.%s (key, name, val) VALUES (?, ?, ?)",
+                                                         KEYSPACE1, TABLE2),
+                                           "dropKs", "col0", "anyvalue");
+        }
+        catch (Throwable th)
+        {
+            success = false;
+        }
+        assertFalse("This mutation should have failed since the KS no longer exists.", success);
+
+        // reads should fail too.
+        boolean threw = false;
+        try
+        {
+            Keyspace.open(ks.name);
+        }
+        catch (Throwable th)
+        {
+            threw = true;
+        }
+        assertTrue(threw);
+    }
+}
diff --git a/test/unit/org/apache/cassandra/schema/MigrationManagerTest.java b/test/unit/org/apache/cassandra/schema/MigrationManagerTest.java
index ff58151..9a26032 100644
--- a/test/unit/org/apache/cassandra/schema/MigrationManagerTest.java
+++ b/test/unit/org/apache/cassandra/schema/MigrationManagerTest.java
@@ -30,9 +30,7 @@ import org.junit.BeforeClass;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExpectedException;
-import org.junit.runner.RunWith;
 
-import org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
 import org.apache.cassandra.cql3.ColumnIdentifier;
@@ -63,7 +61,6 @@ import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 
 
-@RunWith(OrderedJUnit4ClassRunner.class)
 public class MigrationManagerTest
 {
     private static final String KEYSPACE1 = "keyspace1";
@@ -261,56 +258,6 @@ public class MigrationManagerTest
     }
 
     @Test
-    public void dropKS() throws ConfigurationException
-    {
-        // sanity
-        final KeyspaceMetadata ks = Schema.instance.getKeyspaceMetadata(KEYSPACE1);
-        assertNotNull(ks);
-        final TableMetadata cfm = ks.tables.getNullable(TABLE2);
-        assertNotNull(cfm);
-
-        // write some data, force a flush, then verify that files exist on disk.
-        for (int i = 0; i < 100; i++)
-            QueryProcessor.executeInternal(String.format("INSERT INTO %s.%s (key, name, val) VALUES (?, ?, ?)",
-                                                         KEYSPACE1, TABLE2),
-                                           "dropKs", "col" + i, "anyvalue");
-        ColumnFamilyStore cfs = Keyspace.open(cfm.keyspace).getColumnFamilyStore(cfm.name);
-        assertNotNull(cfs);
-        cfs.forceBlockingFlush();
-        assertTrue(!cfs.getDirectories().sstableLister(Directories.OnTxnErr.THROW).list().isEmpty());
-
-        MigrationManager.announceKeyspaceDrop(ks.name);
-
-        assertNull(Schema.instance.getKeyspaceMetadata(ks.name));
-
-        // write should fail.
-        boolean success = true;
-        try
-        {
-            QueryProcessor.executeInternal(String.format("INSERT INTO %s.%s (key, name, val) VALUES (?, ?, ?)",
-                                                         KEYSPACE1, TABLE2),
-                                           "dropKs", "col0", "anyvalue");
-        }
-        catch (Throwable th)
-        {
-            success = false;
-        }
-        assertFalse("This mutation should have failed since the KS no longer exists.", success);
-
-        // reads should fail too.
-        boolean threw = false;
-        try
-        {
-            Keyspace.open(ks.name);
-        }
-        catch (Throwable th)
-        {
-            threw = true;
-        }
-        assertTrue(threw);
-    }
-
-    @Test
     public void dropKSUnflushed() throws ConfigurationException
     {
         // sanity
diff --git a/test/unit/org/apache/cassandra/service/GCInspectorTest.java b/test/unit/org/apache/cassandra/service/GCInspectorTest.java
index 0c5ddef..7e3a3d3 100644
--- a/test/unit/org/apache/cassandra/service/GCInspectorTest.java
+++ b/test/unit/org/apache/cassandra/service/GCInspectorTest.java
@@ -17,15 +17,12 @@
  */
 package org.apache.cassandra.service;
 
-import org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
-import org.junit.runner.RunWith;
 
-@RunWith(OrderedJUnit4ClassRunner.class)
 public class GCInspectorTest
 {
     
diff --git a/test/unit/org/apache/cassandra/service/QueryPagerTest.java b/test/unit/org/apache/cassandra/service/QueryPagerTest.java
index 11d1cb0..50bb3e2 100644
--- a/test/unit/org/apache/cassandra/service/QueryPagerTest.java
+++ b/test/unit/org/apache/cassandra/service/QueryPagerTest.java
@@ -24,8 +24,6 @@ import java.util.*;
 
 import org.junit.BeforeClass;
 import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
 
 import org.apache.cassandra.*;
 import org.apache.cassandra.cql3.statements.schema.CreateTableStatement;
@@ -51,7 +49,6 @@ import static org.apache.cassandra.cql3.QueryProcessor.executeInternal;
 import static org.apache.cassandra.utils.ByteBufferUtil.bytes;
 import static org.junit.Assert.*;
 
-@RunWith(OrderedJUnit4ClassRunner.class)
 public class QueryPagerTest
 {
     public static final String KEYSPACE1 = "QueryPagerTest";
diff --git a/test/unit/org/apache/cassandra/service/StorageServiceServerM3PTest.java b/test/unit/org/apache/cassandra/service/StorageServiceServerM3PTest.java
new file mode 100644
index 0000000..69b0642
--- /dev/null
+++ b/test/unit/org/apache/cassandra/service/StorageServiceServerM3PTest.java
@@ -0,0 +1,68 @@
+/*
+ * 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.service;
+
+import java.io.File;
+
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.commitlog.CommitLog;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.gms.Gossiper;
+import org.apache.cassandra.locator.IEndpointSnitch;
+import org.apache.cassandra.locator.PropertyFileSnitch;
+
+import static org.apache.cassandra.ServerTestUtils.cleanup;
+import static org.apache.cassandra.ServerTestUtils.mkdirs;
+import static org.junit.Assert.assertTrue;
+
+public class StorageServiceServerM3PTest
+{
+    @BeforeClass
+    public static void setUp() throws ConfigurationException
+    {
+        System.setProperty(Gossiper.Props.DISABLE_THREAD_VALIDATION, "true");
+        DatabaseDescriptor.daemonInitialization();
+        CommitLog.instance.start();
+        IEndpointSnitch snitch = new PropertyFileSnitch();
+        DatabaseDescriptor.setEndpointSnitch(snitch);
+        Keyspace.setInitialized();
+    }
+
+    @Test
+    public void testRegularMode() throws ConfigurationException
+    {
+        mkdirs();
+        cleanup();
+        StorageService.instance.initServer(0);
+        for (String path : DatabaseDescriptor.getAllDataFileLocations())
+        {
+            // verify that storage directories are there.
+            assertTrue(new File(path).exists());
+        }
+        // a proper test would be to call decommission here, but decommission() mixes both shutdown and datatransfer
+        // calls.  This test is only interested in the shutdown-related items which a properly handled by just
+        // stopping the client.
+        //StorageService.instance.decommission();
+        StorageService.instance.stopClient();
+    }
+}
diff --git a/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java b/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java
index 9dda9f9..8c6c9aa 100644
--- a/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java
+++ b/test/unit/org/apache/cassandra/service/StorageServiceServerTest.java
@@ -34,9 +34,7 @@ import org.apache.cassandra.db.SystemKeyspace;
 import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
-import org.junit.runner.RunWith;
 
-import org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.audit.AuditLogManager;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.Keyspace;
@@ -59,13 +57,10 @@ import org.apache.cassandra.schema.*;
 import org.apache.cassandra.utils.FBUtilities;
 import org.assertj.core.api.Assertions;
 
-import static org.apache.cassandra.ServerTestUtils.cleanup;
-import static org.apache.cassandra.ServerTestUtils.mkdirs;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assume.assumeTrue;
 
-@RunWith(OrderedJUnit4ClassRunner.class)
 public class StorageServiceServerTest
 {
     @BeforeClass
@@ -80,24 +75,6 @@ public class StorageServiceServerTest
     }
 
     @Test
-    public void testRegularMode() throws ConfigurationException
-    {
-        mkdirs();
-        cleanup();
-        StorageService.instance.initServer(0);
-        for (String path : DatabaseDescriptor.getAllDataFileLocations())
-        {
-            // verify that storage directories are there.
-            assertTrue(new File(path).exists());
-        }
-        // a proper test would be to call decommission here, but decommission() mixes both shutdown and datatransfer
-        // calls.  This test is only interested in the shutdown-related items which a properly handled by just
-        // stopping the client.
-        //StorageService.instance.decommission();
-        StorageService.instance.stopClient();
-    }
-
-    @Test
     public void testGetAllRangesEmpty()
     {
         List<Token> toks = Collections.emptyList();
@@ -709,7 +686,7 @@ public class StorageServiceServerTest
             Assert.assertFalse(StorageService.instance.isReplacingSameHostAddressAndHostId(differentHostId));
 
             final String hostAddress = FBUtilities.getBroadcastAddressAndPort().getHostAddress(false);
-            UUID localHostId = SystemKeyspace.getLocalHostId();
+            UUID localHostId = SystemKeyspace.getOrInitializeLocalHostId();
             Gossiper.instance.initializeNodeUnsafe(FBUtilities.getBroadcastAddressAndPort(), localHostId, 1);
 
             // Check detects replacing the same host address with the same hostid
diff --git a/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java b/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
index d99d35f..7b00f93 100644
--- a/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
+++ b/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
@@ -29,12 +29,11 @@ import com.google.common.util.concurrent.MoreExecutors;
 import org.apache.cassandra.locator.RangesAtEndpoint;
 import org.junit.BeforeClass;
 import org.junit.Test;
-import org.junit.runner.RunWith;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.junit.Assert;
-import org.apache.cassandra.OrderedJUnit4ClassRunner;
+
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
 import org.apache.cassandra.db.streaming.CassandraOutgoingFile;
@@ -63,7 +62,6 @@ import static org.apache.cassandra.SchemaLoader.standardCFMD;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
 
-@RunWith(OrderedJUnit4ClassRunner.class)
 public class StreamingTransferTest
 {
     private static final Logger logger = LoggerFactory.getLogger(StreamingTransferTest.class);
diff --git a/test/unit/org/apache/cassandra/tools/BulkLoaderTest.java b/test/unit/org/apache/cassandra/tools/BulkLoaderTest.java
index 382f352..a46cbf9 100644
--- a/test/unit/org/apache/cassandra/tools/BulkLoaderTest.java
+++ b/test/unit/org/apache/cassandra/tools/BulkLoaderTest.java
@@ -19,22 +19,18 @@
 package org.apache.cassandra.tools;
 
 import org.junit.Test;
-import org.junit.runner.RunWith;
 
 import com.datastax.driver.core.exceptions.NoHostAvailableException;
-import org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.tools.ToolRunner.ToolResult;
 import org.hamcrest.CoreMatchers;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThat;
-import static org.junit.Assert.fail;
 
-@RunWith(OrderedJUnit4ClassRunner.class)
 public class BulkLoaderTest extends OfflineToolUtils
 {
     @Test
-    public void testBulkLoader_NoArgs() throws Exception
+    public void testBulkLoader_NoArgs()
     {
         ToolResult tool = ToolRunner.invokeClass(BulkLoader.class);
         assertEquals(1, tool.getExitCode());
diff --git a/test/unit/org/apache/cassandra/tools/CompactionStressTest.java b/test/unit/org/apache/cassandra/tools/CompactionStressTest.java
index 09b82fe..7125f2f 100644
--- a/test/unit/org/apache/cassandra/tools/CompactionStressTest.java
+++ b/test/unit/org/apache/cassandra/tools/CompactionStressTest.java
@@ -21,12 +21,9 @@ package org.apache.cassandra.tools;
 import java.io.File;
 
 import org.junit.Test;
-import org.junit.runner.RunWith;
 
-import org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.tools.ToolRunner.ToolResult;
 
-@RunWith(OrderedJUnit4ClassRunner.class)
 public class CompactionStressTest extends OfflineToolUtils
 {
     @Test
diff --git a/test/unit/org/apache/cassandra/tools/GetVersionTest.java b/test/unit/org/apache/cassandra/tools/GetVersionTest.java
index 1feeb45..b23c1fe 100644
--- a/test/unit/org/apache/cassandra/tools/GetVersionTest.java
+++ b/test/unit/org/apache/cassandra/tools/GetVersionTest.java
@@ -19,12 +19,9 @@
 package org.apache.cassandra.tools;
 
 import org.junit.Test;
-import org.junit.runner.RunWith;
 
-import org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.tools.ToolRunner.ToolResult;
 
-@RunWith(OrderedJUnit4ClassRunner.class)
 public class GetVersionTest extends OfflineToolUtils
 {
     @Test
diff --git a/test/unit/org/apache/cassandra/tools/NodeToolGossipInfoTest.java b/test/unit/org/apache/cassandra/tools/NodeToolGossipInfoTest.java
index caca5ae..8e9735b 100644
--- a/test/unit/org/apache/cassandra/tools/NodeToolGossipInfoTest.java
+++ b/test/unit/org/apache/cassandra/tools/NodeToolGossipInfoTest.java
@@ -25,9 +25,7 @@ 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.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.cql3.CQLTester;
 import org.apache.cassandra.net.Message;
 import org.apache.cassandra.net.MessagingService;
@@ -40,7 +38,6 @@ import static org.apache.cassandra.net.Verb.ECHO_REQ;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
-@RunWith(OrderedJUnit4ClassRunner.class)
 public class NodeToolGossipInfoTest extends CQLTester
 {
     private static NodeProbe probe;
diff --git a/test/unit/org/apache/cassandra/tools/NodeToolTPStatsTest.java b/test/unit/org/apache/cassandra/tools/NodeToolTPStatsTest.java
index b507bcd..6de2b5b 100644
--- a/test/unit/org/apache/cassandra/tools/NodeToolTPStatsTest.java
+++ b/test/unit/org/apache/cassandra/tools/NodeToolTPStatsTest.java
@@ -30,10 +30,8 @@ import org.apache.commons.lang3.tuple.Pair;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
-import org.junit.runner.RunWith;
 
 import com.fasterxml.jackson.databind.ObjectMapper;
-import org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.cql3.CQLTester;
 import org.apache.cassandra.net.Message;
 import org.apache.cassandra.net.MessagingService;
@@ -49,7 +47,6 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotEquals;
 import static org.junit.Assert.assertTrue;
 
-@RunWith(OrderedJUnit4ClassRunner.class)
 public class NodeToolTPStatsTest extends CQLTester
 {
     private static NodeProbe probe;
diff --git a/test/unit/org/apache/cassandra/tools/NodetoolNetStatsTest.java b/test/unit/org/apache/cassandra/tools/NodetoolNetStatsTest.java
index 7ef2195..bcf8704 100644
--- a/test/unit/org/apache/cassandra/tools/NodetoolNetStatsTest.java
+++ b/test/unit/org/apache/cassandra/tools/NodetoolNetStatsTest.java
@@ -28,9 +28,7 @@ import java.util.List;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
-import org.junit.runner.RunWith;
 
-import org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.cql3.CQLTester;
 import org.apache.cassandra.locator.InetAddressAndPort;
 import org.apache.cassandra.net.Message;
@@ -50,7 +48,6 @@ import org.hamcrest.CoreMatchers;
 import static org.apache.cassandra.net.Verb.ECHO_REQ;
 import static org.junit.Assert.assertThat;
 
-@RunWith(OrderedJUnit4ClassRunner.class)
 public class NodetoolNetStatsTest extends CQLTester
 {
     private static NodeProbe probe;
diff --git a/test/unit/org/apache/cassandra/tools/OfflineToolUtils.java b/test/unit/org/apache/cassandra/tools/OfflineToolUtils.java
index bcb7020..c1678d7 100644
--- a/test/unit/org/apache/cassandra/tools/OfflineToolUtils.java
+++ b/test/unit/org/apache/cassandra/tools/OfflineToolUtils.java
@@ -34,6 +34,7 @@ import java.util.stream.Collectors;
 
 import org.apache.commons.io.FileUtils;
 
+import org.junit.AfterClass;
 import org.junit.BeforeClass;
 
 import org.slf4j.LoggerFactory;
diff --git a/test/unit/org/apache/cassandra/tools/SSTableExpiredBlockersTest.java b/test/unit/org/apache/cassandra/tools/SSTableExpiredBlockersTest.java
index 0476453..4fc2cbe 100644
--- a/test/unit/org/apache/cassandra/tools/SSTableExpiredBlockersTest.java
+++ b/test/unit/org/apache/cassandra/tools/SSTableExpiredBlockersTest.java
@@ -19,9 +19,7 @@
 package org.apache.cassandra.tools;
 
 import org.junit.Test;
-import org.junit.runner.RunWith;
 
-import org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.tools.ToolRunner.ToolResult;
 import org.assertj.core.api.Assertions;
 import org.hamcrest.CoreMatchers;
@@ -29,7 +27,6 @@ import org.hamcrest.CoreMatchers;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThat;
 
-@RunWith(OrderedJUnit4ClassRunner.class)
 public class SSTableExpiredBlockersTest extends OfflineToolUtils
 {
     @Test
diff --git a/test/unit/org/apache/cassandra/tools/SSTableExportTest.java b/test/unit/org/apache/cassandra/tools/SSTableExportTest.java
index eb082dd..ecd5f6e 100644
--- a/test/unit/org/apache/cassandra/tools/SSTableExportTest.java
+++ b/test/unit/org/apache/cassandra/tools/SSTableExportTest.java
@@ -24,12 +24,10 @@ import java.util.Map;
 
 import org.junit.BeforeClass;
 import org.junit.Test;
-import org.junit.runner.RunWith;
 
 import com.fasterxml.jackson.core.type.TypeReference;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.databind.exc.MismatchedInputException;
-import org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.tools.ToolRunner.ToolResult;
 import org.assertj.core.api.Assertions;
 
@@ -43,7 +41,6 @@ import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
-@RunWith(OrderedJUnit4ClassRunner.class) // tests calling assertSchemaNotLoaded should be the first ones
 public class SSTableExportTest extends OfflineToolUtils
 {
     private static final ObjectMapper mapper = new ObjectMapper();
@@ -246,7 +243,6 @@ public class SSTableExportTest extends OfflineToolUtils
      * {@code false} if the test shoudln't load the schema in any case. Note that a test not loading the schema can
      * still end with the schema loaded if a previous test already loaded it, so we should always run first the tests
      * that don't load the schema, and then the ones that may or may not load it. We also need to use the
-     * {@link OrderedJUnit4ClassRunner} runner to guarantee the desired run order.
      */
     private void assertPostTestEnv(boolean maybeLoadsSchema)
     {
diff --git a/test/unit/org/apache/cassandra/tools/SSTableLevelResetterTest.java b/test/unit/org/apache/cassandra/tools/SSTableLevelResetterTest.java
index 3f1c892..a6606bf 100644
--- a/test/unit/org/apache/cassandra/tools/SSTableLevelResetterTest.java
+++ b/test/unit/org/apache/cassandra/tools/SSTableLevelResetterTest.java
@@ -19,9 +19,7 @@
 package org.apache.cassandra.tools;
 
 import org.junit.Test;
-import org.junit.runner.RunWith;
 
-import org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.tools.ToolRunner.ToolResult;
 import org.assertj.core.api.Assertions;
 import org.hamcrest.CoreMatchers;
@@ -29,7 +27,6 @@ import org.hamcrest.CoreMatchers;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThat;
 
-@RunWith(OrderedJUnit4ClassRunner.class)
 public class SSTableLevelResetterTest extends OfflineToolUtils
 {
     @Test
diff --git a/test/unit/org/apache/cassandra/tools/SSTableMetadataViewerTest.java b/test/unit/org/apache/cassandra/tools/SSTableMetadataViewerTest.java
index 7fd1353..2f6e64b 100644
--- a/test/unit/org/apache/cassandra/tools/SSTableMetadataViewerTest.java
+++ b/test/unit/org/apache/cassandra/tools/SSTableMetadataViewerTest.java
@@ -27,9 +27,7 @@ import org.apache.commons.lang3.tuple.Pair;
 
 import org.junit.BeforeClass;
 import org.junit.Test;
-import org.junit.runner.RunWith;
 
-import org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.tools.ToolRunner.ToolResult;
 import org.assertj.core.api.Assertions;
 import org.hamcrest.CoreMatchers;
@@ -38,7 +36,6 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 
-@RunWith(OrderedJUnit4ClassRunner.class)
 public class SSTableMetadataViewerTest extends OfflineToolUtils
 {
     private static String sstable;
diff --git a/test/unit/org/apache/cassandra/tools/SSTableOfflineRelevelTest.java b/test/unit/org/apache/cassandra/tools/SSTableOfflineRelevelTest.java
index a49dd49..ee07f99 100644
--- a/test/unit/org/apache/cassandra/tools/SSTableOfflineRelevelTest.java
+++ b/test/unit/org/apache/cassandra/tools/SSTableOfflineRelevelTest.java
@@ -19,9 +19,7 @@
 package org.apache.cassandra.tools;
 
 import org.junit.Test;
-import org.junit.runner.RunWith;
 
-import org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.tools.ToolRunner.ToolResult;
 import org.assertj.core.api.Assertions;
 import org.hamcrest.CoreMatchers;
@@ -29,7 +27,6 @@ import org.hamcrest.CoreMatchers;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThat;
 
-@RunWith(OrderedJUnit4ClassRunner.class)
 public class SSTableOfflineRelevelTest extends OfflineToolUtils
 {
     @Test
diff --git a/test/unit/org/apache/cassandra/tools/SSTableRepairedAtSetterTest.java b/test/unit/org/apache/cassandra/tools/SSTableRepairedAtSetterTest.java
index e1f5f95..859b6b8 100644
--- a/test/unit/org/apache/cassandra/tools/SSTableRepairedAtSetterTest.java
+++ b/test/unit/org/apache/cassandra/tools/SSTableRepairedAtSetterTest.java
@@ -21,12 +21,9 @@ package org.apache.cassandra.tools;
 import java.io.File;
 import java.io.IOException;
 import java.nio.file.Files;
-import java.nio.file.Paths;
 
 import org.junit.Test;
-import org.junit.runner.RunWith;
 
-import org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.tools.ToolRunner.ToolResult;
 import org.assertj.core.api.Assertions;
@@ -35,7 +32,6 @@ import org.hamcrest.CoreMatchers;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThat;
 
-@RunWith(OrderedJUnit4ClassRunner.class)
 public class SSTableRepairedAtSetterTest extends OfflineToolUtils
 {
     @Test
diff --git a/test/unit/org/apache/cassandra/tools/StandaloneSSTableUtilTest.java b/test/unit/org/apache/cassandra/tools/StandaloneSSTableUtilTest.java
index efe7396..633ffce 100644
--- a/test/unit/org/apache/cassandra/tools/StandaloneSSTableUtilTest.java
+++ b/test/unit/org/apache/cassandra/tools/StandaloneSSTableUtilTest.java
@@ -23,9 +23,7 @@ import java.util.Arrays;
 import org.apache.commons.lang3.tuple.Pair;
 
 import org.junit.Test;
-import org.junit.runner.RunWith;
 
-import org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.tools.ToolRunner.ToolResult;
 import org.assertj.core.api.Assertions;
 import org.hamcrest.CoreMatchers;
@@ -33,25 +31,9 @@ import org.hamcrest.CoreMatchers;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThat;
 
-@RunWith(OrderedJUnit4ClassRunner.class)
 public class StandaloneSSTableUtilTest extends OfflineToolUtils
 {
     @Test
-    public void testNoArgsPrintsHelp()
-    {
-        ToolResult tool = ToolRunner.invokeClass(StandaloneSSTableUtil.class);
-        assertThat(tool.getStdout(), CoreMatchers.containsStringIgnoringCase("usage:"));
-        assertThat(tool.getCleanedStderr(), CoreMatchers.containsStringIgnoringCase("Missing arguments"));
-        assertEquals(1, tool.getExitCode());
-        assertNoUnexpectedThreadsStarted(null, null);
-        assertSchemaNotLoaded();
-        assertCLSMNotLoaded();
-        assertSystemKSNotLoaded();
-        assertKeyspaceNotLoaded();
-        assertServerNotLoaded();
-    }
-
-    @Test
     public void testWrongArgFailsAndPrintsHelp()
     {
         ToolResult tool = ToolRunner.invokeClass(StandaloneSSTableUtil.class, "--debugwrong", "system_schema", "tables");
diff --git a/test/unit/org/apache/cassandra/tools/StandaloneScrubberTest.java b/test/unit/org/apache/cassandra/tools/StandaloneScrubberTest.java
index ea1499e..5bebf30 100644
--- a/test/unit/org/apache/cassandra/tools/StandaloneScrubberTest.java
+++ b/test/unit/org/apache/cassandra/tools/StandaloneScrubberTest.java
@@ -23,9 +23,7 @@ import java.util.Arrays;
 import org.apache.commons.lang3.tuple.Pair;
 
 import org.junit.Test;
-import org.junit.runner.RunWith;
 
-import org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.tools.ToolRunner.ToolResult;
 import org.assertj.core.api.Assertions;
 import org.hamcrest.CoreMatchers;
@@ -40,25 +38,9 @@ import static org.junit.Assert.assertTrue;
  * For TTL sstable scrubbing tests look at {@link TTLTest}
  */
 
-@RunWith(OrderedJUnit4ClassRunner.class)
 public class StandaloneScrubberTest extends OfflineToolUtils
 {
     @Test
-    public void testNoArgsPrintsHelp()
-    {
-        ToolResult tool = ToolRunner.invokeClass(StandaloneScrubber.class);
-        assertThat(tool.getStdout(), CoreMatchers.containsStringIgnoringCase("usage:"));
-        assertThat(tool.getCleanedStderr(), CoreMatchers.containsStringIgnoringCase("Missing arguments"));
-        assertEquals(1, tool.getExitCode());
-        assertNoUnexpectedThreadsStarted(null, null);
-        assertSchemaNotLoaded();
-        assertCLSMNotLoaded();
-        assertSystemKSNotLoaded();
-        assertKeyspaceNotLoaded();
-        assertServerNotLoaded();
-    }
-
-    @Test
     public void testMaybeChangeDocs()
     {
         // If you added, modified options or help, please update docs if necessary
diff --git a/test/unit/org/apache/cassandra/tools/StandaloneSplitterTest.java b/test/unit/org/apache/cassandra/tools/StandaloneSplitterTest.java
index 287275b..d28558f 100644
--- a/test/unit/org/apache/cassandra/tools/StandaloneSplitterTest.java
+++ b/test/unit/org/apache/cassandra/tools/StandaloneSplitterTest.java
@@ -24,9 +24,7 @@ import org.apache.commons.lang3.tuple.Pair;
 
 import org.junit.BeforeClass;
 import org.junit.Test;
-import org.junit.runner.RunWith;
 
-import org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.tools.ToolRunner.ToolResult;
 import org.assertj.core.api.Assertions;
 import org.hamcrest.CoreMatchers;
@@ -34,7 +32,6 @@ import org.hamcrest.CoreMatchers;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThat;
 
-@RunWith(OrderedJUnit4ClassRunner.class)
 public class StandaloneSplitterTest extends OfflineToolUtils
 {
     // Note: StandaloneSplitter modifies sstables
@@ -49,21 +46,6 @@ public class StandaloneSplitterTest extends OfflineToolUtils
     }
 
     @Test
-    public void testNoArgsPrintsHelp()
-    {
-        ToolResult tool = ToolRunner.invokeClass(StandaloneSplitter.class);
-        assertThat(tool.getStdout(), CoreMatchers.containsStringIgnoringCase("usage:"));
-        assertThat(tool.getCleanedStderr(), CoreMatchers.containsStringIgnoringCase("No sstables to split"));
-        assertEquals(1, tool.getExitCode());
-        assertNoUnexpectedThreadsStarted(null, null);
-        assertSchemaNotLoaded();
-        assertCLSMNotLoaded();
-        assertSystemKSNotLoaded();
-        assertKeyspaceNotLoaded();
-        assertServerNotLoaded();
-    }
-
-    @Test
     public void testMaybeChangeDocs()
     {
         // If you added, modified options or help, please update docs if necessary
diff --git a/test/unit/org/apache/cassandra/tools/StandaloneSplitterWithCQLTesterTest.java b/test/unit/org/apache/cassandra/tools/StandaloneSplitterWithCQLTesterTest.java
index 9785d84..62e0166 100644
--- a/test/unit/org/apache/cassandra/tools/StandaloneSplitterWithCQLTesterTest.java
+++ b/test/unit/org/apache/cassandra/tools/StandaloneSplitterWithCQLTesterTest.java
@@ -19,19 +19,15 @@
 package org.apache.cassandra.tools;
 
 import java.io.File;
-import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 import java.util.Set;
 import java.util.stream.Collectors;
 
-import com.google.common.io.Files;
-
+import org.junit.Before;
 import org.junit.Test;
-import org.junit.runner.RunWith;
 
-import org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.cql3.CQLTester;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
@@ -41,32 +37,23 @@ import org.assertj.core.api.Assertions;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
-@RunWith(OrderedJUnit4ClassRunner.class)
 public class StandaloneSplitterWithCQLTesterTest extends CQLTester
 {
     private static String sstableFileName;
     private static File sstablesDir;
-    private static File sstablesBackupDir;
     private static List<File> origSstables;
 
-    // CQLTester post test method cleanup needs to be avoided by overriding as it'd clean all sstables, env, etc.
-    @Override
-    public void afterTest() throws Throwable
-    {
-    }
-
-    @Test
-    public void setupEnv() throws Throwable
+    @Before
+    public void before() throws Throwable
     {
-        // Stop the server after setup as we're going to be changing things under it's feet
         setupTestSstables();
         tearDownClass();
+        SSTableReader.resetTidying();
     }
 
     @Test
     public void testMinFileSizeCheck() throws Throwable
     {
-        restoreOrigSstables();
         ToolResult tool  = ToolRunner.invokeClass(StandaloneSplitter.class, sstableFileName);
         Assertions.assertThat(tool.getStdout()).contains("is less than the split size");
         assertTrue(tool.getCleanedStderr(), tool.getCleanedStderr().isEmpty());
@@ -76,8 +63,6 @@ public class StandaloneSplitterWithCQLTesterTest extends CQLTester
     @Test
     public void testSplittingSSTable() throws Throwable
     {
-        restoreOrigSstables();
-
         ToolResult tool  = ToolRunner.invokeClass(StandaloneSplitter.class, "-s", "1", sstableFileName);
         List<File> splitFiles = Arrays.asList(sstablesDir.listFiles());
         splitFiles.stream().forEach(f -> {
@@ -94,7 +79,6 @@ public class StandaloneSplitterWithCQLTesterTest extends CQLTester
     @Test
     public void testSplittingMultipleSSTables() throws Throwable
     {
-        restoreOrigSstables();
         ArrayList<String> args = new ArrayList<>(Arrays.asList("-s", "1"));
 
         args.addAll(Arrays.asList(sstablesDir.listFiles())
@@ -117,7 +101,6 @@ public class StandaloneSplitterWithCQLTesterTest extends CQLTester
     @Test
     public void testNoSnapshotOption() throws Throwable
     {
-        restoreOrigSstables();
         ToolResult tool  = ToolRunner.invokeClass(StandaloneSplitter.class, "-s", "1", "--no-snapshot", sstableFileName);
         assertTrue(origSstables.size() < Arrays.asList(sstablesDir.listFiles()).size());
         assertTrue(tool.getStdout(), tool.getStdout().isEmpty());
@@ -125,15 +108,9 @@ public class StandaloneSplitterWithCQLTesterTest extends CQLTester
         assertEquals(0, tool.getExitCode());
     }
 
-    @Test
-    public void cleanEnv() throws Throwable
-    {
-        super.afterTest();
-        System.clearProperty(Util.ALLOW_TOOL_REINIT_FOR_TEST);
-    }
-
     private void setupTestSstables() throws Throwable
     {
+        SSTableReader.resetTidying();
         createTable("CREATE TABLE %s (id text primary key, val text)");
         for (int i = 0; i < 100000; i++)
             executeFormattedQuery(formatQuery("INSERT INTO %s (id, val) VALUES (?, ?)"), "mockData" + i, "mockData" + i);
@@ -145,44 +122,7 @@ public class StandaloneSplitterWithCQLTesterTest extends CQLTester
         sstableFileName = sstables.iterator().next().getFilename();
         assertTrue("Generated sstable must be at least 1MB", (new File(sstableFileName)).length() > 1024*1024);
         sstablesDir = new File(sstableFileName).getParentFile();
-        sstablesBackupDir = new File(sstablesDir.getAbsolutePath() + "/testbackup");
-        sstablesBackupDir.mkdir();
         origSstables = Arrays.asList(sstablesDir.listFiles());
-
-        // Back up orig sstables
-        origSstables.stream().forEach(f -> {
-            if (f.isFile())
-                try
-                {
-                    Files.copy(f, new File(sstablesBackupDir.getAbsolutePath() + "/" + f.getName()));
-                }
-                catch(IOException e)
-                {
-                    throw new RuntimeException(e);
-                }
-        });
-
         System.setProperty(Util.ALLOW_TOOL_REINIT_FOR_TEST, "true"); // Necessary for testing
     }
-
-    private void restoreOrigSstables()
-    {
-        Arrays.asList(sstablesDir.listFiles()).stream().forEach(f -> {
-            if (f.isFile())
-                f.delete();
-        });
-        Arrays.asList(sstablesBackupDir.listFiles()).stream().forEach(f -> {
-            if (f.isFile())
-                try
-                {
-                    Files.copy(f, new File(sstablesDir.getAbsolutePath() + "/" + f.getName()));
-                }
-                catch(IOException e)
-                {
-                    throw new RuntimeException(e);
-                }
-        });
-
-        SSTableReader.resetTidying();
-    }
 }
diff --git a/test/unit/org/apache/cassandra/tools/StandaloneUpgraderOnSStablesTest.java b/test/unit/org/apache/cassandra/tools/StandaloneUpgraderOnSStablesTest.java
index 570610b..ca1c649 100644
--- a/test/unit/org/apache/cassandra/tools/StandaloneUpgraderOnSStablesTest.java
+++ b/test/unit/org/apache/cassandra/tools/StandaloneUpgraderOnSStablesTest.java
@@ -28,9 +28,7 @@ import java.util.stream.Collectors;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
-import org.junit.runner.RunWith;
 
-import org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.exceptions.ConfigurationException;
@@ -50,7 +48,6 @@ import static org.junit.Assert.assertEquals;
  * 
  * Caution: heavy hacking ahead.
  */
-@RunWith(OrderedJUnit4ClassRunner.class)
 public class StandaloneUpgraderOnSStablesTest
 {
     String legacyId = LegacySSTableTest.legacyVersions[LegacySSTableTest.legacyVersions.length - 1];
@@ -87,6 +84,9 @@ public class StandaloneUpgraderOnSStablesTest
         List<String> newFiles = getSStableFiles("legacy_tables", "legacy_" + legacyId + "_simple");
         origFiles.removeAll(newFiles);
         assertEquals(0, origFiles.size()); // check previous version files are kept
+
+        // need to make sure the new sstables are live, so that they get truncated later
+        Keyspace.open("legacy_tables").getColumnFamilyStore("legacy_" + legacyId + "_simple").loadNewSSTables();
     }
 
     @Test
@@ -134,6 +134,8 @@ public class StandaloneUpgraderOnSStablesTest
         int origSize = origFiles.size();
         origFiles.removeAll(newFiles);
         assertEquals(origSize, origFiles.size()); // check previous version files are gone
+        // need to make sure the new sstables are live, so that they get truncated later
+        Keyspace.open("legacy_tables").getColumnFamilyStore("legacy_" + legacyId + "_simple").loadNewSSTables();
     }
 
     private List<String> getSStableFiles(String ks, String table) throws StartupException
diff --git a/test/unit/org/apache/cassandra/tools/StandaloneUpgraderTest.java b/test/unit/org/apache/cassandra/tools/StandaloneUpgraderTest.java
index 0f0c6b3..6b83d44 100644
--- a/test/unit/org/apache/cassandra/tools/StandaloneUpgraderTest.java
+++ b/test/unit/org/apache/cassandra/tools/StandaloneUpgraderTest.java
@@ -21,9 +21,7 @@ package org.apache.cassandra.tools;
 import java.util.Arrays;
 
 import org.junit.Test;
-import org.junit.runner.RunWith;
 
-import org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.tools.ToolRunner.ToolResult;
 import org.assertj.core.api.Assertions;
 import org.hamcrest.CoreMatchers;
@@ -31,25 +29,9 @@ import org.hamcrest.CoreMatchers;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThat;
 
-@RunWith(OrderedJUnit4ClassRunner.class)
 public class StandaloneUpgraderTest extends OfflineToolUtils
 {
     @Test
-    public void testNoArgsPrintsHelp()
-    {
-        ToolResult tool = ToolRunner.invokeClass(StandaloneUpgrader.class);
-        assertThat(tool.getStdout(), CoreMatchers.containsStringIgnoringCase("usage:"));
-        assertThat(tool.getCleanedStderr(), CoreMatchers.containsStringIgnoringCase("Missing arguments"));
-        assertEquals(1, tool.getExitCode());
-        assertNoUnexpectedThreadsStarted(null, null);
-        assertSchemaNotLoaded();
-        assertCLSMNotLoaded();
-        assertSystemKSNotLoaded();
-        assertKeyspaceNotLoaded();
-        assertServerNotLoaded();
-    }
-
-    @Test
     public void testMaybeChangeDocs()
     {
         // If you added, modified options or help, please update docs if necessary
diff --git a/test/unit/org/apache/cassandra/tools/StandaloneVerifierTest.java b/test/unit/org/apache/cassandra/tools/StandaloneVerifierTest.java
index 0494f1f..73890cd 100644
--- a/test/unit/org/apache/cassandra/tools/StandaloneVerifierTest.java
+++ b/test/unit/org/apache/cassandra/tools/StandaloneVerifierTest.java
@@ -21,9 +21,7 @@ package org.apache.cassandra.tools;
 import java.util.Arrays;
 
 import org.junit.Test;
-import org.junit.runner.RunWith;
 
-import org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.tools.ToolRunner.ToolResult;
 import org.assertj.core.api.Assertions;
 import org.hamcrest.CoreMatchers;
@@ -31,25 +29,8 @@ import org.hamcrest.CoreMatchers;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThat;
 
-@RunWith(OrderedJUnit4ClassRunner.class)
 public class StandaloneVerifierTest extends OfflineToolUtils
 {
-
-    @Test
-    public void testNoArgsPrintsHelp()
-    {
-        ToolResult tool = ToolRunner.invokeClass(StandaloneVerifier.class);
-        assertThat(tool.getStdout(), CoreMatchers.containsStringIgnoringCase("usage:"));
-        assertThat(tool.getCleanedStderr(), CoreMatchers.containsStringIgnoringCase("Missing arguments"));
-        assertEquals(1, tool.getExitCode());
-        assertNoUnexpectedThreadsStarted(null, null);
-        assertSchemaNotLoaded();
-        assertCLSMNotLoaded();
-        assertSystemKSNotLoaded();
-        assertKeyspaceNotLoaded();
-        assertServerNotLoaded();
-    }
-
     @Test
     public void testMaybeChangeDocs()
     {
diff --git a/test/unit/org/apache/cassandra/tools/ToolsSchemaLoadingTest.java b/test/unit/org/apache/cassandra/tools/ToolsSchemaLoadingTest.java
new file mode 100644
index 0000000..58a444c
--- /dev/null
+++ b/test/unit/org/apache/cassandra/tools/ToolsSchemaLoadingTest.java
@@ -0,0 +1,104 @@
+/*
+ * 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.junit.Test;
+
+import org.hamcrest.CoreMatchers;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertThat;
+
+public class ToolsSchemaLoadingTest extends OfflineToolUtils
+{
+    @Test
+    public void testNoArgsPrintsHelpStandaloneVerifier()
+    {
+        ToolRunner.ToolResult tool = ToolRunner.invokeClass(StandaloneVerifier.class);
+        assertThat(tool.getStdout(), CoreMatchers.containsStringIgnoringCase("usage:"));
+        assertThat(tool.getCleanedStderr(), CoreMatchers.containsStringIgnoringCase("Missing arguments"));
+        assertEquals(1, tool.getExitCode());
+        assertNoUnexpectedThreadsStarted(null, null);
+        assertSchemaNotLoaded();
+        assertCLSMNotLoaded();
+        assertSystemKSNotLoaded();
+        assertKeyspaceNotLoaded();
+        assertServerNotLoaded();
+    }
+
+    @Test
+    public void testNoArgsPrintsHelpStandaloneScrubber()
+    {
+        ToolRunner.ToolResult tool = ToolRunner.invokeClass(StandaloneScrubber.class);
+        assertThat(tool.getStdout(), CoreMatchers.containsStringIgnoringCase("usage:"));
+        assertThat(tool.getCleanedStderr(), CoreMatchers.containsStringIgnoringCase("Missing arguments"));
+        assertEquals(1, tool.getExitCode());
+        assertNoUnexpectedThreadsStarted(null, null);
+        assertSchemaNotLoaded();
+        assertCLSMNotLoaded();
+        assertSystemKSNotLoaded();
+        assertKeyspaceNotLoaded();
+        assertServerNotLoaded();
+    }
+
+    @Test
+    public void testNoArgsPrintsHelpStandaloneSplitter()
+    {
+        ToolRunner.ToolResult tool = ToolRunner.invokeClass(StandaloneSplitter.class);
+        assertThat(tool.getStdout(), CoreMatchers.containsStringIgnoringCase("usage:"));
+        assertThat(tool.getCleanedStderr(), CoreMatchers.containsStringIgnoringCase("No sstables to split"));
+        assertEquals(1, tool.getExitCode());
+        assertNoUnexpectedThreadsStarted(null, null);
+        assertSchemaNotLoaded();
+        assertCLSMNotLoaded();
+        assertSystemKSNotLoaded();
+        assertKeyspaceNotLoaded();
+        assertServerNotLoaded();
+    }
+
+    @Test
+    public void testNoArgsPrintsHelpStandaloneSSTableUtil()
+    {
+        ToolRunner.ToolResult tool = ToolRunner.invokeClass(StandaloneSSTableUtil.class);
+        assertThat(tool.getStdout(), CoreMatchers.containsStringIgnoringCase("usage:"));
+        assertThat(tool.getCleanedStderr(), CoreMatchers.containsStringIgnoringCase("Missing arguments"));
+        assertEquals(1, tool.getExitCode());
+        assertNoUnexpectedThreadsStarted(null, null);
+        assertSchemaNotLoaded();
+        assertCLSMNotLoaded();
+        assertSystemKSNotLoaded();
+        assertKeyspaceNotLoaded();
+        assertServerNotLoaded();
+    }
+
+    @Test
+    public void testNoArgsPrintsHelpStandaloneUpgrader()
+    {
+        ToolRunner.ToolResult tool = ToolRunner.invokeClass(StandaloneUpgrader.class);
+        assertThat(tool.getStdout(), CoreMatchers.containsStringIgnoringCase("usage:"));
+        assertThat(tool.getCleanedStderr(), CoreMatchers.containsStringIgnoringCase("Missing arguments"));
+        assertEquals(1, tool.getExitCode());
+        assertNoUnexpectedThreadsStarted(null, null);
+        assertSchemaNotLoaded();
+        assertCLSMNotLoaded();
+        assertSystemKSNotLoaded();
+        assertKeyspaceNotLoaded();
+        assertServerNotLoaded();
+    }
+}
diff --git a/test/unit/org/apache/cassandra/tools/nodetool/RingTest.java b/test/unit/org/apache/cassandra/tools/nodetool/RingTest.java
index 83771f6..bc83f50 100644
--- a/test/unit/org/apache/cassandra/tools/nodetool/RingTest.java
+++ b/test/unit/org/apache/cassandra/tools/nodetool/RingTest.java
@@ -22,9 +22,7 @@ import java.util.Arrays;
 
 import org.junit.BeforeClass;
 import org.junit.Test;
-import org.junit.runner.RunWith;
 
-import org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.cql3.CQLTester;
 import org.apache.cassandra.locator.SimpleSnitch;
 import org.apache.cassandra.service.StorageService;
@@ -38,7 +36,6 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 
-@RunWith(OrderedJUnit4ClassRunner.class)
 public class RingTest extends CQLTester
 {
     private static String token;
diff --git a/test/unit/org/apache/cassandra/tools/nodetool/StatusTest.java b/test/unit/org/apache/cassandra/tools/nodetool/StatusTest.java
index e3be5d9..13acb19 100644
--- a/test/unit/org/apache/cassandra/tools/nodetool/StatusTest.java
+++ b/test/unit/org/apache/cassandra/tools/nodetool/StatusTest.java
@@ -22,9 +22,7 @@ import java.util.regex.Pattern;
 
 import org.junit.BeforeClass;
 import org.junit.Test;
-import org.junit.runner.RunWith;
 
-import org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.cql3.CQLTester;
 import org.apache.cassandra.locator.SimpleSnitch;
 import org.apache.cassandra.schema.SchemaConstants;
@@ -36,7 +34,6 @@ import static org.hamcrest.CoreMatchers.*;
 import static org.hamcrest.Matchers.matchesPattern;
 import static org.junit.Assert.assertThat;
 
-@RunWith(OrderedJUnit4ClassRunner.class)
 public class StatusTest extends CQLTester
 {
     private static final Pattern PATTERN = Pattern.compile("\\R");
diff --git a/test/unit/org/apache/cassandra/tools/nodetool/stats/NodetoolTableStatsTest.java b/test/unit/org/apache/cassandra/tools/nodetool/stats/NodetoolTableStatsTest.java
index a459f0f..6891cad 100644
--- a/test/unit/org/apache/cassandra/tools/nodetool/stats/NodetoolTableStatsTest.java
+++ b/test/unit/org/apache/cassandra/tools/nodetool/stats/NodetoolTableStatsTest.java
@@ -28,9 +28,7 @@ import org.apache.commons.lang3.StringUtils;
 
 import org.junit.BeforeClass;
 import org.junit.Test;
-import org.junit.runner.RunWith;
 
-import org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.cql3.CQLTester;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.tools.ToolRunner;
@@ -43,7 +41,6 @@ import static org.junit.Assert.assertNotEquals;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 
-@RunWith(OrderedJUnit4ClassRunner.class)
 public class NodetoolTableStatsTest extends CQLTester
 {
     @BeforeClass
diff --git a/test/unit/org/apache/cassandra/transport/CQLUserAuditTest.java b/test/unit/org/apache/cassandra/transport/CQLUserAuditTest.java
index 32717bf..7f3c9cb 100644
--- a/test/unit/org/apache/cassandra/transport/CQLUserAuditTest.java
+++ b/test/unit/org/apache/cassandra/transport/CQLUserAuditTest.java
@@ -34,13 +34,11 @@ import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
-import org.junit.runner.RunWith;
 
 import com.datastax.driver.core.Cluster;
 import com.datastax.driver.core.PreparedStatement;
 import com.datastax.driver.core.Session;
 import com.datastax.driver.core.exceptions.AuthenticationException;
-import org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.audit.AuditEvent;
 import org.apache.cassandra.audit.AuditLogEntryType;
 import org.apache.cassandra.audit.AuditLogManager;
@@ -56,7 +54,6 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
 
-@RunWith(OrderedJUnit4ClassRunner.class)
 public class CQLUserAuditTest
 {
     private static EmbeddedCassandraService embedded;

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