You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by ty...@apache.org on 2014/06/20 19:56:45 UTC

[1/6] Reduce unit test times due to schema loading

Repository: cassandra
Updated Branches:
  refs/heads/trunk 51016876a -> d2a3827a6


http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java b/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
index 47136a0..52b2236 100644
--- a/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
+++ b/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
@@ -35,17 +35,23 @@ import org.slf4j.LoggerFactory;
 import org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
+import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.columniterator.IdentityQueryFilter;
 import org.apache.cassandra.db.context.CounterContext;
 import org.apache.cassandra.db.filter.IDiskAtomFilter;
 import org.apache.cassandra.db.filter.QueryFilter;
+import org.apache.cassandra.db.marshal.BytesType;
+import org.apache.cassandra.db.marshal.CounterColumnType;
+import org.apache.cassandra.db.marshal.IntegerType;
 import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.io.sstable.SSTableReader;
 import org.apache.cassandra.io.sstable.SSTableUtils;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.CounterId;
@@ -57,16 +63,42 @@ import static org.apache.cassandra.Util.cellname;
 import static org.apache.cassandra.Util.column;
 
 @RunWith(OrderedJUnit4ClassRunner.class)
-public class StreamingTransferTest extends SchemaLoader
+public class StreamingTransferTest
 {
     private static final Logger logger = LoggerFactory.getLogger(StreamingTransferTest.class);
 
     public static final InetAddress LOCAL = FBUtilities.getBroadcastAddress();
+    public static final String KEYSPACE1 = "StreamingTransferTest1";
+    public static final String CF_STANDARD = "Standard1";
+    public static final String CF_COUNTER = "Counter1";
+    public static final String CF_STANDARDINT = "StandardInteger1";
+    public static final String CF_INDEX = "Indexed1";
+    public static final String KEYSPACE_CACHEKEY = "KeyStreamingTransferTestSpace";
+    public static final String CF_STANDARD2 = "Standard2";
+    public static final String CF_STANDARD3 = "Standard3";
+    public static final String KEYSPACE2 = "StreamingTransferTest2";
 
     @BeforeClass
-    public static void setup() throws Exception
+    public static void defineSchema() throws Exception
     {
+        SchemaLoader.prepareServer();
         StorageService.instance.initServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD),
+                                    CFMetaData.denseCFMetaData(KEYSPACE1, CF_COUNTER, BytesType.instance).defaultValidator(CounterColumnType.instance),
+                                    CFMetaData.denseCFMetaData(KEYSPACE1, CF_STANDARDINT, IntegerType.instance),
+                                    SchemaLoader.indexCFMD(KEYSPACE1, CF_INDEX, true));
+        SchemaLoader.createKeyspace(KEYSPACE2,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1));
+        SchemaLoader.createKeyspace(KEYSPACE_CACHEKEY,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE_CACHEKEY, CF_STANDARD),
+                                    SchemaLoader.standardCFMD(KEYSPACE_CACHEKEY, CF_STANDARD2),
+                                    SchemaLoader.standardCFMD(KEYSPACE_CACHEKEY, CF_STANDARD3));
     }
 
     /**
@@ -105,7 +137,7 @@ public class StreamingTransferTest extends SchemaLoader
         ranges.add(new Range<>(p.getToken(ByteBufferUtil.bytes("key2")), p.getMinimumToken()));
 
         StreamResultFuture futureResult = new StreamPlan("StreamingTransferTest")
-                                                  .requestRanges(LOCAL, "Keyspace2", ranges)
+                                                  .requestRanges(LOCAL, KEYSPACE2, ranges)
                                                   .execute();
 
         UUID planId = futureResult.planId;
@@ -219,7 +251,7 @@ public class StreamingTransferTest extends SchemaLoader
 
     private void doTransferTable(boolean transferSSTables) throws Exception
     {
-        final Keyspace keyspace = Keyspace.open("Keyspace1");
+        final Keyspace keyspace = Keyspace.open(KEYSPACE1);
         final ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("Indexed1");
 
         List<String> keys = createAndTransfer(cfs, new Mutator()
@@ -230,7 +262,7 @@ public class StreamingTransferTest extends SchemaLoader
                 ColumnFamily cf = ArrayBackedSortedColumns.factory.create(keyspace.getName(), cfs.name);
                 cf.addColumn(column(col, "v", timestamp));
                 cf.addColumn(new BufferCell(cellname("birthdate"), ByteBufferUtil.bytes(val), timestamp));
-                Mutation rm = new Mutation("Keyspace1", ByteBufferUtil.bytes(key), cf);
+                Mutation rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes(key), cf);
                 logger.debug("Applying row to transfer {}", rm);
                 rm.apply();
             }
@@ -258,7 +290,7 @@ public class StreamingTransferTest extends SchemaLoader
     @Test
     public void testTransferRangeTombstones() throws Exception
     {
-        String ks = "Keyspace1";
+        String ks = KEYSPACE1;
         String cfname = "StandardInteger1";
         Keyspace keyspace = Keyspace.open(ks);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfname);
@@ -301,7 +333,7 @@ public class StreamingTransferTest extends SchemaLoader
     @Test
     public void testTransferTableCounter() throws Exception
     {
-        final Keyspace keyspace = Keyspace.open("Keyspace1");
+        final Keyspace keyspace = Keyspace.open(KEYSPACE1);
         final ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("Counter1");
         final CounterContext cc = new CounterContext();
 
@@ -358,7 +390,7 @@ public class StreamingTransferTest extends SchemaLoader
         content.add("test");
         content.add("test2");
         content.add("test3");
-        SSTableReader sstable = SSTableUtils.prepare().write(content);
+        SSTableReader sstable = new SSTableUtils(KEYSPACE1, CF_STANDARD).prepare().write(content);
         String keyspaceName = sstable.getKeyspaceName();
         String cfname = sstable.getColumnFamilyName();
 
@@ -397,7 +429,7 @@ public class StreamingTransferTest extends SchemaLoader
     @Test
     public void testTransferOfMultipleColumnFamilies() throws Exception
     {
-        String keyspace = "KeyCacheSpace";
+        String keyspace = KEYSPACE_CACHEKEY;
         IPartitioner p = StorageService.getPartitioner();
         String[] columnFamilies = new String[] { "Standard1", "Standard2", "Standard3" };
         List<SSTableReader> ssTableReaders = new ArrayList<>();
@@ -445,7 +477,7 @@ public class StreamingTransferTest extends SchemaLoader
     @Test
     public void testRandomSSTableTransfer() throws Exception
     {
-        final Keyspace keyspace = Keyspace.open("Keyspace1");
+        final Keyspace keyspace = Keyspace.open(KEYSPACE1);
         final ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("Standard1");
         Mutator mutator = new Mutator()
         {
@@ -454,7 +486,7 @@ public class StreamingTransferTest extends SchemaLoader
                 ColumnFamily cf = ArrayBackedSortedColumns.factory.create(keyspace.getName(), cfs.name);
                 cf.addColumn(column(colName, "value", timestamp));
                 cf.addColumn(new BufferCell(cellname("birthdate"), ByteBufferUtil.bytes(new Date(timestamp).toString()), timestamp));
-                Mutation rm = new Mutation("Keyspace1", ByteBufferUtil.bytes(key), cf);
+                Mutation rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes(key), cf);
                 logger.debug("Applying row to transfer {}", rm);
                 rm.apply();
             }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/thrift/MultiSliceTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/thrift/MultiSliceTest.java b/test/unit/org/apache/cassandra/thrift/MultiSliceTest.java
index 50e409e..9716876 100644
--- a/test/unit/org/apache/cassandra/thrift/MultiSliceTest.java
+++ b/test/unit/org/apache/cassandra/thrift/MultiSliceTest.java
@@ -28,25 +28,33 @@ import java.util.List;
 import junit.framework.Assert;
 
 import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.config.KSMetaData;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.service.EmbeddedCassandraService;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.thrift.TException;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
-public class MultiSliceTest extends SchemaLoader
+public class MultiSliceTest
 {
     private static CassandraServer server;
-    
+    public static final String KEYSPACE1 = "MultiSliceTest";
+    public static final String CF_STANDARD = "Standard1";
+
     @BeforeClass
-    public static void setup() throws IOException, TException 
+    public static void defineSchema() throws ConfigurationException, IOException, TException
     {
-        Schema.instance.clear(); // Schema are now written on disk and will be reloaded
+        SchemaLoader.prepareServer();
         new EmbeddedCassandraService().start();
-        ThriftSessionManager.instance.setCurrentSocket(new InetSocketAddress(9160));        
+        ThriftSessionManager.instance.setCurrentSocket(new InetSocketAddress(9160));
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD));
         server = new CassandraServer();
-        server.set_keyspace("Keyspace1");
+        server.set_keyspace(KEYSPACE1);
     }
 
     private static MultiSliceRequest makeMultiSliceRequest(ByteBuffer key)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/thrift/ThriftValidationTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/thrift/ThriftValidationTest.java b/test/unit/org/apache/cassandra/thrift/ThriftValidationTest.java
index df0f98c..e81dd3d 100644
--- a/test/unit/org/apache/cassandra/thrift/ThriftValidationTest.java
+++ b/test/unit/org/apache/cassandra/thrift/ThriftValidationTest.java
@@ -20,40 +20,62 @@ package org.apache.cassandra.thrift;
  *
  */
 
-import org.apache.cassandra.db.marshal.LongType;
-import org.apache.cassandra.exceptions.*;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.config.*;
-import org.apache.cassandra.db.marshal.AsciiType;
+import org.apache.cassandra.db.marshal.*;
+import org.apache.cassandra.exceptions.*;
 import org.apache.cassandra.locator.LocalStrategy;
 import org.apache.cassandra.locator.NetworkTopologyStrategy;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.thrift.TException;
 
+import java.io.IOException;
 import java.util.Arrays;
 
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertEquals;
 
-public class ThriftValidationTest extends SchemaLoader
+public class ThriftValidationTest
 {
+    public static final String KEYSPACE1 = "MultiSliceTest";
+    public static final String CF_STANDARD = "Standard1";
+    public static final String CF_COUNTER = "Counter1";
+    public static final String CF_UUID = "UUIDKeys";
+    public static final String CF_STANDARDLONG3 = "StandardLong3";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException, IOException, TException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD),
+                                    CFMetaData.denseCFMetaData(KEYSPACE1, CF_COUNTER, BytesType.instance).defaultValidator(CounterColumnType.instance),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_UUID).keyValidator(UUIDType.instance),
+                                    CFMetaData.denseCFMetaData(KEYSPACE1, CF_STANDARDLONG3, IntegerType.instance));
+    }
+    
     @Test(expected=org.apache.cassandra.exceptions.InvalidRequestException.class)
     public void testValidateCommutativeWithStandard() throws org.apache.cassandra.exceptions.InvalidRequestException
     {
-        ThriftValidation.validateColumnFamily("Keyspace1", "Standard1", true);
+        ThriftValidation.validateColumnFamily(KEYSPACE1, "Standard1", true);
     }
 
     @Test
     public void testValidateCommutativeWithCounter() throws org.apache.cassandra.exceptions.InvalidRequestException
     {
-        ThriftValidation.validateColumnFamily("Keyspace1", "Counter1", true);
+        ThriftValidation.validateColumnFamily(KEYSPACE1, "Counter1", true);
     }
 
     @Test
     public void testColumnNameEqualToKeyAlias() throws org.apache.cassandra.exceptions.InvalidRequestException
     {
-        CFMetaData metaData = Schema.instance.getCFMetaData("Keyspace1", "Standard1");
+        CFMetaData metaData = Schema.instance.getCFMetaData(KEYSPACE1, "Standard1");
         CFMetaData newMetadata = metaData.copy();
 
         boolean gotException = false;
@@ -98,7 +120,7 @@ public class ThriftValidationTest extends SchemaLoader
     @Test
     public void testColumnNameEqualToDefaultKeyAlias() throws org.apache.cassandra.exceptions.InvalidRequestException
     {
-        CFMetaData metaData = Schema.instance.getCFMetaData("Keyspace1", "UUIDKeys");
+        CFMetaData metaData = Schema.instance.getCFMetaData(KEYSPACE1, "UUIDKeys");
         ColumnDefinition definition = metaData.getColumnDefinition(ByteBufferUtil.bytes(CFMetaData.DEFAULT_KEY_ALIAS));
         assertNotNull(definition);
         assertEquals(ColumnDefinition.Kind.PARTITION_KEY, definition.kind);
@@ -116,7 +138,7 @@ public class ThriftValidationTest extends SchemaLoader
     @Test
     public void testColumnNameEqualToDefaultColumnAlias() throws org.apache.cassandra.exceptions.InvalidRequestException
     {
-        CFMetaData metaData = Schema.instance.getCFMetaData("Keyspace1", "StandardLong3");
+        CFMetaData metaData = Schema.instance.getCFMetaData(KEYSPACE1, "StandardLong3");
         ColumnDefinition definition = metaData.getColumnDefinition(ByteBufferUtil.bytes(CFMetaData.DEFAULT_COLUMN_ALIAS + 1));
         assertNotNull(definition);
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/tools/SSTableExportTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/tools/SSTableExportTest.java b/test/unit/org/apache/cassandra/tools/SSTableExportTest.java
index d9133f6..dbc6703 100644
--- a/test/unit/org/apache/cassandra/tools/SSTableExportTest.java
+++ b/test/unit/org/apache/cassandra/tools/SSTableExportTest.java
@@ -34,23 +34,50 @@ import java.io.PrintStream;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
 import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.filter.QueryFilter;
+import org.apache.cassandra.db.marshal.BytesType;
+import org.apache.cassandra.db.marshal.CounterColumnType;
 import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.db.marshal.UUIDType;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.sstable.Descriptor;
 import org.apache.cassandra.io.sstable.SSTableReader;
 import org.apache.cassandra.io.sstable.SSTableWriter;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.service.ActiveRepairService;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.UUIDGen;
+import org.apache.thrift.TException;
 import org.json.simple.JSONArray;
 import org.json.simple.JSONObject;
 import org.json.simple.JSONValue;
 import org.json.simple.parser.ParseException;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
-public class SSTableExportTest extends SchemaLoader
+public class SSTableExportTest
 {
+    public static final String KEYSPACE1 = "SSTableExportTest";
+    public static final String CF_STANDARD = "Standard1";
+    public static final String CF_COUNTER = "Counter1";
+    public static final String CF_UUID = "UUIDKeys";
+    public static final String CF_VALSWITHQUOTES = "ValuesWithQuotes";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException, IOException, TException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD),
+                                    CFMetaData.denseCFMetaData(KEYSPACE1, CF_COUNTER, BytesType.instance).defaultValidator(CounterColumnType.instance),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_UUID).keyValidator(UUIDType.instance),
+                                    CFMetaData.denseCFMetaData(KEYSPACE1, CF_VALSWITHQUOTES, BytesType.instance).defaultValidator(UTF8Type.instance));
+    }
+
     public String asHex(String str)
     {
         return bytesToHex(ByteBufferUtil.bytes(str));
@@ -59,8 +86,8 @@ public class SSTableExportTest extends SchemaLoader
     @Test
     public void testEnumeratekeys() throws IOException
     {
-        File tempSS = tempSSTableFile("Keyspace1", "Standard1");
-        ColumnFamily cfamily = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        File tempSS = tempSSTableFile(KEYSPACE1, "Standard1");
+        ColumnFamily cfamily = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Standard1");
         SSTableWriter writer = new SSTableWriter(tempSS.getPath(), 2, ActiveRepairService.UNREPAIRED_SSTABLE);
 
         // Add rowA
@@ -94,8 +121,8 @@ public class SSTableExportTest extends SchemaLoader
     @Test
     public void testExportSimpleCf() throws IOException, ParseException
     {
-        File tempSS = tempSSTableFile("Keyspace1", "Standard1");
-        ColumnFamily cfamily = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        File tempSS = tempSSTableFile(KEYSPACE1, "Standard1");
+        ColumnFamily cfamily = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Standard1");
         SSTableWriter writer = new SSTableWriter(tempSS.getPath(), 2, ActiveRepairService.UNREPAIRED_SSTABLE);
 
         int nowInSec = (int)(System.currentTimeMillis() / 1000) + 42; //live for 42 seconds
@@ -149,9 +176,9 @@ public class SSTableExportTest extends SchemaLoader
     @Test
     public void testRoundTripStandardCf() throws IOException
     {
-        ColumnFamilyStore cfs = Keyspace.open("Keyspace1").getColumnFamilyStore("Standard1");
-        File tempSS = tempSSTableFile("Keyspace1", "Standard1");
-        ColumnFamily cfamily = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore("Standard1");
+        File tempSS = tempSSTableFile(KEYSPACE1, "Standard1");
+        ColumnFamily cfamily = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Standard1");
         SSTableWriter writer = new SSTableWriter(tempSS.getPath(), 2, ActiveRepairService.UNREPAIRED_SSTABLE);
 
         // Add rowA
@@ -171,8 +198,8 @@ public class SSTableExportTest extends SchemaLoader
         SSTableExport.export(reader, new PrintStream(tempJson.getPath()), new String[]{asHex("rowExclude")});
 
         // Import JSON to another SSTable file
-        File tempSS2 = tempSSTableFile("Keyspace1", "Standard1");
-        new SSTableImport().importJson(tempJson.getPath(), "Keyspace1", "Standard1", tempSS2.getPath());
+        File tempSS2 = tempSSTableFile(KEYSPACE1, "Standard1");
+        new SSTableImport().importJson(tempJson.getPath(), KEYSPACE1, "Standard1", tempSS2.getPath());
 
         reader = SSTableReader.open(Descriptor.fromFilename(tempSS2.getPath()));
         QueryFilter qf = Util.namesQueryFilter(cfs, Util.dk("rowA"), "name");
@@ -189,8 +216,8 @@ public class SSTableExportTest extends SchemaLoader
     @Test
     public void testExportCounterCf() throws IOException, ParseException
     {
-        File tempSS = tempSSTableFile("Keyspace1", "Counter1");
-        ColumnFamily cfamily = ArrayBackedSortedColumns.factory.create("Keyspace1", "Counter1");
+        File tempSS = tempSSTableFile(KEYSPACE1, "Counter1");
+        ColumnFamily cfamily = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Counter1");
         SSTableWriter writer = new SSTableWriter(tempSS.getPath(), 2, ActiveRepairService.UNREPAIRED_SSTABLE);
 
         // Add rowA
@@ -220,8 +247,8 @@ public class SSTableExportTest extends SchemaLoader
     @Test
     public void testEscapingDoubleQuotes() throws IOException, ParseException
     {
-        File tempSS = tempSSTableFile("Keyspace1", "ValuesWithQuotes");
-        ColumnFamily cfamily = ArrayBackedSortedColumns.factory.create("Keyspace1", "ValuesWithQuotes");
+        File tempSS = tempSSTableFile(KEYSPACE1, "ValuesWithQuotes");
+        ColumnFamily cfamily = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "ValuesWithQuotes");
         SSTableWriter writer = new SSTableWriter(tempSS.getPath(), 2, ActiveRepairService.UNREPAIRED_SSTABLE);
 
         // Add rowA
@@ -251,8 +278,8 @@ public class SSTableExportTest extends SchemaLoader
     @Test
     public void testExportColumnsWithMetadata() throws IOException, ParseException
     {
-        File tempSS = tempSSTableFile("Keyspace1", "Standard1");
-        ColumnFamily cfamily = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        File tempSS = tempSSTableFile(KEYSPACE1, "Standard1");
+        ColumnFamily cfamily = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Standard1");
         SSTableWriter writer = new SSTableWriter(tempSS.getPath(), 2, ActiveRepairService.UNREPAIRED_SSTABLE);
 
         // Add rowA
@@ -312,8 +339,8 @@ public class SSTableExportTest extends SchemaLoader
     @Test
     public void testColumnNameEqualToDefaultKeyAlias() throws IOException, ParseException
     {
-        File tempSS = tempSSTableFile("Keyspace1", "UUIDKeys");
-        ColumnFamily cfamily = ArrayBackedSortedColumns.factory.create("Keyspace1", "UUIDKeys");
+        File tempSS = tempSSTableFile(KEYSPACE1, "UUIDKeys");
+        ColumnFamily cfamily = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "UUIDKeys");
         SSTableWriter writer = new SSTableWriter(tempSS.getPath(), 2, ActiveRepairService.UNREPAIRED_SSTABLE);
 
         // Add a row

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/tools/SSTableImportTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/tools/SSTableImportTest.java b/test/unit/org/apache/cassandra/tools/SSTableImportTest.java
index 2fdeaf4..dba8408 100644
--- a/test/unit/org/apache/cassandra/tools/SSTableImportTest.java
+++ b/test/unit/org/apache/cassandra/tools/SSTableImportTest.java
@@ -27,26 +27,48 @@ import java.io.IOException;
 import java.net.URI;
 import java.net.URISyntaxException;
 
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.columniterator.OnDiskAtomIterator;
 import org.apache.cassandra.db.filter.QueryFilter;
+import org.apache.cassandra.db.marshal.BytesType;
+import org.apache.cassandra.db.marshal.CounterColumnType;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.io.sstable.Descriptor;
 import org.apache.cassandra.io.sstable.SSTableReader;
-import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.thrift.TException;
 
-public class SSTableImportTest extends SchemaLoader
+public class SSTableImportTest
 {
+    public static final String KEYSPACE1 = "SSTableImportTest";
+    public static final String CF_STANDARD = "Standard1";
+    public static final String CF_COUNTER = "Counter1";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException, IOException, TException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD),
+                                    CFMetaData.denseCFMetaData(KEYSPACE1, CF_COUNTER, BytesType.instance).defaultValidator(CounterColumnType.instance));
+    }
+
     @Test
     public void testImportSimpleCf() throws IOException, URISyntaxException
     {
         // Import JSON to temp SSTable file
         String jsonUrl = resourcePath("SimpleCF.json");
-        File tempSS = tempSSTableFile("Keyspace1", "Standard1");
-        new SSTableImport(true).importJson(jsonUrl, "Keyspace1", "Standard1", tempSS.getPath());
+        File tempSS = tempSSTableFile(KEYSPACE1, "Standard1");
+        new SSTableImport(true).importJson(jsonUrl, KEYSPACE1, "Standard1", tempSS.getPath());
 
         // Verify results
         SSTableReader reader = SSTableReader.open(Descriptor.fromFilename(tempSS.getPath()));
@@ -78,9 +100,9 @@ public class SSTableImportTest extends SchemaLoader
     public void testImportUnsortedMode() throws IOException, URISyntaxException
     {
         String jsonUrl = resourcePath("UnsortedCF.json");
-        File tempSS = tempSSTableFile("Keyspace1", "Standard1");
+        File tempSS = tempSSTableFile(KEYSPACE1, "Standard1");
 
-        new SSTableImport().importJson(jsonUrl, "Keyspace1", "Standard1", tempSS.getPath());
+        new SSTableImport().importJson(jsonUrl, KEYSPACE1, "Standard1", tempSS.getPath());
 
         SSTableReader reader = SSTableReader.open(Descriptor.fromFilename(tempSS.getPath()));
         QueryFilter qf = QueryFilter.getIdentityFilter(Util.dk("rowA"), "Standard1", System.currentTimeMillis());
@@ -101,8 +123,8 @@ public class SSTableImportTest extends SchemaLoader
     {
         // Import JSON to temp SSTable file
         String jsonUrl = resourcePath("SimpleCFWithDeletionInfo.json");
-        File tempSS = tempSSTableFile("Keyspace1", "Standard1");
-        new SSTableImport(true).importJson(jsonUrl, "Keyspace1", "Standard1", tempSS.getPath());
+        File tempSS = tempSSTableFile(KEYSPACE1, "Standard1");
+        new SSTableImport(true).importJson(jsonUrl, KEYSPACE1, "Standard1", tempSS.getPath());
 
         // Verify results
         SSTableReader reader = SSTableReader.open(Descriptor.fromFilename(tempSS.getPath()));
@@ -125,8 +147,8 @@ public class SSTableImportTest extends SchemaLoader
     {
         // Import JSON to temp SSTable file
         String jsonUrl = resourcePath("CounterCF.json");
-        File tempSS = tempSSTableFile("Keyspace1", "Counter1");
-        new SSTableImport(true).importJson(jsonUrl, "Keyspace1", "Counter1", tempSS.getPath());
+        File tempSS = tempSSTableFile(KEYSPACE1, "Counter1");
+        new SSTableImport(true).importJson(jsonUrl, KEYSPACE1, "Counter1", tempSS.getPath());
 
         // Verify results
         SSTableReader reader = SSTableReader.open(Descriptor.fromFilename(tempSS.getPath()));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/triggers/TriggersSchemaTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/triggers/TriggersSchemaTest.java b/test/unit/org/apache/cassandra/triggers/TriggersSchemaTest.java
index c6a1ac5..577e7d3 100644
--- a/test/unit/org/apache/cassandra/triggers/TriggersSchemaTest.java
+++ b/test/unit/org/apache/cassandra/triggers/TriggersSchemaTest.java
@@ -19,6 +19,7 @@ package org.apache.cassandra.triggers;
 
 import java.util.Collections;
 
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
@@ -26,6 +27,7 @@ import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.config.TriggerDefinition;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.service.MigrationManager;
 
@@ -33,13 +35,19 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
-public class TriggersSchemaTest extends SchemaLoader
+public class TriggersSchemaTest
 {
     String ksName = "ks" + System.nanoTime();
     String cfName = "cf" + System.nanoTime();
     String triggerName = "trigger_" + System.nanoTime();
     String triggerClass = "org.apache.cassandra.triggers.NoSuchTrigger.class";
 
+    @BeforeClass
+    public static void beforeTest() throws ConfigurationException
+    {
+        SchemaLoader.loadSchema();
+    }
+
     @Test
     public void newKsContainsCfWithTrigger() throws Exception
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/triggers/TriggersTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/triggers/TriggersTest.java b/test/unit/org/apache/cassandra/triggers/TriggersTest.java
index 74fde69..41d4bb8 100644
--- a/test/unit/org/apache/cassandra/triggers/TriggersTest.java
+++ b/test/unit/org/apache/cassandra/triggers/TriggersTest.java
@@ -24,6 +24,7 @@ import java.util.Collections;
 
 import org.junit.AfterClass;
 import org.junit.Before;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
@@ -35,6 +36,7 @@ import org.apache.cassandra.db.BufferCell;
 import org.apache.cassandra.db.ColumnFamily;
 import org.apache.cassandra.db.ConsistencyLevel;
 import org.apache.cassandra.db.Mutation;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.RequestExecutionException;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.thrift.*;
@@ -46,7 +48,7 @@ import static org.junit.Assert.assertTrue;
 import static org.apache.cassandra.utils.ByteBufferUtil.bytes;
 import static org.apache.cassandra.utils.ByteBufferUtil.toInt;
 
-public class TriggersTest extends SchemaLoader
+public class TriggersTest
 {
     private static boolean triggerCreated = false;
     private static ThriftServer thriftServer;
@@ -55,6 +57,12 @@ public class TriggersTest extends SchemaLoader
     private static String cfName = "test_table";
     private static String otherCf = "other_table";
 
+    @BeforeClass
+    public static void beforeTest() throws ConfigurationException
+    {
+        SchemaLoader.loadSchema();
+    }
+
     @Before
     public void setup() throws Exception
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/utils/EncodedStreamsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/utils/EncodedStreamsTest.java b/test/unit/org/apache/cassandra/utils/EncodedStreamsTest.java
index 2a67d42..957979f 100644
--- a/test/unit/org/apache/cassandra/utils/EncodedStreamsTest.java
+++ b/test/unit/org/apache/cassandra/utils/EncodedStreamsTest.java
@@ -26,25 +26,42 @@ import java.io.DataOutputStream;
 import java.io.IOException;
 
 import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.db.ArrayBackedSortedColumns;
 import org.apache.cassandra.db.ColumnFamily;
 import org.apache.cassandra.db.TypeSizes;
+import org.apache.cassandra.db.marshal.BytesType;
+import org.apache.cassandra.db.marshal.CounterColumnType;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.utils.vint.EncodedDataInputStream;
 import org.apache.cassandra.utils.vint.EncodedDataOutputStream;
 
 import org.junit.Assert;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
-public class EncodedStreamsTest extends SchemaLoader
+public class EncodedStreamsTest
 {
-    private String keyspaceName = "Keyspace1";
-    private String standardCFName = "Standard1";
-    private String counterCFName = "Counter1";
-    private String superCFName = "Super1";
-
+    private static final String KEYSPACE1 = "Keyspace1";
+    private static final String CF_STANDARD = "Standard1";
+    private static final String CF_COUNTER = "Counter1";
     private int version = MessagingService.current_version;
 
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+    SchemaLoader.prepareServer();
+    SchemaLoader.createKeyspace(KEYSPACE1,
+                                SimpleStrategy.class,
+                                KSMetaData.optsWithRF(1),
+                                SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD),
+                                CFMetaData.denseCFMetaData(KEYSPACE1, CF_COUNTER, BytesType.instance)
+                                          .defaultValidator(CounterColumnType.instance));
+    }
+
     @Test
     public void testStreams() throws IOException
     {
@@ -97,7 +114,7 @@ public class EncodedStreamsTest extends SchemaLoader
 
     private ColumnFamily createCF()
     {
-        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(keyspaceName, standardCFName);
+        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(KEYSPACE1, CF_STANDARD);
         cf.addColumn(column("vijay", "try", 1));
         cf.addColumn(column("to", "be_nice", 1));
         return cf;
@@ -105,7 +122,7 @@ public class EncodedStreamsTest extends SchemaLoader
 
     private ColumnFamily createCounterCF()
     {
-        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(keyspaceName, counterCFName);
+        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(KEYSPACE1, CF_COUNTER);
         cf.addCounter(cellname("vijay"), 1);
         cf.addCounter(cellname("wants"), 1000000);
         return cf;


[3/6] Reduce unit test times due to schema loading

Posted by ty...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/db/TimeSortTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/TimeSortTest.java b/test/unit/org/apache/cassandra/db/TimeSortTest.java
index 80c0ff1..77890fb 100644
--- a/test/unit/org/apache/cassandra/db/TimeSortTest.java
+++ b/test/unit/org/apache/cassandra/db/TimeSortTest.java
@@ -21,6 +21,7 @@ package org.apache.cassandra.db;
 import java.io.IOException;
 import java.util.*;
 
+import org.junit.BeforeClass;
 import org.junit.Test;
 import static org.junit.Assert.assertEquals;
 
@@ -29,28 +30,44 @@ import static org.apache.cassandra.Util.cellname;
 import static org.apache.cassandra.Util.getBytes;
 import org.apache.cassandra.Util;
 
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.db.composites.*;
 import org.apache.cassandra.db.filter.QueryFilter;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 
-public class TimeSortTest extends SchemaLoader
+public class TimeSortTest
 {
+    private static final String KEYSPACE1 = "TimeSortTest";
+    private static final String CF_STANDARD1 = "StandardLong1";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1));
+    }
+
     @Test
     public void testMixedSources()
     {
-        Keyspace keyspace = Keyspace.open("Keyspace1");
-        ColumnFamilyStore cfStore = keyspace.getColumnFamilyStore("StandardLong1");
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
+        ColumnFamilyStore cfStore = keyspace.getColumnFamilyStore(CF_STANDARD1);
         Mutation rm;
         DecoratedKey key = Util.dk("key0");
 
-        rm = new Mutation("Keyspace1", key.getKey());
-        rm.add("StandardLong1", cellname(100), ByteBufferUtil.bytes("a"), 100);
+        rm = new Mutation(KEYSPACE1, key.getKey());
+        rm.add(CF_STANDARD1, cellname(100), ByteBufferUtil.bytes("a"), 100);
         rm.apply();
         cfStore.forceBlockingFlush();
 
-        rm = new Mutation("Keyspace1", key.getKey());
-        rm.add("StandardLong1", cellname(0), ByteBufferUtil.bytes("b"), 0);
+        rm = new Mutation(KEYSPACE1, key.getKey());
+        rm.add(CF_STANDARD1, cellname(0), ByteBufferUtil.bytes("b"), 0);
         rm.apply();
 
         ColumnFamily cf = cfStore.getColumnFamily(key, cellname(10), Composites.EMPTY, false, 1000, System.currentTimeMillis());
@@ -61,15 +78,15 @@ public class TimeSortTest extends SchemaLoader
     @Test
     public void testTimeSort() throws IOException
     {
-        Keyspace keyspace = Keyspace.open("Keyspace1");
-        ColumnFamilyStore cfStore = keyspace.getColumnFamilyStore("StandardLong1");
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
+        ColumnFamilyStore cfStore = keyspace.getColumnFamilyStore(CF_STANDARD1);
 
         for (int i = 900; i < 1000; ++i)
         {
-            Mutation rm = new Mutation("Keyspace1", ByteBufferUtil.bytes(Integer.toString(i)));
+            Mutation rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes(Integer.toString(i)));
             for (int j = 0; j < 8; ++j)
             {
-                rm.add("StandardLong1", cellname(j * 2), ByteBufferUtil.bytes("a"), j * 2);
+                rm.add(CF_STANDARD1, cellname(j * 2), ByteBufferUtil.bytes("a"), j * 2);
             }
             rm.apply();
         }
@@ -81,16 +98,16 @@ public class TimeSortTest extends SchemaLoader
 
         // interleave some new data to test memtable + sstable
         DecoratedKey key = Util.dk("900");
-        Mutation rm = new Mutation("Keyspace1", key.getKey());
+        Mutation rm = new Mutation(KEYSPACE1, key.getKey());
         for (int j = 0; j < 4; ++j)
         {
-            rm.add("StandardLong1", cellname(j * 2 + 1), ByteBufferUtil.bytes("b"), j * 2 + 1);
+            rm.add(CF_STANDARD1, cellname(j * 2 + 1), ByteBufferUtil.bytes("b"), j * 2 + 1);
         }
         rm.apply();
         // and some overwrites
-        rm = new Mutation("Keyspace1", key.getKey());
-        rm.add("StandardLong1", cellname(0), ByteBufferUtil.bytes("c"), 100);
-        rm.add("StandardLong1", cellname(10), ByteBufferUtil.bytes("c"), 100);
+        rm = new Mutation(KEYSPACE1, key.getKey());
+        rm.add(CF_STANDARD1, cellname(0), ByteBufferUtil.bytes("c"), 100);
+        rm.add(CF_STANDARD1, cellname(10), ByteBufferUtil.bytes("c"), 100);
         rm.apply();
 
         // verify
@@ -107,7 +124,7 @@ public class TimeSortTest extends SchemaLoader
         TreeSet<CellName> columnNames = new TreeSet<CellName>(cfStore.getComparator());
         columnNames.add(cellname(10));
         columnNames.add(cellname(0));
-        cf = cfStore.getColumnFamily(QueryFilter.getNamesFilter(Util.dk("900"), "StandardLong1", columnNames, System.currentTimeMillis()));
+        cf = cfStore.getColumnFamily(QueryFilter.getNamesFilter(Util.dk("900"), CF_STANDARD1, columnNames, System.currentTimeMillis()));
         assert "c".equals(ByteBufferUtil.string(cf.getColumn(cellname(0)).value()));
         assert "c".equals(ByteBufferUtil.string(cf.getColumn(cellname(10)).value()));
     }
@@ -119,7 +136,7 @@ public class TimeSortTest extends SchemaLoader
             DecoratedKey key = Util.dk(Integer.toString(i));
             for (int j = 0; j < 8; j += 3)
             {
-                ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("StandardLong1");
+                ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF_STANDARD1);
                 ColumnFamily cf = cfs.getColumnFamily(key, cellname(j * 2), Composites.EMPTY, false, 1000, System.currentTimeMillis());
                 Collection<Cell> cells = cf.getSortedColumns();
                 assert cells.size() == 8 - j;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/db/compaction/AntiCompactionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/AntiCompactionTest.java b/test/unit/org/apache/cassandra/db/compaction/AntiCompactionTest.java
index b8637a8..ac628ee 100644
--- a/test/unit/org/apache/cassandra/db/compaction/AntiCompactionTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/AntiCompactionTest.java
@@ -22,6 +22,11 @@ import java.util.Arrays;
 import java.util.Collection;
 import java.util.List;
 import java.util.concurrent.ExecutionException;
+
+import org.apache.cassandra.config.KSMetaData;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.locator.SimpleStrategy;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
@@ -41,11 +46,21 @@ import static junit.framework.Assert.assertFalse;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
-public class AntiCompactionTest extends SchemaLoader
+public class AntiCompactionTest
 {
-    private static final String KEYSPACE1 = "Keyspace1";
+    private static final String KEYSPACE1 = "AntiCompactionTest";
     private static final String CF = "Standard1";
 
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF));
+    }
+
     @Test
     public void antiCompactOne() throws InterruptedException, ExecutionException, IOException
     {
@@ -58,7 +73,7 @@ public class AntiCompactionTest extends SchemaLoader
             DecoratedKey key = Util.dk(Integer.toString(i));
             Mutation rm = new Mutation(KEYSPACE1, key.getKey());
             for (int j = 0; j < 10; j++)
-                rm.add("Standard1", Util.cellname(Integer.toString(j)),
+                rm.add(CF, Util.cellname(Integer.toString(j)),
                        ByteBufferUtil.EMPTY_BYTE_BUFFER,
                        timestamp,
                        0);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/db/compaction/BlacklistingCompactionsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/BlacklistingCompactionsTest.java b/test/unit/org/apache/cassandra/db/compaction/BlacklistingCompactionsTest.java
index e6626ea..8beea7f 100644
--- a/test/unit/org/apache/cassandra/db/compaction/BlacklistingCompactionsTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/BlacklistingCompactionsTest.java
@@ -31,20 +31,34 @@ import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.db.*;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.sstable.SSTableReader;
 import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.apache.cassandra.Util.cellname;
 
-public class BlacklistingCompactionsTest extends SchemaLoader
+public class BlacklistingCompactionsTest
 {
-    public static final String KEYSPACE = "Keyspace1";
+    private static final String KEYSPACE1 = "BlacklistingCompactionsTest";
+    private static final String CF_STANDARD1 = "Standard1";
 
     @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1));
+        closeStdErr();
+    }
+
     public static void closeStdErr()
     {
         // These tests generate an error message per CorruptSSTableException since it goes through
@@ -70,7 +84,7 @@ public class BlacklistingCompactionsTest extends SchemaLoader
     public void testBlacklisting(String compactionStrategy) throws Exception
     {
         // this test does enough rows to force multiple block indexes to be used
-        Keyspace keyspace = Keyspace.open(KEYSPACE);
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
         final ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("Standard1");
 
         final int ROWS_PER_SSTABLE = 10;
@@ -89,7 +103,7 @@ public class BlacklistingCompactionsTest extends SchemaLoader
             for (int i = 0; i < ROWS_PER_SSTABLE; i++)
             {
                 DecoratedKey key = Util.dk(String.valueOf(i % 2));
-                Mutation rm = new Mutation(KEYSPACE, key.getKey());
+                Mutation rm = new Mutation(KEYSPACE1, key.getKey());
                 long timestamp = j * ROWS_PER_SSTABLE + i;
                 rm.add("Standard1", cellname(i / 2), ByteBufferUtil.EMPTY_BYTE_BUFFER, timestamp);
                 maxTimestampExpected = Math.max(timestamp, maxTimestampExpected);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/db/compaction/CompactionsPurgeTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/CompactionsPurgeTest.java b/test/unit/org/apache/cassandra/db/compaction/CompactionsPurgeTest.java
index 80608f5..e8656c4 100644
--- a/test/unit/org/apache/cassandra/db/compaction/CompactionsPurgeTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/CompactionsPurgeTest.java
@@ -22,8 +22,14 @@ import java.util.Collection;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Future;
 
+import org.apache.cassandra.cache.CachingOptions;
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.db.*;
 
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.locator.SimpleStrategy;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
@@ -45,10 +51,43 @@ import static org.apache.cassandra.Util.cellname;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 
-public class CompactionsPurgeTest extends SchemaLoader
+public class CompactionsPurgeTest
 {
-    public static final String KEYSPACE1 = "Keyspace1";
-    public static final String KEYSPACE2 = "Keyspace2";
+    private static final String KEYSPACE1 = "CompactionsPurgeTest1";
+    private static final String CF_STANDARD1 = "Standard1";
+    private static final String CF_STANDARD2 = "Standard2";
+    private static final String KEYSPACE2 = "CompactionsPurgeTest2";
+    private static final String KEYSPACE_CACHED = "CompactionsPurgeTestCached";
+    private static final String CF_CACHED = "CachedCF";
+    private static final String KEYSPACE_CQL = "cql_keyspace";
+    private static final String CF_CQL = "table1";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD2));
+        SchemaLoader.createKeyspace(KEYSPACE2,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE2, CF_STANDARD1));
+        SchemaLoader.createKeyspace(KEYSPACE_CACHED,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE_CACHED, CF_CACHED).caching(CachingOptions.ALL));
+        SchemaLoader.createKeyspace(KEYSPACE_CQL,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    CFMetaData.compile("CREATE TABLE " + CF_CQL + " ("
+                                                     + "k int PRIMARY KEY,"
+                                                     + "v1 text,"
+                                                     + "v2 int"
+                                                     + ")", KEYSPACE_CQL));
+    }
 
     @Test
     public void testMajorCompactionPurge() throws ExecutionException, InterruptedException
@@ -235,8 +274,8 @@ public class CompactionsPurgeTest extends SchemaLoader
     {
         CompactionManager.instance.disableAutoCompaction();
 
-        String keyspaceName = "RowCacheSpace";
-        String cfName = "CachedCF";
+        String keyspaceName = KEYSPACE_CACHED;
+        String cfName = CF_CACHED;
         Keyspace keyspace = Keyspace.open(keyspaceName);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfName);
 
@@ -283,7 +322,7 @@ public class CompactionsPurgeTest extends SchemaLoader
     {
         CompactionManager.instance.disableAutoCompaction();
 
-        String keyspaceName = "Keyspace1";
+        String keyspaceName = KEYSPACE1;
         String cfName = "Standard1";
         Keyspace keyspace = Keyspace.open(keyspaceName);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfName);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/db/compaction/CompactionsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/CompactionsTest.java b/test/unit/org/apache/cassandra/db/compaction/CompactionsTest.java
index 58473b4..808a87e 100644
--- a/test/unit/org/apache/cassandra/db/compaction/CompactionsTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/CompactionsTest.java
@@ -21,37 +21,37 @@ package org.apache.cassandra.db.compaction;
 import java.io.File;
 import java.io.IOException;
 import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Set;
-import java.util.UUID;
+import java.util.*;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.columniterator.OnDiskAtomIterator;
 import org.apache.cassandra.db.filter.QueryFilter;
+import org.apache.cassandra.db.marshal.BytesType;
+import org.apache.cassandra.db.marshal.LongType;
 import org.apache.cassandra.dht.BytesToken;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.sstable.Component;
 import org.apache.cassandra.io.sstable.SSTableReader;
 import org.apache.cassandra.io.sstable.SSTableScanner;
 import org.apache.cassandra.io.sstable.SSTableWriter;
 import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
 import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.Pair;
+import org.junit.BeforeClass;
 import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.runner.RunWith;
@@ -63,15 +63,39 @@ import com.google.common.collect.Sets;
 import static org.junit.Assert.*;
 
 @RunWith(OrderedJUnit4ClassRunner.class)
-public class CompactionsTest extends SchemaLoader
+public class CompactionsTest
 {
-    private static final String STANDARD1 = "Standard1";
-    public static final String KEYSPACE1 = "Keyspace1";
+    private static final String KEYSPACE1 = "Keyspace1";
+    private static final String CF_STANDARD1 = "CF_STANDARD1";
+    private static final String CF_STANDARD2 = "Standard2";
+    private static final String CF_STANDARD3 = "Standard3";
+    private static final String CF_STANDARD4 = "Standard4";
+    private static final String CF_SUPER1 = "Super1";
+    private static final String CF_SUPER5 = "Super5";
+    private static final String CF_SUPERGC = "SuperDirectGC";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        Map<String, String> compactionOptions = new HashMap<>();
+        compactionOptions.put("tombstone_compaction_interval", "1");
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1).compactionStrategyOptions(compactionOptions),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD2),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD3),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD4),
+                                    SchemaLoader.superCFMD(KEYSPACE1, CF_SUPER1, LongType.instance),
+                                    SchemaLoader.superCFMD(KEYSPACE1, CF_SUPER5, BytesType.instance),
+                                    SchemaLoader.superCFMD(KEYSPACE1, CF_SUPERGC, BytesType.instance).gcGraceSeconds(0));
+    }
 
     public ColumnFamilyStore testSingleSSTableCompaction(String strategyClassName) throws Exception
     {
         Keyspace keyspace = Keyspace.open(KEYSPACE1);
-        ColumnFamilyStore store = keyspace.getColumnFamilyStore(STANDARD1);
+        ColumnFamilyStore store = keyspace.getColumnFamilyStore(CF_STANDARD1);
         store.clearUnsafe();
         store.metadata.gcGraceSeconds(1);
         store.setCompactionStrategyClass(strategyClassName);
@@ -79,7 +103,7 @@ public class CompactionsTest extends SchemaLoader
         // disable compaction while flushing
         store.disableAutoCompaction();
 
-        long timestamp = populate(KEYSPACE1, STANDARD1, 0, 9, 3); //ttl=3s
+        long timestamp = populate(KEYSPACE1, CF_STANDARD1, 0, 9, 3); //ttl=3s
 
         store.forceBlockingFlush();
         assertEquals(1, store.getSSTables().size());
@@ -180,7 +204,7 @@ public class CompactionsTest extends SchemaLoader
     public void testUncheckedTombstoneSizeTieredCompaction() throws Exception
     {
         Keyspace keyspace = Keyspace.open(KEYSPACE1);
-        ColumnFamilyStore store = keyspace.getColumnFamilyStore(STANDARD1);
+        ColumnFamilyStore store = keyspace.getColumnFamilyStore(CF_STANDARD1);
         store.clearUnsafe();
         store.metadata.gcGraceSeconds(1);
         store.metadata.compactionStrategyOptions.put("tombstone_compaction_interval", "1");
@@ -192,11 +216,11 @@ public class CompactionsTest extends SchemaLoader
         store.disableAutoCompaction();
 
         //Populate sstable1 with with keys [0..9]
-        populate(KEYSPACE1, STANDARD1, 0, 9, 3); //ttl=3s
+        populate(KEYSPACE1, CF_STANDARD1, 0, 9, 3); //ttl=3s
         store.forceBlockingFlush();
 
         //Populate sstable2 with with keys [10..19] (keys do not overlap with SSTable1)
-        long timestamp2 = populate(KEYSPACE1, STANDARD1, 10, 19, 3); //ttl=3s
+        long timestamp2 = populate(KEYSPACE1, CF_STANDARD1, 10, 19, 3); //ttl=3s
         store.forceBlockingFlush();
 
         assertEquals(2, store.getSSTables().size());
@@ -456,7 +480,7 @@ public class CompactionsTest extends SchemaLoader
 
         String cf = "Standard4";
         ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(cf);
-        insertData(KEYSPACE1, cf, 0, 1);
+        SchemaLoader.insertData(KEYSPACE1, cf, 0, 1);
         cfs.forceBlockingFlush();
 
         Collection<SSTableReader> sstables = cfs.getSSTables();
@@ -545,7 +569,7 @@ public class CompactionsTest extends SchemaLoader
         long timestamp = System.currentTimeMillis();
         DecoratedKey decoratedKey = Util.dk(String.format("%03d", key));
         Mutation rm = new Mutation(KEYSPACE1, decoratedKey.getKey());
-        rm.add("Standard1", Util.cellname("col"), ByteBufferUtil.EMPTY_BYTE_BUFFER, timestamp, 1000);
+        rm.add("CF_STANDARD1", Util.cellname("col"), ByteBufferUtil.EMPTY_BYTE_BUFFER, timestamp, 1000);
         rm.apply();
     }
 
@@ -554,7 +578,7 @@ public class CompactionsTest extends SchemaLoader
     public void testNeedsCleanup()
     {
         Keyspace keyspace = Keyspace.open(KEYSPACE1);
-        ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard1");
+        ColumnFamilyStore store = keyspace.getColumnFamilyStore("CF_STANDARD1");
         store.clearUnsafe();
 
         // disable compaction while flushing

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java b/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java
index defb087..71bf6e5 100644
--- a/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java
@@ -18,23 +18,24 @@
 package org.apache.cassandra.db.compaction;
 
 import java.nio.ByteBuffer;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.List;
-import java.util.UUID;
+import java.util.*;
 
 import org.junit.After;
 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.Util;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.sstable.SSTableReader;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.repair.RepairJobDesc;
 import org.apache.cassandra.repair.Validator;
 import org.apache.cassandra.service.ActiveRepairService;
@@ -45,16 +46,32 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
 @RunWith(OrderedJUnit4ClassRunner.class)
-public class LeveledCompactionStrategyTest extends SchemaLoader
+public class LeveledCompactionStrategyTest
 {
-    private String ksname = "Keyspace1";
-    private String cfname = "StandardLeveled";
-    private Keyspace keyspace = Keyspace.open(ksname);
-    private ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfname);
+    private static final String KEYSPACE1 = "LeveledCompactionStrategyTest";
+    private static final String CF_STANDARDDLEVELED = "StandardLeveled";
+    private Keyspace keyspace;
+    private ColumnFamilyStore cfs;
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        Map<String, String> leveledOptions = new HashMap<>();
+        leveledOptions.put("sstable_size_in_mb", "1");
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARDDLEVELED)
+                                                .compactionStrategyClass(LeveledCompactionStrategy.class)
+                                                .compactionStrategyOptions(leveledOptions));
+        }
 
     @Before
     public void enableCompaction()
     {
+        keyspace = Keyspace.open(KEYSPACE1);
+        cfs = keyspace.getColumnFamilyStore(CF_STANDARDDLEVELED);
         cfs.enableAutoCompaction();
     }
 
@@ -83,10 +100,10 @@ public class LeveledCompactionStrategyTest extends SchemaLoader
         for (int r = 0; r < rows; r++)
         {
             DecoratedKey key = Util.dk(String.valueOf(r));
-            Mutation rm = new Mutation(ksname, key.getKey());
+            Mutation rm = new Mutation(KEYSPACE1, key.getKey());
             for (int c = 0; c < columns; c++)
             {
-                rm.add(cfname, Util.cellname("column" + c), value, 0);
+                rm.add(CF_STANDARDDLEVELED, Util.cellname("column" + c), value, 0);
             }
             rm.apply();
             cfs.forceBlockingFlush();
@@ -99,10 +116,10 @@ public class LeveledCompactionStrategyTest extends SchemaLoader
         assert strategy.getLevelSize(2) > 0;
 
         Range<Token> range = new Range<>(Util.token(""), Util.token(""));
-        int gcBefore = keyspace.getColumnFamilyStore(cfname).gcBefore(System.currentTimeMillis());
+        int gcBefore = keyspace.getColumnFamilyStore(CF_STANDARDDLEVELED).gcBefore(System.currentTimeMillis());
         UUID parentRepSession = UUID.randomUUID();
         ActiveRepairService.instance.registerParentRepairSession(parentRepSession, Arrays.asList(cfs), Arrays.asList(range));
-        RepairJobDesc desc = new RepairJobDesc(parentRepSession, UUID.randomUUID(), ksname, cfname, range);
+        RepairJobDesc desc = new RepairJobDesc(parentRepSession, UUID.randomUUID(), KEYSPACE1, CF_STANDARDDLEVELED, range);
         Validator validator = new Validator(desc, FBUtilities.getBroadcastAddress(), gcBefore);
         CompactionManager.instance.submitValidation(cfs, validator).get();
     }
@@ -128,10 +145,10 @@ public class LeveledCompactionStrategyTest extends SchemaLoader
         for (int r = 0; r < rows; r++)
         {
             DecoratedKey key = Util.dk(String.valueOf(r));
-            Mutation rm = new Mutation(ksname, key.getKey());
+            Mutation rm = new Mutation(KEYSPACE1, key.getKey());
             for (int c = 0; c < columns; c++)
             {
-                rm.add(cfname, Util.cellname("column" + c), value, 0);
+                rm.add(CF_STANDARDDLEVELED, Util.cellname("column" + c), value, 0);
             }
             rm.apply();
             cfs.forceBlockingFlush();
@@ -167,10 +184,10 @@ public class LeveledCompactionStrategyTest extends SchemaLoader
         for (int r = 0; r < rows; r++)
         {
             DecoratedKey key = Util.dk(String.valueOf(r));
-            Mutation rm = new Mutation(ksname, key.getKey());
+            Mutation rm = new Mutation(KEYSPACE1, key.getKey());
             for (int c = 0; c < columns; c++)
             {
-                rm.add(cfname, Util.cellname("column" + c), value, 0);
+                rm.add(CF_STANDARDDLEVELED, Util.cellname("column" + c), value, 0);
             }
             rm.apply();
             cfs.forceBlockingFlush();
@@ -213,10 +230,10 @@ public class LeveledCompactionStrategyTest extends SchemaLoader
         for (int r = 0; r < rows; r++)
         {
             DecoratedKey key = Util.dk(String.valueOf(r));
-            Mutation rm = new Mutation(ksname, key.getKey());
+            Mutation rm = new Mutation(KEYSPACE1, key.getKey());
             for (int c = 0; c < columns; c++)
             {
-                rm.add(cfname, Util.cellname("column" + c), value, 0);
+                rm.add(CF_STANDARDDLEVELED, Util.cellname("column" + c), value, 0);
             }
             rm.apply();
             cfs.forceBlockingFlush();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/db/compaction/OneCompactionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/OneCompactionTest.java b/test/unit/org/apache/cassandra/db/compaction/OneCompactionTest.java
index bc3fe35..375dfe8 100644
--- a/test/unit/org/apache/cassandra/db/compaction/OneCompactionTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/OneCompactionTest.java
@@ -18,12 +18,15 @@
 */
 package org.apache.cassandra.db.compaction;
 
+import java.util.HashMap;
+import java.util.Map;
 import java.util.concurrent.ExecutionException;
 import java.util.Set;
 import java.util.HashSet;
 
 import org.apache.cassandra.Util;
 
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import static org.junit.Assert.assertEquals;
@@ -31,22 +34,42 @@ import static org.junit.Assert.assertEquals;
 import org.apache.cassandra.db.*;
 
 import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.KSMetaData;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 
-public class OneCompactionTest extends SchemaLoader
+public class OneCompactionTest
 {
+    public static final String KEYSPACE1 = "OneCompactionTest";
+    private static final String CF_STANDARD1 = "Standard1";
+    private static final String CF_STANDARD2 = "Standard2";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        Map<String, String> leveledOptions = new HashMap<>();
+        leveledOptions.put("sstable_size_in_mb", "1");
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1).compactionStrategyOptions(leveledOptions),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD2));
+    }
+
     private void testCompaction(String columnFamilyName, int insertsPerTable) throws ExecutionException, InterruptedException
     {
         CompactionManager.instance.disableAutoCompaction();
 
-        Keyspace keyspace = Keyspace.open("Keyspace1");
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
         ColumnFamilyStore store = keyspace.getColumnFamilyStore(columnFamilyName);
 
         Set<DecoratedKey> inserted = new HashSet<DecoratedKey>();
         for (int j = 0; j < insertsPerTable; j++) {
             DecoratedKey key = Util.dk(String.valueOf(j));
-            Mutation rm = new Mutation("Keyspace1", key.getKey());
+            Mutation rm = new Mutation(KEYSPACE1, key.getKey());
             rm.add(columnFamilyName, Util.cellname("0"), ByteBufferUtil.EMPTY_BYTE_BUFFER, j);
             rm.apply();
             inserted.add(key);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategyTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategyTest.java b/test/unit/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategyTest.java
index 6132dad..ef886e7 100644
--- a/test/unit/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategyTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategyTest.java
@@ -20,13 +20,16 @@ package org.apache.cassandra.db.compaction;
 import java.nio.ByteBuffer;
 import java.util.*;
 
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.sstable.SSTableReader;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.metrics.RestorableMeter;
 import org.apache.cassandra.utils.Pair;
 
@@ -38,8 +41,23 @@ import static org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy.va
 
 import static org.junit.Assert.*;
 
-public class SizeTieredCompactionStrategyTest extends SchemaLoader
+public class SizeTieredCompactionStrategyTest
 {
+    public static final String KEYSPACE1 = "SizeTieredCompactionStrategyTest";
+    private static final String CF_STANDARD1 = "Standard1";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        Map<String, String> leveledOptions = new HashMap<>();
+        leveledOptions.put("sstable_size_in_mb", "1");
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1)
+                                                .compactionStrategyOptions(leveledOptions));
+    }
 
     @Test
     public void testOptionsValidation() throws ConfigurationException
@@ -146,7 +164,7 @@ public class SizeTieredCompactionStrategyTest extends SchemaLoader
     @Test
     public void testPrepBucket() throws Exception
     {
-        String ksname = "Keyspace1";
+        String ksname = KEYSPACE1;
         String cfname = "Standard1";
         Keyspace keyspace = Keyspace.open(ksname);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfname);
@@ -190,7 +208,7 @@ public class SizeTieredCompactionStrategyTest extends SchemaLoader
     @Test
     public void testFilterColdSSTables() throws Exception
     {
-        String ksname = "Keyspace1";
+        String ksname = KEYSPACE1;
         String cfname = "Standard1";
         Keyspace keyspace = Keyspace.open(ksname);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfname);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/db/compaction/TTLExpiryTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/TTLExpiryTest.java b/test/unit/org/apache/cassandra/db/compaction/TTLExpiryTest.java
index b98af68..3a2bc4a 100644
--- a/test/unit/org/apache/cassandra/db/compaction/TTLExpiryTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/TTLExpiryTest.java
@@ -20,31 +20,48 @@ package org.apache.cassandra.db.compaction;
  * 
  */
 
+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.config.KSMetaData;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.columniterator.OnDiskAtomIterator;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.sstable.SSTableReader;
 import org.apache.cassandra.io.sstable.SSTableScanner;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
 @RunWith(OrderedJUnit4ClassRunner.class)
-public class TTLExpiryTest extends SchemaLoader
+public class TTLExpiryTest
 {
+    public static final String KEYSPACE1 = "TTLExpiryTest";
+    private static final String CF_STANDARD1 = "Standard1";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1));
+    }
+
     @Test
     public void testSimpleExpire() throws InterruptedException
     {
-        ColumnFamilyStore cfs = Keyspace.open("Keyspace1").getColumnFamilyStore("Standard1");
+        ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore("Standard1");
         cfs.disableAutoCompaction();
         cfs.metadata.gcGraceSeconds(0);
         long timestamp = System.currentTimeMillis();
-        Mutation rm = new Mutation("Keyspace1", Util.dk("ttl").getKey());
+        Mutation rm = new Mutation(KEYSPACE1, Util.dk("ttl").getKey());
         rm.add("Standard1", Util.cellname("col"),
                ByteBufferUtil.EMPTY_BYTE_BUFFER,
                timestamp,
@@ -57,21 +74,21 @@ public class TTLExpiryTest extends SchemaLoader
         rm.apply();
         cfs.forceBlockingFlush();
 
-        rm = new Mutation("Keyspace1", Util.dk("ttl").getKey());
+        rm = new Mutation(KEYSPACE1, Util.dk("ttl").getKey());
                 rm.add("Standard1", Util.cellname("col2"),
                        ByteBufferUtil.EMPTY_BYTE_BUFFER,
                        timestamp,
                        1);
                 rm.apply();
         cfs.forceBlockingFlush();
-        rm = new Mutation("Keyspace1", Util.dk("ttl").getKey());
+        rm = new Mutation(KEYSPACE1, Util.dk("ttl").getKey());
         rm.add("Standard1", Util.cellname("col3"),
                    ByteBufferUtil.EMPTY_BYTE_BUFFER,
                    timestamp,
                    1);
         rm.apply();
         cfs.forceBlockingFlush();
-        rm = new Mutation("Keyspace1", Util.dk("ttl").getKey());
+        rm = new Mutation(KEYSPACE1, Util.dk("ttl").getKey());
         rm.add("Standard1", Util.cellname("col311"),
                    ByteBufferUtil.EMPTY_BYTE_BUFFER,
                    timestamp,
@@ -88,11 +105,11 @@ public class TTLExpiryTest extends SchemaLoader
     @Test
     public void testNoExpire() throws InterruptedException
     {
-        ColumnFamilyStore cfs = Keyspace.open("Keyspace1").getColumnFamilyStore("Standard1");
+        ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore("Standard1");
         cfs.disableAutoCompaction();
         cfs.metadata.gcGraceSeconds(0);
         long timestamp = System.currentTimeMillis();
-        Mutation rm = new Mutation("Keyspace1", Util.dk("ttl").getKey());
+        Mutation rm = new Mutation(KEYSPACE1, Util.dk("ttl").getKey());
         rm.add("Standard1", Util.cellname("col"),
                ByteBufferUtil.EMPTY_BYTE_BUFFER,
                timestamp,
@@ -105,14 +122,14 @@ public class TTLExpiryTest extends SchemaLoader
         rm.apply();
         cfs.forceBlockingFlush();
 
-        rm = new Mutation("Keyspace1", Util.dk("ttl").getKey());
+        rm = new Mutation(KEYSPACE1, Util.dk("ttl").getKey());
                 rm.add("Standard1", Util.cellname("col2"),
                        ByteBufferUtil.EMPTY_BYTE_BUFFER,
                        timestamp,
                        1);
                 rm.apply();
         cfs.forceBlockingFlush();
-        rm = new Mutation("Keyspace1", Util.dk("ttl").getKey());
+        rm = new Mutation(KEYSPACE1, Util.dk("ttl").getKey());
         rm.add("Standard1", Util.cellname("col3"),
                    ByteBufferUtil.EMPTY_BYTE_BUFFER,
                    timestamp,
@@ -120,7 +137,7 @@ public class TTLExpiryTest extends SchemaLoader
         rm.apply();
         cfs.forceBlockingFlush();
         DecoratedKey noTTLKey = Util.dk("nottl");
-        rm = new Mutation("Keyspace1", noTTLKey.getKey());
+        rm = new Mutation(KEYSPACE1, noTTLKey.getKey());
         rm.add("Standard1", Util.cellname("col311"),
                    ByteBufferUtil.EMPTY_BYTE_BUFFER,
                    timestamp);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/db/index/PerRowSecondaryIndexTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/index/PerRowSecondaryIndexTest.java b/test/unit/org/apache/cassandra/db/index/PerRowSecondaryIndexTest.java
index 158dd2c..e312182 100644
--- a/test/unit/org/apache/cassandra/db/index/PerRowSecondaryIndexTest.java
+++ b/test/unit/org/apache/cassandra/db/index/PerRowSecondaryIndexTest.java
@@ -23,11 +23,13 @@ import java.util.Arrays;
 import java.util.Set;
 
 import org.junit.Before;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
 import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.db.Cell;
 import org.apache.cassandra.db.ColumnFamily;
 import org.apache.cassandra.db.ColumnFamilyStore;
@@ -36,12 +38,13 @@ import org.apache.cassandra.db.Mutation;
 import org.apache.cassandra.db.composites.CellName;
 import org.apache.cassandra.db.filter.QueryFilter;
 import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.concurrent.OpOrder;
 
 import static org.junit.Assert.*;
 
-public class PerRowSecondaryIndexTest extends SchemaLoader
+public class PerRowSecondaryIndexTest
 {
 
     // test that when index(key) is called on a PRSI index,
@@ -50,6 +53,19 @@ public class PerRowSecondaryIndexTest extends SchemaLoader
     // indexed & stashes it in a static variable for inspection
     // in the test.
 
+    private static final String KEYSPACE1 = "PerRowSecondaryIndexTest";
+    private static final String CF_INDEXED = "Indexed1";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.perRowIndexedCFMD(KEYSPACE1, CF_INDEXED));
+    }
+
     @Before
     public void clearTestStub()
     {
@@ -61,7 +77,7 @@ public class PerRowSecondaryIndexTest extends SchemaLoader
     {
         // create a row then test that the configured index instance was able to read the row
         Mutation rm;
-        rm = new Mutation("PerRowSecondaryIndex", ByteBufferUtil.bytes("k1"));
+        rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes("k1"));
         rm.add("Indexed1", Util.cellname("indexed"), ByteBufferUtil.bytes("foo"), 1);
         rm.apply();
 
@@ -70,7 +86,7 @@ public class PerRowSecondaryIndexTest extends SchemaLoader
         assertEquals(ByteBufferUtil.bytes("foo"), indexedRow.getColumn(Util.cellname("indexed")).value());
 
         // update the row and verify what was indexed
-        rm = new Mutation("PerRowSecondaryIndex", ByteBufferUtil.bytes("k1"));
+        rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes("k1"));
         rm.add("Indexed1", Util.cellname("indexed"), ByteBufferUtil.bytes("bar"), 2);
         rm.apply();
 
@@ -85,7 +101,7 @@ public class PerRowSecondaryIndexTest extends SchemaLoader
     {
         // issue a column delete and test that the configured index instance was notified to update
         Mutation rm;
-        rm = new Mutation("PerRowSecondaryIndex", ByteBufferUtil.bytes("k2"));
+        rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes("k2"));
         rm.delete("Indexed1", Util.cellname("indexed"), 1);
         rm.apply();
 
@@ -103,7 +119,7 @@ public class PerRowSecondaryIndexTest extends SchemaLoader
     {
         // issue a row level delete and test that the configured index instance was notified to update
         Mutation rm;
-        rm = new Mutation("PerRowSecondaryIndex", ByteBufferUtil.bytes("k3"));
+        rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes("k3"));
         rm.delete("Indexed1", 1);
         rm.apply();
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/db/marshal/CompositeTypeTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/marshal/CompositeTypeTest.java b/test/unit/org/apache/cassandra/db/marshal/CompositeTypeTest.java
index f606780..35be43d 100644
--- a/test/unit/org/apache/cassandra/db/marshal/CompositeTypeTest.java
+++ b/test/unit/org/apache/cassandra/db/marshal/CompositeTypeTest.java
@@ -19,28 +19,30 @@
 package org.apache.cassandra.db.marshal;
 
 import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-import java.util.UUID;
+import java.util.*;
 
-import org.apache.cassandra.serializers.MarshalException;
+import org.junit.BeforeClass;
 import org.junit.Test;
 import static org.junit.Assert.fail;
 import static org.junit.Assert.assertEquals;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.SyntaxException;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.composites.CellNames;
 import org.apache.cassandra.db.filter.QueryFilter;
+import org.apache.cassandra.locator.SimpleStrategy;
+import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.utils.*;
 
-public class CompositeTypeTest extends SchemaLoader
+public class CompositeTypeTest
 {
-    private static final String cfName = "StandardComposite";
+    private static final String KEYSPACE1 = "CompositeTypeTest";
+    private static final String CF_STANDARDCOMPOSITE = "StandardComposite";
     private static final CompositeType comparator;
     static
     {
@@ -59,6 +61,17 @@ public class CompositeTypeTest extends SchemaLoader
             uuids[i] = UUIDGen.getTimeUUID();
     }
 
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        AbstractType<?> composite = CompositeType.getInstance(Arrays.asList(new AbstractType<?>[]{BytesType.instance, TimeUUIDType.instance, IntegerType.instance}));
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    CFMetaData.denseCFMetaData(KEYSPACE1, CF_STANDARDCOMPOSITE, composite));
+    }
+
     @Test
     public void testEndOfComponent()
     {
@@ -164,8 +177,8 @@ public class CompositeTypeTest extends SchemaLoader
     @Test
     public void testFullRound() throws Exception
     {
-        Keyspace keyspace = Keyspace.open("Keyspace1");
-        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfName);
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF_STANDARDCOMPOSITE);
 
         ByteBuffer cname1 = createCompositeKey("test1", null, -1, false);
         ByteBuffer cname2 = createCompositeKey("test1", uuids[0], 24, false);
@@ -174,7 +187,7 @@ public class CompositeTypeTest extends SchemaLoader
         ByteBuffer cname5 = createCompositeKey("test2", uuids[1], 42, false);
 
         ByteBuffer key = ByteBufferUtil.bytes("k");
-        Mutation rm = new Mutation("Keyspace1", key);
+        Mutation rm = new Mutation(KEYSPACE1, key);
         addColumn(rm, cname5);
         addColumn(rm, cname1);
         addColumn(rm, cname4);
@@ -182,7 +195,7 @@ public class CompositeTypeTest extends SchemaLoader
         addColumn(rm, cname3);
         rm.apply();
 
-        ColumnFamily cf = cfs.getColumnFamily(QueryFilter.getIdentityFilter(Util.dk("k"), cfName, System.currentTimeMillis()));
+        ColumnFamily cf = cfs.getColumnFamily(QueryFilter.getIdentityFilter(Util.dk("k"), CF_STANDARDCOMPOSITE, System.currentTimeMillis()));
 
         Iterator<Cell> iter = cf.getSortedColumns().iterator();
 
@@ -258,7 +271,7 @@ public class CompositeTypeTest extends SchemaLoader
 
     private void addColumn(Mutation rm, ByteBuffer cname)
     {
-        rm.add(cfName, CellNames.simpleDense(cname), ByteBufferUtil.EMPTY_BYTE_BUFFER, 0);
+        rm.add(CF_STANDARDCOMPOSITE, CellNames.simpleDense(cname), ByteBufferUtil.EMPTY_BYTE_BUFFER, 0);
     }
 
     private ByteBuffer createCompositeKey(String s, UUID uuid, int i, boolean lastIsOne)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/db/marshal/DynamicCompositeTypeTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/marshal/DynamicCompositeTypeTest.java b/test/unit/org/apache/cassandra/db/marshal/DynamicCompositeTypeTest.java
index 26d2e62..fa10dbb 100644
--- a/test/unit/org/apache/cassandra/db/marshal/DynamicCompositeTypeTest.java
+++ b/test/unit/org/apache/cassandra/db/marshal/DynamicCompositeTypeTest.java
@@ -24,25 +24,31 @@ import java.util.HashMap;
 import java.util.Map;
 import java.util.UUID;
 
-import org.apache.cassandra.serializers.MarshalException;
+import org.junit.BeforeClass;
 import org.junit.Test;
 import static org.junit.Assert.fail;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.composites.*;
 import org.apache.cassandra.db.filter.QueryFilter;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.locator.SimpleStrategy;
+import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.utils.*;
 
-public class DynamicCompositeTypeTest extends SchemaLoader
+public class DynamicCompositeTypeTest
 {
-    private static final String cfName = "StandardDynamicComposite";
+    private static final String KEYSPACE1 = "DynamicCompositeType";
+    private static final String CF_STANDARDDYNCOMPOSITE = "StandardDynamicComposite";
+    private static Map<Byte, AbstractType<?>> aliases = new HashMap<>();
 
     private static final DynamicCompositeType comparator;
     static
     {
-        Map<Byte, AbstractType<?>> aliases = new HashMap<Byte, AbstractType<?>>();
         aliases.put((byte)'b', BytesType.instance);
         aliases.put((byte)'t', TimeUUIDType.instance);
         comparator = DynamicCompositeType.getInstance(aliases);
@@ -56,6 +62,17 @@ public class DynamicCompositeTypeTest extends SchemaLoader
             uuids[i] = UUIDGen.getTimeUUID();
     }
 
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        AbstractType<?> dynamicComposite = DynamicCompositeType.getInstance(aliases);
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    CFMetaData.denseCFMetaData(KEYSPACE1, CF_STANDARDDYNCOMPOSITE, dynamicComposite));
+    }
+
     @Test
     public void testEndOfComponent()
     {
@@ -163,8 +180,8 @@ public class DynamicCompositeTypeTest extends SchemaLoader
     @Test
     public void testFullRound() throws Exception
     {
-        Keyspace keyspace = Keyspace.open("Keyspace1");
-        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfName);
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF_STANDARDDYNCOMPOSITE);
 
         ByteBuffer cname1 = createDynamicCompositeKey("test1", null, -1, false);
         ByteBuffer cname2 = createDynamicCompositeKey("test1", uuids[0], 24, false);
@@ -173,7 +190,7 @@ public class DynamicCompositeTypeTest extends SchemaLoader
         ByteBuffer cname5 = createDynamicCompositeKey("test2", uuids[1], 42, false);
 
         ByteBuffer key = ByteBufferUtil.bytes("k");
-        Mutation rm = new Mutation("Keyspace1", key);
+        Mutation rm = new Mutation(KEYSPACE1, key);
         addColumn(rm, cname5);
         addColumn(rm, cname1);
         addColumn(rm, cname4);
@@ -181,7 +198,7 @@ public class DynamicCompositeTypeTest extends SchemaLoader
         addColumn(rm, cname3);
         rm.apply();
 
-        ColumnFamily cf = cfs.getColumnFamily(QueryFilter.getIdentityFilter(Util.dk("k"), cfName, System.currentTimeMillis()));
+        ColumnFamily cf = cfs.getColumnFamily(QueryFilter.getIdentityFilter(Util.dk("k"), CF_STANDARDDYNCOMPOSITE, System.currentTimeMillis()));
 
         Iterator<Cell> iter = cf.getSortedColumns().iterator();
 
@@ -232,7 +249,7 @@ public class DynamicCompositeTypeTest extends SchemaLoader
 
     private void addColumn(Mutation rm, ByteBuffer cname)
     {
-        rm.add(cfName, CellNames.simpleDense(cname), ByteBufferUtil.EMPTY_BYTE_BUFFER, 0);
+        rm.add(CF_STANDARDDYNCOMPOSITE, CellNames.simpleDense(cname), ByteBufferUtil.EMPTY_BYTE_BUFFER, 0);
     }
 
     private ByteBuffer createDynamicCompositeKey(String s, UUID uuid, int i, boolean lastIsOne)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/dht/BootStrapperTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/dht/BootStrapperTest.java b/test/unit/org/apache/cassandra/dht/BootStrapperTest.java
index ffad335..b95572f 100644
--- a/test/unit/org/apache/cassandra/dht/BootStrapperTest.java
+++ b/test/unit/org/apache/cassandra/dht/BootStrapperTest.java
@@ -25,6 +25,7 @@ import java.util.HashSet;
 import java.util.Set;
 import java.util.Map;
 
+import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 
@@ -32,6 +33,7 @@ import org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.gms.IFailureDetectionEventListener;
 import org.apache.cassandra.gms.IFailureDetector;
 import org.apache.cassandra.locator.TokenMetadata;
@@ -40,8 +42,16 @@ import org.apache.cassandra.service.StorageService;
 import static org.junit.Assert.*;
 
 @RunWith(OrderedJUnit4ClassRunner.class)
-public class BootStrapperTest extends SchemaLoader
+public class BootStrapperTest
 {
+    @BeforeClass
+    public static void setup() throws ConfigurationException
+    {
+        SchemaLoader.startGossiper();
+        SchemaLoader.prepareServer();
+        SchemaLoader.schemaDefinition("BootStrapperTest");
+    }
+
     @Test
     public void testSourceTargetComputation() throws UnknownHostException
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/io/BloomFilterTrackerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/BloomFilterTrackerTest.java b/test/unit/org/apache/cassandra/io/BloomFilterTrackerTest.java
index 4943a3a..afcf2a5 100644
--- a/test/unit/org/apache/cassandra/io/BloomFilterTrackerTest.java
+++ b/test/unit/org/apache/cassandra/io/BloomFilterTrackerTest.java
@@ -23,12 +23,11 @@ package org.apache.cassandra.io;
 
 import org.junit.Test;
 
-import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.io.sstable.BloomFilterTracker;
 
 import static org.junit.Assert.assertEquals;
 
-public class BloomFilterTrackerTest extends SchemaLoader
+public class BloomFilterTrackerTest
 {
     @Test
     public void testAddingFalsePositives()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/io/sstable/IndexSummaryManagerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/IndexSummaryManagerTest.java b/test/unit/org/apache/cassandra/io/sstable/IndexSummaryManagerTest.java
index 5281449..83c9daa 100644
--- a/test/unit/org/apache/cassandra/io/sstable/IndexSummaryManagerTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/IndexSummaryManagerTest.java
@@ -26,6 +26,7 @@ import java.util.concurrent.TimeUnit;
 
 import org.junit.After;
 import org.junit.Before;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.slf4j.Logger;
@@ -33,8 +34,12 @@ import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
+import org.apache.cassandra.cache.CachingOptions;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.filter.QueryFilter;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.metrics.RestorableMeter;
 
 import static org.apache.cassandra.io.sstable.Downsampling.BASE_SAMPLING_LEVEL;
@@ -46,7 +51,7 @@ import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 
-public class IndexSummaryManagerTest extends SchemaLoader
+public class IndexSummaryManagerTest
 {
     private static final Logger logger = LoggerFactory.getLogger(IndexSummaryManagerTest.class);
 
@@ -54,11 +59,28 @@ public class IndexSummaryManagerTest extends SchemaLoader
     int originalMaxIndexInterval;
     long originalCapacity;
 
+    private static final String KEYSPACE1 = "IndexSummaryManagerTest";
+    // index interval of 8, no key caching
+    private static final String CF_STANDARDLOWiINTERVAL = "StandardLowIndexInterval";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARDLOWiINTERVAL)
+                                                .minIndexInterval(8)
+                                                .maxIndexInterval(256)
+                                                .caching(CachingOptions.NONE));
+    }
+
     @Before
     public void beforeTest()
     {
-        String ksname = "Keyspace1";
-        String cfname = "StandardLowIndexInterval"; // index interval of 8, no key caching
+        String ksname = KEYSPACE1;
+        String cfname = CF_STANDARDLOWiINTERVAL; // index interval of 8, no key caching
         Keyspace keyspace = Keyspace.open(ksname);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfname);
         originalMinIndexInterval = cfs.metadata.getMinIndexInterval();
@@ -69,8 +91,8 @@ public class IndexSummaryManagerTest extends SchemaLoader
     @After
     public void afterTest()
     {
-        String ksname = "Keyspace1";
-        String cfname = "StandardLowIndexInterval"; // index interval of 8, no key caching
+        String ksname = KEYSPACE1;
+        String cfname = CF_STANDARDLOWiINTERVAL; // index interval of 8, no key caching
         Keyspace keyspace = Keyspace.open(ksname);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfname);
         cfs.metadata.minIndexInterval(originalMinIndexInterval);
@@ -162,8 +184,8 @@ public class IndexSummaryManagerTest extends SchemaLoader
     @Test
     public void testChangeMinIndexInterval() throws IOException
     {
-        String ksname = "Keyspace1";
-        String cfname = "StandardLowIndexInterval"; // index interval of 8, no key caching
+        String ksname = KEYSPACE1;
+        String cfname = CF_STANDARDLOWiINTERVAL; // index interval of 8, no key caching
         Keyspace keyspace = Keyspace.open(ksname);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfname);
         int numSSTables = 1;
@@ -234,8 +256,8 @@ public class IndexSummaryManagerTest extends SchemaLoader
     @Test
     public void testChangeMaxIndexInterval() throws IOException
     {
-        String ksname = "Keyspace1";
-        String cfname = "StandardLowIndexInterval"; // index interval of 8, no key caching
+        String ksname = KEYSPACE1;
+        String cfname = CF_STANDARDLOWiINTERVAL; // index interval of 8, no key caching
         Keyspace keyspace = Keyspace.open(ksname);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfname);
         int numSSTables = 1;
@@ -274,8 +296,8 @@ public class IndexSummaryManagerTest extends SchemaLoader
     @Test(timeout = 10000)
     public void testRedistributeSummaries() throws IOException
     {
-        String ksname = "Keyspace1";
-        String cfname = "StandardLowIndexInterval"; // index interval of 8, no key caching
+        String ksname = KEYSPACE1;
+        String cfname = CF_STANDARDLOWiINTERVAL; // index interval of 8, no key caching
         Keyspace keyspace = Keyspace.open(ksname);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfname);
         int numSSTables = 4;
@@ -395,8 +417,8 @@ public class IndexSummaryManagerTest extends SchemaLoader
     @Test
     public void testRebuildAtSamplingLevel() throws IOException
     {
-        String ksname = "Keyspace1";
-        String cfname = "StandardLowIndexInterval";
+        String ksname = KEYSPACE1;
+        String cfname = CF_STANDARDLOWiINTERVAL;
         Keyspace keyspace = Keyspace.open(ksname);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfname);
         cfs.truncateBlocking();
@@ -452,8 +474,8 @@ public class IndexSummaryManagerTest extends SchemaLoader
         manager.setMemoryPoolCapacityInMB(10);
         assertEquals(10, manager.getMemoryPoolCapacityInMB());
 
-        String ksname = "Keyspace1";
-        String cfname = "StandardLowIndexInterval"; // index interval of 8, no key caching
+        String ksname = KEYSPACE1;
+        String cfname = CF_STANDARDLOWiINTERVAL; // index interval of 8, no key caching
         Keyspace keyspace = Keyspace.open(ksname);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfname);
         cfs.truncateBlocking();
@@ -478,7 +500,7 @@ public class IndexSummaryManagerTest extends SchemaLoader
         assertTrue(manager.getAverageIndexInterval() >= cfs.metadata.getMinIndexInterval());
         Map<String, Integer> intervals = manager.getIndexIntervals();
         for (Map.Entry<String, Integer> entry : intervals.entrySet())
-            if (entry.getKey().contains("StandardLowIndexInterval"))
+            if (entry.getKey().contains(CF_STANDARDLOWiINTERVAL))
                 assertEquals(cfs.metadata.getMinIndexInterval(), entry.getValue(), 0.001);
 
         manager.setMemoryPoolCapacityInMB(0);
@@ -487,7 +509,7 @@ public class IndexSummaryManagerTest extends SchemaLoader
         intervals = manager.getIndexIntervals();
         for (Map.Entry<String, Integer> entry : intervals.entrySet())
         {
-            if (entry.getKey().contains("StandardLowIndexInterval"))
+            if (entry.getKey().contains(CF_STANDARDLOWiINTERVAL))
                 assertTrue(entry.getValue() >= cfs.metadata.getMinIndexInterval());
         }
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java b/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java
index 2dc07ec..0a7f2c4 100644
--- a/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java
@@ -31,6 +31,7 @@ import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.db.ColumnFamily;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.DecoratedKey;
@@ -41,6 +42,8 @@ import org.apache.cassandra.db.composites.CellNameType;
 import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.streaming.StreamPlan;
 import org.apache.cassandra.streaming.StreamSession;
@@ -50,16 +53,26 @@ import org.apache.cassandra.utils.FBUtilities;
 /**
  * Tests backwards compatibility for SSTables
  */
-public class LegacySSTableTest extends SchemaLoader
+public class LegacySSTableTest
 {
     public static final String LEGACY_SSTABLE_PROP = "legacy-sstable-root";
-    public static final String KSNAME = "Keyspace1";
+    public static final String KSNAME = "LegacySSTableTest";
     public static final String CFNAME = "Standard1";
 
     public static Set<String> TEST_DATA;
     public static File LEGACY_SSTABLE_ROOT;
 
     @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KSNAME,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KSNAME, CFNAME));
+        beforeClass();
+    }
+
     public static void beforeClass()
     {
         Keyspace.setInitialized();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/io/sstable/SSTableLoaderTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableLoaderTest.java b/test/unit/org/apache/cassandra/io/sstable/SSTableLoaderTest.java
index 39beb94..815fc68 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableLoaderTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableLoaderTest.java
@@ -27,12 +27,14 @@ import org.junit.Test;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
 import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.Row;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
@@ -40,12 +42,24 @@ import org.apache.cassandra.utils.OutputHandler;
 
 import static org.junit.Assert.assertEquals;
 
-public class SSTableLoaderTest extends SchemaLoader
+public class SSTableLoaderTest
 {
+    public static final String KEYSPACE1 = "SSTableLoaderTest";
+    public static final String CF_STANDARD = "Standard1";
+
     @BeforeClass
+    public static void defineSchema() throws Exception
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD));
+        setup();
+    }
+
     public static void setup() throws Exception
     {
-        Keyspace.setInitialized();
         StorageService.instance.initServer();
     }
 
@@ -53,9 +67,9 @@ public class SSTableLoaderTest extends SchemaLoader
     public void testLoadingSSTable() throws Exception
     {
         File tempdir = Files.createTempDir();
-        File dataDir = new File(tempdir.getAbsolutePath() + File.separator + "Keyspace1" + File.separator + "Standard1");
+        File dataDir = new File(tempdir.getAbsolutePath() + File.separator + KEYSPACE1 + File.separator + CF_STANDARD);
         assert dataDir.mkdirs();
-        CFMetaData cfmeta = Schema.instance.getCFMetaData("Keyspace1", "Standard1");
+        CFMetaData cfmeta = Schema.instance.getCFMetaData(KEYSPACE1, CF_STANDARD);
         SSTableSimpleUnsortedWriter writer = new SSTableSimpleUnsortedWriter(dataDir,
                                                                              cfmeta,
                                                                              StorageService.getPartitioner(),
@@ -69,7 +83,7 @@ public class SSTableLoaderTest extends SchemaLoader
         {
             public void init(String keyspace)
             {
-                for (Range<Token> range : StorageService.instance.getLocalRanges("Keyspace1"))
+                for (Range<Token> range : StorageService.instance.getLocalRanges(KEYSPACE1))
                     addRangeForEndpoint(range, FBUtilities.getBroadcastAddress());
                 setPartitioner(StorageService.getPartitioner());
             }
@@ -82,7 +96,7 @@ public class SSTableLoaderTest extends SchemaLoader
 
         loader.stream().get();
 
-        List<Row> rows = Util.getRangeSlice(Keyspace.open("Keyspace1").getColumnFamilyStore("Standard1"));
+        List<Row> rows = Util.getRangeSlice(Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF_STANDARD));
         assertEquals(1, rows.size());
         assertEquals(key, rows.get(0).key);
         assertEquals(ByteBufferUtil.bytes(100), rows.get(0).cf.getColumn(Util.cellname("col1")).value());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/io/sstable/SSTableMetadataTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableMetadataTest.java b/test/unit/org/apache/cassandra/io/sstable/SSTableMetadataTest.java
index 72307c5..19d984a 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableMetadataTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableMetadataTest.java
@@ -23,15 +23,22 @@ package org.apache.cassandra.io.sstable;
 import java.nio.ByteBuffer;
 import java.nio.charset.CharacterCodingException;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.List;
 import java.util.concurrent.ExecutionException;
+
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.composites.*;
 import org.apache.cassandra.db.context.CounterContext;
+import org.apache.cassandra.db.marshal.*;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.CounterId;
 
@@ -41,18 +48,40 @@ import static org.junit.Assert.assertTrue;
 
 import static org.apache.cassandra.Util.cellname;
 
-public class SSTableMetadataTest extends SchemaLoader
+public class SSTableMetadataTest
 {
+    public static final String KEYSPACE1 = "SSTableMetadataTest";
+    public static final String CF_STANDARD = "Standard1";
+    public static final String CF_STANDARD2 = "Standard2";
+    public static final String CF_STANDARD3 = "Standard3";
+    public static final String CF_STANDARDCOMPOSITE2 = "StandardComposite2";
+    public static final String CF_COUNTER1 = "Counter1";
+
+    @BeforeClass
+    public static void defineSchema() throws Exception
+    {
+        AbstractType<?> compositeMaxMin = CompositeType.getInstance(Arrays.asList(new AbstractType<?>[]{BytesType.instance, IntegerType.instance}));
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD2),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD3),
+                                    CFMetaData.denseCFMetaData(KEYSPACE1, CF_STANDARDCOMPOSITE2, compositeMaxMin),
+                                    CFMetaData.denseCFMetaData(KEYSPACE1, CF_COUNTER1, BytesType.instance).defaultValidator(CounterColumnType.instance));
+    }
+
     @Test
     public void testTrackMaxDeletionTime()
     {
-        Keyspace keyspace = Keyspace.open("Keyspace1");
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
         ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard1");
         long timestamp = System.currentTimeMillis();
         for(int i = 0; i < 10; i++)
         {
             DecoratedKey key = Util.dk(Integer.toString(i));
-            Mutation rm = new Mutation("Keyspace1", key.getKey());
+            Mutation rm = new Mutation(KEYSPACE1, key.getKey());
             for (int j = 0; j < 10; j++)
                 rm.add("Standard1", cellname(Integer.toString(j)),
                        ByteBufferUtil.EMPTY_BYTE_BUFFER,
@@ -60,7 +89,7 @@ public class SSTableMetadataTest extends SchemaLoader
                        10 + j);
             rm.apply();
         }
-        Mutation rm = new Mutation("Keyspace1", Util.dk("longttl").getKey());
+        Mutation rm = new Mutation(KEYSPACE1, Util.dk("longttl").getKey());
         rm.add("Standard1", cellname("col"),
                ByteBufferUtil.EMPTY_BYTE_BUFFER,
                timestamp,
@@ -76,7 +105,7 @@ public class SSTableMetadataTest extends SchemaLoader
             assertEquals(ttltimestamp + 10000, firstDelTime, 10);
 
         }
-        rm = new Mutation("Keyspace1", Util.dk("longttl2").getKey());
+        rm = new Mutation(KEYSPACE1, Util.dk("longttl2").getKey());
         rm.add("Standard1", cellname("col"),
                ByteBufferUtil.EMPTY_BYTE_BUFFER,
                timestamp,
@@ -119,11 +148,11 @@ public class SSTableMetadataTest extends SchemaLoader
     @Test
     public void testWithDeletes() throws ExecutionException, InterruptedException
     {
-        Keyspace keyspace = Keyspace.open("Keyspace1");
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
         ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard2");
         long timestamp = System.currentTimeMillis();
         DecoratedKey key = Util.dk("deletetest");
-        Mutation rm = new Mutation("Keyspace1", key.getKey());
+        Mutation rm = new Mutation(KEYSPACE1, key.getKey());
         for (int i = 0; i<5; i++)
             rm.add("Standard2", cellname("deletecolumn" + i),
                        ByteBufferUtil.EMPTY_BYTE_BUFFER,
@@ -143,7 +172,7 @@ public class SSTableMetadataTest extends SchemaLoader
             firstMaxDelTime = sstable.getSSTableMetadata().maxLocalDeletionTime;
             assertEquals(ttltimestamp + 1000, firstMaxDelTime, 10);
         }
-        rm = new Mutation("Keyspace1", key.getKey());
+        rm = new Mutation(KEYSPACE1, key.getKey());
         rm.delete("Standard2", cellname("todelete"), timestamp + 1);
         rm.apply();
         store.forceBlockingFlush();
@@ -169,13 +198,13 @@ public class SSTableMetadataTest extends SchemaLoader
     @Test
     public void trackMaxMinColNames() throws CharacterCodingException, ExecutionException, InterruptedException
     {
-        Keyspace keyspace = Keyspace.open("Keyspace1");
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
         ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard3");
         store.getCompactionStrategy();
         for (int j = 0; j < 8; j++)
         {
             DecoratedKey key = Util.dk("row"+j);
-            Mutation rm = new Mutation("Keyspace1", key.getKey());
+            Mutation rm = new Mutation(KEYSPACE1, key.getKey());
             for (int i = 100; i<150; i++)
             {
                 rm.add("Standard3", cellname(j + "col" + i), ByteBufferUtil.EMPTY_BYTE_BUFFER, System.currentTimeMillis());
@@ -190,7 +219,7 @@ public class SSTableMetadataTest extends SchemaLoader
             assertEquals(ByteBufferUtil.string(sstable.getSSTableMetadata().maxColumnNames.get(0)), "7col149");
         }
         DecoratedKey key = Util.dk("row2");
-        Mutation rm = new Mutation("Keyspace1", key.getKey());
+        Mutation rm = new Mutation(KEYSPACE1, key.getKey());
         for (int i = 101; i<299; i++)
         {
             rm.add("Standard3", cellname(9 + "col" + i), ByteBufferUtil.EMPTY_BYTE_BUFFER, System.currentTimeMillis());
@@ -221,7 +250,7 @@ public class SSTableMetadataTest extends SchemaLoader
         ---------------------
         meaning max columns are b9 and 9, min is a0 and 0
          */
-        Keyspace keyspace = Keyspace.open("Keyspace1");
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
 
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("StandardComposite2");
 
@@ -230,7 +259,7 @@ public class SSTableMetadataTest extends SchemaLoader
         ByteBuffer key = ByteBufferUtil.bytes("k");
         for (int i = 0; i < 10; i++)
         {
-            Mutation rm = new Mutation("Keyspace1", key);
+            Mutation rm = new Mutation(KEYSPACE1, key);
             CellName colName = type.makeCellName(ByteBufferUtil.bytes("a"+(9-i)), ByteBufferUtil.bytes(i));
             rm.add("StandardComposite2", colName, ByteBufferUtil.EMPTY_BYTE_BUFFER, 0);
             rm.apply();
@@ -240,7 +269,7 @@ public class SSTableMetadataTest extends SchemaLoader
         key = ByteBufferUtil.bytes("k2");
         for (int i = 0; i < 10; i++)
         {
-            Mutation rm = new Mutation("Keyspace1", key);
+            Mutation rm = new Mutation(KEYSPACE1, key);
             CellName colName = type.makeCellName(ByteBufferUtil.bytes("b"+(9-i)), ByteBufferUtil.bytes(i));
             rm.add("StandardComposite2", colName, ByteBufferUtil.EMPTY_BYTE_BUFFER, 0);
             rm.apply();
@@ -260,7 +289,7 @@ public class SSTableMetadataTest extends SchemaLoader
     @Test
     public void testLegacyCounterShardTracking()
     {
-        ColumnFamilyStore cfs = Keyspace.open("Keyspace1").getColumnFamilyStore("Counter1");
+        ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore("Counter1");
 
         // A cell with all shards
         CounterContext.ContextState state = CounterContext.ContextState.allocate(1, 1, 1);


[4/6] Reduce unit test times due to schema loading

Posted by ty...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/db/KeyspaceTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/KeyspaceTest.java b/test/unit/org/apache/cassandra/db/KeyspaceTest.java
index 12e6d9f..a71b0dd 100644
--- a/test/unit/org/apache/cassandra/db/KeyspaceTest.java
+++ b/test/unit/org/apache/cassandra/db/KeyspaceTest.java
@@ -27,14 +27,23 @@ import java.io.IOException;
 
 import com.google.common.collect.Iterables;
 import org.apache.commons.lang3.StringUtils;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import static org.junit.Assert.*;
 import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.db.composites.*;
 import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.db.filter.QueryFilter;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.BytesType;
+import org.apache.cassandra.db.marshal.CompositeType;
+import org.apache.cassandra.db.marshal.IntegerType;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.utils.WrappedRunnable;
 import static org.apache.cassandra.Util.column;
 import static org.apache.cassandra.Util.expiringColumn;
@@ -44,11 +53,38 @@ import org.apache.cassandra.io.sstable.SSTableReader;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 
-public class KeyspaceTest extends SchemaLoader
+public class KeyspaceTest
 {
     private static final DecoratedKey TEST_KEY = Util.dk("key1");
     private static final DecoratedKey TEST_SLICE_KEY = Util.dk("key1-slicerange");
 
+    private static final String KEYSPACE1 = "Keyspace1";
+    private static final String CF_STANDARD1 = "Standard1";
+    private static final String CF_STANDARD2 = "Standard2";
+    private static final String CF_STANDARDLONG = "StandardLong1";
+    private static final String CF_STANDARDCOMPOSITE2 = "StandardComposite2";
+
+    private static final String KEYSPACE2 = "Keyspace2";
+    private static final String CF_STANDARD3 = "Standard3";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        AbstractType<?> compositeMaxMin = CompositeType.getInstance(Arrays.asList(new AbstractType<?>[]{BytesType.instance, IntegerType.instance}));
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD2),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARDLONG),
+                                    CFMetaData.denseCFMetaData(KEYSPACE1, CF_STANDARDCOMPOSITE2, compositeMaxMin));
+        SchemaLoader.createKeyspace(KEYSPACE2,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE2, CF_STANDARD3));
+    }
+
     public static void reTest(ColumnFamilyStore cfs, Runnable verify) throws Exception
     {
         verify.run();
@@ -59,12 +95,12 @@ public class KeyspaceTest extends SchemaLoader
     @Test
     public void testGetRowNoColumns() throws Throwable
     {
-        final Keyspace keyspace = Keyspace.open("Keyspace2");
+        final Keyspace keyspace = Keyspace.open(KEYSPACE2);
         final ColumnFamilyStore cfStore = keyspace.getColumnFamilyStore("Standard3");
 
-        ColumnFamily cf = ArrayBackedSortedColumns.factory.create("Keyspace2", "Standard3");
+        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(KEYSPACE2, "Standard3");
         cf.addColumn(column("col1","val1", 1L));
-        Mutation rm = new Mutation("Keyspace2", TEST_KEY.getKey(), cf);
+        Mutation rm = new Mutation(KEYSPACE2, TEST_KEY.getKey(), cf);
         rm.apply();
 
         Runnable verify = new WrappedRunnable()
@@ -89,14 +125,14 @@ public class KeyspaceTest extends SchemaLoader
     @Test
     public void testGetRowSingleColumn() throws Throwable
     {
-        final Keyspace keyspace = Keyspace.open("Keyspace1");
+        final Keyspace keyspace = Keyspace.open(KEYSPACE1);
         final ColumnFamilyStore cfStore = keyspace.getColumnFamilyStore("Standard1");
 
-        ColumnFamily cf = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Standard1");
         cf.addColumn(column("col1","val1", 1L));
         cf.addColumn(column("col2","val2", 1L));
         cf.addColumn(column("col3","val3", 1L));
-        Mutation rm = new Mutation("Keyspace1", TEST_KEY.getKey(), cf);
+        Mutation rm = new Mutation(KEYSPACE1, TEST_KEY.getKey(), cf);
         rm.apply();
 
         Runnable verify = new WrappedRunnable()
@@ -118,15 +154,15 @@ public class KeyspaceTest extends SchemaLoader
     @Test
     public void testGetRowSliceByRange() throws Throwable
     {
-    	DecoratedKey key = TEST_SLICE_KEY;
-    	Keyspace keyspace = Keyspace.open("Keyspace1");
+        DecoratedKey key = TEST_SLICE_KEY;
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
         ColumnFamilyStore cfStore = keyspace.getColumnFamilyStore("Standard1");
-        ColumnFamily cf = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Standard1");
         // First write "a", "b", "c"
         cf.addColumn(column("a", "val1", 1L));
         cf.addColumn(column("b", "val2", 1L));
         cf.addColumn(column("c", "val3", 1L));
-        Mutation rm = new Mutation("Keyspace1", key.getKey(), cf);
+        Mutation rm = new Mutation(KEYSPACE1, key.getKey(), cf);
         rm.apply();
 
         cf = cfStore.getColumnFamily(key, cellname("b"), cellname("c"), false, 100, System.currentTimeMillis());
@@ -142,10 +178,10 @@ public class KeyspaceTest extends SchemaLoader
     @Test
     public void testGetSliceNoMatch() throws Throwable
     {
-        Keyspace keyspace = Keyspace.open("Keyspace1");
-        ColumnFamily cf = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard2");
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
+        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Standard2");
         cf.addColumn(column("col1", "val1", 1));
-        Mutation rm = new Mutation("Keyspace1", ByteBufferUtil.bytes("row1000"), cf);
+        Mutation rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes("row1000"), cf);
         rm.apply();
 
         validateGetSliceNoMatch(keyspace);
@@ -162,17 +198,17 @@ public class KeyspaceTest extends SchemaLoader
     public void testGetSliceWithCutoff() throws Throwable
     {
         // tests slicing against data from one row in a memtable and then flushed to an sstable
-        final Keyspace keyspace = Keyspace.open("Keyspace1");
+        final Keyspace keyspace = Keyspace.open(KEYSPACE1);
         final ColumnFamilyStore cfStore = keyspace.getColumnFamilyStore("Standard1");
         final DecoratedKey ROW = Util.dk("row4");
         final NumberFormat fmt = new DecimalFormat("000");
 
-        ColumnFamily cf = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Standard1");
         // at this rate, we're getting 78-79 cos/block, assuming the blocks are set to be about 4k.
         // so if we go to 300, we'll get at least 4 blocks, which is plenty for testing.
         for (int i = 0; i < 300; i++)
             cf.addColumn(column("col" + fmt.format(i), "omg!thisisthevalue!"+i, 1L));
-        Mutation rm = new Mutation("Keyspace1", ROW.getKey(), cf);
+        Mutation rm = new Mutation(KEYSPACE1, ROW.getKey(), cf);
         rm.apply();
 
         Runnable verify = new WrappedRunnable()
@@ -219,15 +255,15 @@ public class KeyspaceTest extends SchemaLoader
     @Test
     public void testReversedWithFlushing()
     {
-        final Keyspace keyspace = Keyspace.open("Keyspace1");
+        final Keyspace keyspace = Keyspace.open(KEYSPACE1);
         final ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("StandardLong1");
         final DecoratedKey ROW = Util.dk("row4");
 
         for (int i = 0; i < 10; i++)
         {
-            ColumnFamily cf = ArrayBackedSortedColumns.factory.create("Keyspace1", "StandardLong1");
+            ColumnFamily cf = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "StandardLong1");
             cf.addColumn(new BufferCell(cellname((long)i), ByteBufferUtil.EMPTY_BYTE_BUFFER, 0));
-            Mutation rm = new Mutation("Keyspace1", ROW.getKey(), cf);
+            Mutation rm = new Mutation(KEYSPACE1, ROW.getKey(), cf);
             rm.apply();
         }
 
@@ -235,9 +271,9 @@ public class KeyspaceTest extends SchemaLoader
 
         for (int i = 10; i < 20; i++)
         {
-            ColumnFamily cf = ArrayBackedSortedColumns.factory.create("Keyspace1", "StandardLong1");
+            ColumnFamily cf = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "StandardLong1");
             cf.addColumn(new BufferCell(cellname((long)i), ByteBufferUtil.EMPTY_BYTE_BUFFER, 0));
-            Mutation rm = new Mutation("Keyspace1", ROW.getKey(), cf);
+            Mutation rm = new Mutation(KEYSPACE1, ROW.getKey(), cf);
             rm.apply();
 
             cf = cfs.getColumnFamily(ROW, Composites.EMPTY, Composites.EMPTY, true, 1, System.currentTimeMillis());
@@ -264,21 +300,21 @@ public class KeyspaceTest extends SchemaLoader
     public void testGetSliceFromBasic() throws Throwable
     {
         // tests slicing against data from one row in a memtable and then flushed to an sstable
-        final Keyspace keyspace = Keyspace.open("Keyspace1");
+        final Keyspace keyspace = Keyspace.open(KEYSPACE1);
         final ColumnFamilyStore cfStore = keyspace.getColumnFamilyStore("Standard1");
         final DecoratedKey ROW = Util.dk("row1");
 
-        ColumnFamily cf = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Standard1");
         cf.addColumn(column("col1", "val1", 1L));
         cf.addColumn(column("col3", "val3", 1L));
         cf.addColumn(column("col4", "val4", 1L));
         cf.addColumn(column("col5", "val5", 1L));
         cf.addColumn(column("col7", "val7", 1L));
         cf.addColumn(column("col9", "val9", 1L));
-        Mutation rm = new Mutation("Keyspace1", ROW.getKey(), cf);
+        Mutation rm = new Mutation(KEYSPACE1, ROW.getKey(), cf);
         rm.apply();
 
-        rm = new Mutation("Keyspace1", ROW.getKey());
+        rm = new Mutation(KEYSPACE1, ROW.getKey());
         rm.delete("Standard1", cellname("col4"), 2L);
         rm.apply();
 
@@ -319,15 +355,15 @@ public class KeyspaceTest extends SchemaLoader
     public void testGetSliceWithExpiration() throws Throwable
     {
         // tests slicing against data from one row with expiring column in a memtable and then flushed to an sstable
-        final Keyspace keyspace = Keyspace.open("Keyspace1");
+        final Keyspace keyspace = Keyspace.open(KEYSPACE1);
         final ColumnFamilyStore cfStore = keyspace.getColumnFamilyStore("Standard1");
         final DecoratedKey ROW = Util.dk("row5");
 
-        ColumnFamily cf = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Standard1");
         cf.addColumn(column("col1", "val1", 1L));
         cf.addColumn(expiringColumn("col2", "val2", 1L, 60)); // long enough not to be tombstoned
         cf.addColumn(column("col3", "val3", 1L));
-        Mutation rm = new Mutation("Keyspace1", ROW.getKey(), cf);
+        Mutation rm = new Mutation(KEYSPACE1, ROW.getKey(), cf);
         rm.apply();
 
         Runnable verify = new WrappedRunnable()
@@ -353,26 +389,26 @@ public class KeyspaceTest extends SchemaLoader
     public void testGetSliceFromAdvanced() throws Throwable
     {
         // tests slicing against data from one row spread across two sstables
-        final Keyspace keyspace = Keyspace.open("Keyspace1");
+        final Keyspace keyspace = Keyspace.open(KEYSPACE1);
         final ColumnFamilyStore cfStore = keyspace.getColumnFamilyStore("Standard1");
         final DecoratedKey ROW = Util.dk("row2");
 
-        ColumnFamily cf = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Standard1");
         cf.addColumn(column("col1", "val1", 1L));
         cf.addColumn(column("col2", "val2", 1L));
         cf.addColumn(column("col3", "val3", 1L));
         cf.addColumn(column("col4", "val4", 1L));
         cf.addColumn(column("col5", "val5", 1L));
         cf.addColumn(column("col6", "val6", 1L));
-        Mutation rm = new Mutation("Keyspace1", ROW.getKey(), cf);
+        Mutation rm = new Mutation(KEYSPACE1, ROW.getKey(), cf);
         rm.apply();
         cfStore.forceBlockingFlush();
 
-        cf = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        cf = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Standard1");
         cf.addColumn(column("col1", "valx", 2L));
         cf.addColumn(column("col2", "valx", 2L));
         cf.addColumn(column("col3", "valx", 2L));
-        rm = new Mutation("Keyspace1", ROW.getKey(), cf);
+        rm = new Mutation(KEYSPACE1, ROW.getKey(), cf);
         rm.apply();
 
         Runnable verify = new WrappedRunnable()
@@ -402,13 +438,13 @@ public class KeyspaceTest extends SchemaLoader
     public void testGetSliceFromLarge() throws Throwable
     {
         // tests slicing against 1000 columns in an sstable
-        Keyspace keyspace = Keyspace.open("Keyspace1");
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
         ColumnFamilyStore cfStore = keyspace.getColumnFamilyStore("Standard1");
         DecoratedKey key = Util.dk("row3");
-        ColumnFamily cf = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Standard1");
         for (int i = 1000; i < 2000; i++)
             cf.addColumn(column("col" + i, ("v" + i), 1L));
-        Mutation rm = new Mutation("Keyspace1", key.getKey(), cf);
+        Mutation rm = new Mutation(KEYSPACE1, key.getKey(), cf);
         rm.apply();
         cfStore.forceBlockingFlush();
 
@@ -430,18 +466,18 @@ public class KeyspaceTest extends SchemaLoader
     @Test
     public void testLimitSSTables() throws CharacterCodingException
     {
-        Keyspace keyspace = Keyspace.open("Keyspace1");
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
         ColumnFamilyStore cfStore = keyspace.getColumnFamilyStore("Standard1");
         cfStore.disableAutoCompaction();
         DecoratedKey key = Util.dk("row_maxmin");
         for (int j = 0; j < 10; j++)
         {
-            ColumnFamily cf = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+            ColumnFamily cf = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Standard1");
             for (int i = 1000 + (j*100); i < 1000 + ((j+1)*100); i++)
             {
                 cf.addColumn(column("col" + i, ("v" + i), i));
             }
-            Mutation rm = new Mutation("Keyspace1", key.getKey(), cf);
+            Mutation rm = new Mutation(KEYSPACE1, key.getKey(), cf);
             rm.apply();
             cfStore.forceBlockingFlush();
         }
@@ -494,7 +530,7 @@ public class KeyspaceTest extends SchemaLoader
         ---------------------
         then we slice out col1 = a5 and col2 > 85 -> which should let us just check 2 sstables and get 2 columns
          */
-        Keyspace keyspace = Keyspace.open("Keyspace1");
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
 
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("StandardComposite2");
         cfs.disableAutoCompaction();
@@ -505,7 +541,7 @@ public class KeyspaceTest extends SchemaLoader
         {
             for (int i = 0; i < 10; i++)
             {
-                Mutation rm = new Mutation("Keyspace1", key.getKey());
+                Mutation rm = new Mutation(KEYSPACE1, key.getKey());
                 CellName colName = type.makeCellName(ByteBufferUtil.bytes("a" + i), ByteBufferUtil.bytes(j*10 + i));
                 rm.add("StandardComposite2", colName, ByteBufferUtil.EMPTY_BYTE_BUFFER, 0);
                 rm.apply();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/db/MultitableTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/MultitableTest.java b/test/unit/org/apache/cassandra/db/MultitableTest.java
index cc11163..536ba1c 100644
--- a/test/unit/org/apache/cassandra/db/MultitableTest.java
+++ b/test/unit/org/apache/cassandra/db/MultitableTest.java
@@ -20,33 +20,55 @@ package org.apache.cassandra.db;
  *
  */
 
-import org.apache.cassandra.Util;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import static org.apache.cassandra.db.KeyspaceTest.assertColumns;
 import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.Util;
 import static org.apache.cassandra.Util.column;
+import org.apache.cassandra.config.KSMetaData;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.locator.SimpleStrategy;
 
-public class MultitableTest extends SchemaLoader
+public class MultitableTest
 {
+    private static final String KEYSPACE1 = "MultitableTest1";
+    private static final String KEYSPACE2 = "MultitableTest2";
+    private static final String CF1 = "Standard1";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF1));
+        SchemaLoader.createKeyspace(KEYSPACE2,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE2, CF1));
+    }
+
     @Test
     public void testSameCFs()
     {
-        Keyspace keyspace1 = Keyspace.open("Keyspace1");
-        Keyspace keyspace2 = Keyspace.open("Keyspace2");
+        Keyspace keyspace1 = Keyspace.open(KEYSPACE1);
+        Keyspace keyspace2 = Keyspace.open(KEYSPACE2);
 
         Mutation rm;
         DecoratedKey dk = Util.dk("keymulti");
         ColumnFamily cf;
 
-        cf = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        cf = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Standard1");
         cf.addColumn(column("col1", "val1", 1L));
-        rm = new Mutation("Keyspace1", dk.getKey(), cf);
+        rm = new Mutation(KEYSPACE1, dk.getKey(), cf);
         rm.apply();
 
-        cf = ArrayBackedSortedColumns.factory.create("Keyspace2", "Standard1");
+        cf = ArrayBackedSortedColumns.factory.create(KEYSPACE2, "Standard1");
         cf.addColumn(column("col2", "val2", 1L));
-        rm = new Mutation("Keyspace2", dk.getKey(), cf);
+        rm = new Mutation(KEYSPACE2, dk.getKey(), cf);
         rm.apply();
 
         keyspace1.getColumnFamilyStore("Standard1").forceBlockingFlush();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/db/NameSortTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/NameSortTest.java b/test/unit/org/apache/cassandra/db/NameSortTest.java
index 6bd71c7..c4361d8 100644
--- a/test/unit/org/apache/cassandra/db/NameSortTest.java
+++ b/test/unit/org/apache/cassandra/db/NameSortTest.java
@@ -26,12 +26,32 @@ import java.util.Collection;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
+import org.apache.cassandra.config.KSMetaData;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
+import org.junit.BeforeClass;
 import org.junit.Test;
 
-public class NameSortTest extends SchemaLoader
+public class NameSortTest
 {
+    private static final String KEYSPACE1 = "NameSortTest";
+    private static final String CF = "Standard1";
+    private static final String CFSUPER = "Super1";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF),
+                                    SchemaLoader.superCFMD(KEYSPACE1, CFSUPER, LongType.instance));
+    }
+
     @Test
     public void testNameSort1() throws IOException
     {
@@ -55,7 +75,7 @@ public class NameSortTest extends SchemaLoader
 
     private void testNameSort(int N) throws IOException
     {
-        Keyspace keyspace = Keyspace.open("Keyspace1");
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
 
         for (int i = 0; i < N; ++i)
         {
@@ -66,7 +86,7 @@ public class NameSortTest extends SchemaLoader
             for (int j = 0; j < 8; ++j)
             {
                 ByteBuffer bytes = j % 2 == 0 ? ByteBufferUtil.bytes("a") : ByteBufferUtil.bytes("b");
-                rm = new Mutation("Keyspace1", key);
+                rm = new Mutation(KEYSPACE1, key);
                 rm.add("Standard1", Util.cellname("Cell-" + j), bytes, j);
                 rm.applyUnsafe();
             }
@@ -74,11 +94,11 @@ public class NameSortTest extends SchemaLoader
             // super
             for (int j = 0; j < 8; ++j)
             {
-                rm = new Mutation("Keyspace1", key);
+                rm = new Mutation(KEYSPACE1, key);
                 for (int k = 0; k < 4; ++k)
                 {
                     String value = (j + k) % 2 == 0 ? "a" : "b";
-                    addMutation(rm, "Super1", "SuperColumn-" + j, k, value, k);
+                    addMutation(rm, CFSUPER, "SuperColumn-" + j, k, value, k);
                 }
                 rm.applyUnsafe();
             }
@@ -87,7 +107,7 @@ public class NameSortTest extends SchemaLoader
         validateNameSort(keyspace, N);
 
         keyspace.getColumnFamilyStore("Standard1").forceBlockingFlush();
-        keyspace.getColumnFamilyStore("Super1").forceBlockingFlush();
+        keyspace.getColumnFamilyStore(CFSUPER).forceBlockingFlush();
         validateNameSort(keyspace, N);
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java b/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java
index 60ccf9f..a7a961d 100644
--- a/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java
+++ b/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java
@@ -28,12 +28,11 @@ import java.util.TreeSet;
 
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Iterators;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.IndexType;
+import org.apache.cassandra.config.*;
 import org.apache.cassandra.db.columniterator.OnDiskAtomIterator;
 import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy;
@@ -48,8 +47,10 @@ import org.apache.cassandra.db.index.PerColumnSecondaryIndex;
 import org.apache.cassandra.db.index.SecondaryIndex;
 import org.apache.cassandra.db.index.SecondaryIndexSearcher;
 import org.apache.cassandra.db.marshal.Int32Type;
+import org.apache.cassandra.db.marshal.IntegerType;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.sstable.SSTableReader;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.concurrent.OpOrder;
 import org.apache.cassandra.utils.memory.MemtableAllocator;
@@ -59,11 +60,21 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
-public class RangeTombstoneTest extends SchemaLoader
+public class RangeTombstoneTest
 {
-    private static final String KSNAME = "Keyspace1";
+    private static final String KSNAME = "RangeTombstoneTest";
     private static final String CFNAME = "StandardInteger1";
 
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KSNAME,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    CFMetaData.denseCFMetaData(KSNAME, CFNAME, IntegerType.instance));
+    }
+
     @Test
     public void simpleQueryWithRangeTombstoneTest() throws Exception
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/db/ReadMessageTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/ReadMessageTest.java b/test/unit/org/apache/cassandra/db/ReadMessageTest.java
index 6d19d3a..d32df49 100644
--- a/test/unit/org/apache/cassandra/db/ReadMessageTest.java
+++ b/test/unit/org/apache/cassandra/db/ReadMessageTest.java
@@ -25,26 +25,50 @@ import java.nio.ByteBuffer;
 import java.util.SortedSet;
 import java.util.TreeSet;
 
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
 import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.db.composites.*;
 import org.apache.cassandra.db.filter.NamesQueryFilter;
 import org.apache.cassandra.db.filter.SliceQueryFilter;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.util.DataOutputBuffer;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 
 
-public class ReadMessageTest extends SchemaLoader
+public class ReadMessageTest
 {
+    private static final String KEYSPACE1 = "ReadMessageTest1";
+    private static final String KEYSPACENOCOMMIT = "ReadMessageTest_NoCommit";
+    private static final String CF = "Standard1";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF));
+        SchemaLoader.createKeyspace(KEYSPACENOCOMMIT,
+                                    false,
+                                    true,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACENOCOMMIT, CF));
+    }
+
     @Test
     public void testMakeReadMessage() throws IOException
     {
-        CellNameType type = Keyspace.open("Keyspace1").getColumnFamilyStore("Standard1").getComparator();
+        CellNameType type = Keyspace.open(KEYSPACE1).getColumnFamilyStore("Standard1").getComparator();
 
         SortedSet<CellName> colList = new TreeSet<CellName>(type);
         colList.add(Util.cellname("col1"));
@@ -54,15 +78,15 @@ public class ReadMessageTest extends SchemaLoader
         DecoratedKey dk = Util.dk("row1");
         long ts = System.currentTimeMillis();
 
-        rm = new SliceByNamesReadCommand("Keyspace1", dk.getKey(), "Standard1", ts, new NamesQueryFilter(colList));
+        rm = new SliceByNamesReadCommand(KEYSPACE1, dk.getKey(), "Standard1", ts, new NamesQueryFilter(colList));
         rm2 = serializeAndDeserializeReadMessage(rm);
         assert rm2.toString().equals(rm.toString());
 
-        rm = new SliceFromReadCommand("Keyspace1", dk.getKey(), "Standard1", ts, new SliceQueryFilter(Composites.EMPTY, Composites.EMPTY, true, 2));
+        rm = new SliceFromReadCommand(KEYSPACE1, dk.getKey(), "Standard1", ts, new SliceQueryFilter(Composites.EMPTY, Composites.EMPTY, true, 2));
         rm2 = serializeAndDeserializeReadMessage(rm);
         assert rm2.toString().equals(rm.toString());
 
-        rm = new SliceFromReadCommand("Keyspace1", dk.getKey(), "Standard1", ts, new SliceQueryFilter(Util.cellname("a"), Util.cellname("z"), true, 5));
+        rm = new SliceFromReadCommand(KEYSPACE1, dk.getKey(), "Standard1", ts, new SliceQueryFilter(Util.cellname("a"), Util.cellname("z"), true, 5));
         rm2 = serializeAndDeserializeReadMessage(rm);
         assert rm2.toString().equals(rm.toString());
     }
@@ -81,17 +105,17 @@ public class ReadMessageTest extends SchemaLoader
     @Test
     public void testGetColumn()
     {
-        Keyspace keyspace = Keyspace.open("Keyspace1");
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
         CellNameType type = keyspace.getColumnFamilyStore("Standard1").getComparator();
         Mutation rm;
         DecoratedKey dk = Util.dk("key1");
 
         // add data
-        rm = new Mutation("Keyspace1", dk.getKey());
+        rm = new Mutation(KEYSPACE1, dk.getKey());
         rm.add("Standard1", Util.cellname("Column1"), ByteBufferUtil.bytes("abcd"), 0);
         rm.apply();
 
-        ReadCommand command = new SliceByNamesReadCommand("Keyspace1", dk.getKey(), "Standard1", System.currentTimeMillis(), new NamesQueryFilter(FBUtilities.singleton(Util.cellname("Column1"), type)));
+        ReadCommand command = new SliceByNamesReadCommand(KEYSPACE1, dk.getKey(), "Standard1", System.currentTimeMillis(), new NamesQueryFilter(FBUtilities.singleton(Util.cellname("Column1"), type)));
         Row row = command.getRow(keyspace);
         Cell col = row.cf.getColumn(Util.cellname("Column1"));
         assertEquals(col.value(), ByteBuffer.wrap("abcd".getBytes()));
@@ -100,11 +124,11 @@ public class ReadMessageTest extends SchemaLoader
     @Test
     public void testNoCommitLog() throws Exception
     {
-        Mutation rm = new Mutation("Keyspace1", ByteBufferUtil.bytes("row"));
+        Mutation rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes("row"));
         rm.add("Standard1", Util.cellname("commit1"), ByteBufferUtil.bytes("abcd"), 0);
         rm.apply();
 
-        rm = new Mutation("NoCommitlogSpace", ByteBufferUtil.bytes("row"));
+        rm = new Mutation(KEYSPACENOCOMMIT, ByteBufferUtil.bytes("row"));
         rm.add("Standard1", Util.cellname("commit2"), ByteBufferUtil.bytes("abcd"), 0);
         rm.apply();
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/db/RecoveryManager2Test.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RecoveryManager2Test.java b/test/unit/org/apache/cassandra/db/RecoveryManager2Test.java
index ede3e9b..03d358f 100644
--- a/test/unit/org/apache/cassandra/db/RecoveryManager2Test.java
+++ b/test/unit/org/apache/cassandra/db/RecoveryManager2Test.java
@@ -21,6 +21,7 @@ package org.apache.cassandra.db;
  */
 
 
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.slf4j.Logger;
@@ -28,15 +29,33 @@ import org.slf4j.LoggerFactory;
 
 import static org.apache.cassandra.Util.column;
 import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.db.commitlog.CommitLog;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 
-public class RecoveryManager2Test extends SchemaLoader
+public class RecoveryManager2Test
 {
     private static Logger logger = LoggerFactory.getLogger(RecoveryManager2Test.class);
 
+    private static final String KEYSPACE1 = "RecoveryManager2Test";
+    private static final String CF_STANDARD1 = "Standard1";
+    private static final String CF_STANDARD2 = "Standard2";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD2));
+    }
+
     @Test
     /* test that commit logs do not replay flushed data */
     public void testWithFlush() throws Exception
@@ -55,7 +74,7 @@ public class RecoveryManager2Test extends SchemaLoader
             insertRow("Standard1", key);
         }
 
-        Keyspace keyspace1 = Keyspace.open("Keyspace1");
+        Keyspace keyspace1 = Keyspace.open(KEYSPACE1);
         ColumnFamilyStore cfs = keyspace1.getColumnFamilyStore("Standard1");
         logger.debug("forcing flush");
         cfs.forceBlockingFlush();
@@ -70,9 +89,9 @@ public class RecoveryManager2Test extends SchemaLoader
 
     private void insertRow(String cfname, String key) 
     {
-        ColumnFamily cf = ArrayBackedSortedColumns.factory.create("Keyspace1", cfname);
+        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(KEYSPACE1, cfname);
         cf.addColumn(column("col1", "val1", 1L));
-        Mutation rm = new Mutation("Keyspace1", ByteBufferUtil.bytes(key), cf);
+        Mutation rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes(key), cf);
         rm.apply();
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/db/RecoveryManager3Test.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RecoveryManager3Test.java b/test/unit/org/apache/cassandra/db/RecoveryManager3Test.java
index c9bc86a..13d649d 100644
--- a/test/unit/org/apache/cassandra/db/RecoveryManager3Test.java
+++ b/test/unit/org/apache/cassandra/db/RecoveryManager3Test.java
@@ -24,37 +24,61 @@ package org.apache.cassandra.db;
 import java.io.File;
 import java.io.IOException;
 
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
 import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.db.commitlog.CommitLog;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.locator.SimpleStrategy;
 
 import static org.apache.cassandra.Util.column;
 import static org.apache.cassandra.db.KeyspaceTest.assertColumns;
 
-public class RecoveryManager3Test extends SchemaLoader
+public class RecoveryManager3Test
 {
+    private static final String KEYSPACE1 = "RecoveryManager3Test1";
+    private static final String CF_STANDARD1 = "Standard1";
+
+    private static final String KEYSPACE2 = "RecoveryManager3Test2";
+    private static final String CF_STANDARD3 = "Standard3";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1));
+        SchemaLoader.createKeyspace(KEYSPACE2,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE2, CF_STANDARD3));
+    }
+
     @Test
     public void testMissingHeader() throws IOException
     {
-        Keyspace keyspace1 = Keyspace.open("Keyspace1");
-        Keyspace keyspace2 = Keyspace.open("Keyspace2");
+        Keyspace keyspace1 = Keyspace.open(KEYSPACE1);
+        Keyspace keyspace2 = Keyspace.open(KEYSPACE2);
 
         Mutation rm;
         DecoratedKey dk = Util.dk("keymulti");
         ColumnFamily cf;
 
-        cf = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        cf = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Standard1");
         cf.addColumn(column("col1", "val1", 1L));
-        rm = new Mutation("Keyspace1", dk.getKey(), cf);
+        rm = new Mutation(KEYSPACE1, dk.getKey(), cf);
         rm.apply();
 
-        cf = ArrayBackedSortedColumns.factory.create("Keyspace2", "Standard3");
+        cf = ArrayBackedSortedColumns.factory.create(KEYSPACE2, "Standard3");
         cf.addColumn(column("col2", "val2", 1L));
-        rm = new Mutation("Keyspace2", dk.getKey(), cf);
+        rm = new Mutation(KEYSPACE2, dk.getKey(), cf);
         rm.apply();
 
         keyspace1.getColumnFamilyStore("Standard1").clearUnsafe();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/db/RecoveryManagerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RecoveryManagerTest.java b/test/unit/org/apache/cassandra/db/RecoveryManagerTest.java
index 687df5a..5d4928c 100644
--- a/test/unit/org/apache/cassandra/db/RecoveryManagerTest.java
+++ b/test/unit/org/apache/cassandra/db/RecoveryManagerTest.java
@@ -24,7 +24,14 @@ import java.util.concurrent.TimeUnit;
 
 import org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.Util;
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.KSMetaData;
+import org.apache.cassandra.db.marshal.BytesType;
+import org.apache.cassandra.db.marshal.CounterColumnType;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.junit.Assert;
+import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 
@@ -37,8 +44,30 @@ import static org.apache.cassandra.db.KeyspaceTest.assertColumns;
 import static org.apache.cassandra.Util.cellname;
 
 @RunWith(OrderedJUnit4ClassRunner.class)
-public class RecoveryManagerTest extends SchemaLoader
+public class RecoveryManagerTest
 {
+    private static final String KEYSPACE1 = "RecoveryManagerTest1";
+    private static final String CF_STANDARD1 = "Standard1";
+    private static final String CF_COUNTER1 = "Counter1";
+
+    private static final String KEYSPACE2 = "RecoveryManagerTest2";
+    private static final String CF_STANDARD3 = "Standard3";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1),
+                                    CFMetaData.denseCFMetaData(KEYSPACE1, CF_COUNTER1, BytesType.instance).defaultValidator(CounterColumnType.instance));
+        SchemaLoader.createKeyspace(KEYSPACE2,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE2, CF_STANDARD3));
+    }
+
     @Test
     public void testNothingToRecover() throws IOException {
         CommitLog.instance.recover();
@@ -47,21 +76,21 @@ public class RecoveryManagerTest extends SchemaLoader
     @Test
     public void testOne() throws IOException
     {
-        Keyspace keyspace1 = Keyspace.open("Keyspace1");
-        Keyspace keyspace2 = Keyspace.open("Keyspace2");
+        Keyspace keyspace1 = Keyspace.open(KEYSPACE1);
+        Keyspace keyspace2 = Keyspace.open(KEYSPACE2);
 
         Mutation rm;
         DecoratedKey dk = Util.dk("keymulti");
         ColumnFamily cf;
 
-        cf = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        cf = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Standard1");
         cf.addColumn(column("col1", "val1", 1L));
-        rm = new Mutation("Keyspace1", dk.getKey(), cf);
+        rm = new Mutation(KEYSPACE1, dk.getKey(), cf);
         rm.apply();
 
-        cf = ArrayBackedSortedColumns.factory.create("Keyspace2", "Standard3");
+        cf = ArrayBackedSortedColumns.factory.create(KEYSPACE2, "Standard3");
         cf.addColumn(column("col2", "val2", 1L));
-        rm = new Mutation("Keyspace2", dk.getKey(), cf);
+        rm = new Mutation(KEYSPACE2, dk.getKey(), cf);
         rm.apply();
 
         keyspace1.getColumnFamilyStore("Standard1").clearUnsafe();
@@ -77,7 +106,7 @@ public class RecoveryManagerTest extends SchemaLoader
     @Test
     public void testRecoverCounter() throws IOException
     {
-        Keyspace keyspace1 = Keyspace.open("Keyspace1");
+        Keyspace keyspace1 = Keyspace.open(KEYSPACE1);
 
         Mutation rm;
         DecoratedKey dk = Util.dk("key");
@@ -85,9 +114,9 @@ public class RecoveryManagerTest extends SchemaLoader
 
         for (int i = 0; i < 10; ++i)
         {
-            cf = ArrayBackedSortedColumns.factory.create("Keyspace1", "Counter1");
+            cf = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Counter1");
             cf.addColumn(BufferCounterCell.createLocal(cellname("col"), 1L, 1L, Long.MIN_VALUE));
-            rm = new Mutation("Keyspace1", dk.getKey(), cf);
+            rm = new Mutation(KEYSPACE1, dk.getKey(), cf);
             rm.apply();
         }
 
@@ -111,14 +140,14 @@ public class RecoveryManagerTest extends SchemaLoader
         Date date = CommitLogArchiver.format.parse("2112:12:12 12:12:12");
         long timeMS = date.getTime() - 5000;
 
-        Keyspace keyspace1 = Keyspace.open("Keyspace1");
+        Keyspace keyspace1 = Keyspace.open(KEYSPACE1);
         DecoratedKey dk = Util.dk("dkey");
         for (int i = 0; i < 10; ++i)
         {
             long ts = TimeUnit.MILLISECONDS.toMicros(timeMS + (i * 1000));
-            ColumnFamily cf = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+            ColumnFamily cf = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Standard1");
             cf.addColumn(column("name-" + i, "value", ts));
-            Mutation rm = new Mutation("Keyspace1", dk.getKey(), cf);
+            Mutation rm = new Mutation(KEYSPACE1, dk.getKey(), cf);
             rm.apply();
         }
         keyspace1.getColumnFamilyStore("Standard1").clearUnsafe();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/db/RecoveryManagerTruncateTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RecoveryManagerTruncateTest.java b/test/unit/org/apache/cassandra/db/RecoveryManagerTruncateTest.java
index 1f7d388..23caaf8 100644
--- a/test/unit/org/apache/cassandra/db/RecoveryManagerTruncateTest.java
+++ b/test/unit/org/apache/cassandra/db/RecoveryManagerTruncateTest.java
@@ -26,28 +26,45 @@ import java.io.IOException;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.db.commitlog.CommitLog;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.locator.SimpleStrategy;
+import org.junit.BeforeClass;
 import org.junit.Test;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 /**
  * Test for the truncate operation.
  */
-public class RecoveryManagerTruncateTest extends SchemaLoader
+public class RecoveryManagerTruncateTest
 {
+    private static final String KEYSPACE1 = "RecoveryManagerTruncateTest";
+    private static final String CF_STANDARD1 = "Standard1";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1));
+    }
+
 	@Test
 	public void testTruncate() throws IOException
 	{
-		Keyspace keyspace = Keyspace.open("Keyspace1");
+		Keyspace keyspace = Keyspace.open(KEYSPACE1);
 		ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("Standard1");
 
 		Mutation rm;
 		ColumnFamily cf;
 
 		// add a single cell
-        cf = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        cf = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Standard1");
 		cf.addColumn(column("col1", "val1", 1L));
-        rm = new Mutation("Keyspace1", ByteBufferUtil.bytes("keymulti"), cf);
+        rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes("keymulti"), cf);
 		rm.apply();
 
 		// Make sure data was written

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/db/RemoveCellTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RemoveCellTest.java b/test/unit/org/apache/cassandra/db/RemoveCellTest.java
index 77ff02d..ac941b5 100644
--- a/test/unit/org/apache/cassandra/db/RemoveCellTest.java
+++ b/test/unit/org/apache/cassandra/db/RemoveCellTest.java
@@ -18,6 +18,7 @@
 */
 package org.apache.cassandra.db;
 
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import static org.junit.Assert.assertFalse;
@@ -25,27 +26,43 @@ import static org.junit.Assert.assertNull;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.db.filter.QueryFilter;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
-public class RemoveCellTest extends SchemaLoader
+public class RemoveCellTest
 {
+    private static final String KEYSPACE1 = "RemoveCellTest";
+    private static final String CF_STANDARD1 = "Standard1";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1));
+    }
+
     @Test
     public void testRemoveColumn()
     {
-        Keyspace keyspace = Keyspace.open("Keyspace1");
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
         ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard1");
         Mutation rm;
         DecoratedKey dk = Util.dk("key1");
 
         // add data
-        rm = new Mutation("Keyspace1", dk.getKey());
+        rm = new Mutation(KEYSPACE1, dk.getKey());
         rm.add("Standard1", Util.cellname("Column1"), ByteBufferUtil.bytes("asdf"), 0);
         rm.apply();
         store.forceBlockingFlush();
 
         // remove
-        rm = new Mutation("Keyspace1", dk.getKey());
+        rm = new Mutation(KEYSPACE1, dk.getKey());
         rm.delete("Standard1", Util.cellname("Column1"), 1);
         rm.apply();
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/db/RemoveColumnFamilyTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RemoveColumnFamilyTest.java b/test/unit/org/apache/cassandra/db/RemoveColumnFamilyTest.java
index 09eed71..c8bab47 100644
--- a/test/unit/org/apache/cassandra/db/RemoveColumnFamilyTest.java
+++ b/test/unit/org/apache/cassandra/db/RemoveColumnFamilyTest.java
@@ -18,6 +18,7 @@
 */
 package org.apache.cassandra.db;
 
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import static org.junit.Assert.assertNull;
@@ -25,26 +26,42 @@ import org.apache.cassandra.db.filter.QueryFilter;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
+import org.apache.cassandra.config.KSMetaData;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 
-public class RemoveColumnFamilyTest extends SchemaLoader
+public class RemoveColumnFamilyTest
 {
+    private static final String KEYSPACE1 = "RemoveColumnFamilyTest";
+    private static final String CF_STANDARD1 = "Standard1";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1));
+    }
+
     @Test
     public void testRemoveColumnFamily()
     {
-        Keyspace keyspace = Keyspace.open("Keyspace1");
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
         ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard1");
         Mutation rm;
         DecoratedKey dk = Util.dk("key1");
 
         // add data
-        rm = new Mutation("Keyspace1", dk.getKey());
+        rm = new Mutation(KEYSPACE1, dk.getKey());
         rm.add("Standard1", Util.cellname("Column1"), ByteBufferUtil.bytes("asdf"), 0);
         rm.apply();
 
         // remove
-        rm = new Mutation("Keyspace1", dk.getKey());
+        rm = new Mutation(KEYSPACE1, dk.getKey());
         rm.delete("Standard1", 1);
         rm.apply();
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush1Test.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush1Test.java b/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush1Test.java
index f898f16..830eba5 100644
--- a/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush1Test.java
+++ b/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush1Test.java
@@ -18,6 +18,7 @@
 */
 package org.apache.cassandra.db;
 
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import static org.junit.Assert.assertNull;
@@ -25,28 +26,44 @@ import org.apache.cassandra.db.filter.QueryFilter;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
+import org.apache.cassandra.config.KSMetaData;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 
-public class RemoveColumnFamilyWithFlush1Test extends SchemaLoader
+public class RemoveColumnFamilyWithFlush1Test
 {
+    private static final String KEYSPACE1 = "RemoveColumnFamilyWithFlush1Test";
+    private static final String CF_STANDARD1 = "Standard1";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1));
+    }
+
     @Test
     public void testRemoveColumnFamilyWithFlush1()
     {
-        Keyspace keyspace = Keyspace.open("Keyspace1");
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
         ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard1");
         Mutation rm;
         DecoratedKey dk = Util.dk("key1");
 
         // add data
-        rm = new Mutation("Keyspace1", dk.getKey());
+        rm = new Mutation(KEYSPACE1, dk.getKey());
         rm.add("Standard1", Util.cellname("Column1"), ByteBufferUtil.bytes("asdf"), 0);
         rm.add("Standard1", Util.cellname("Column2"), ByteBufferUtil.bytes("asdf"), 0);
         rm.apply();
         store.forceBlockingFlush();
 
         // remove
-        rm = new Mutation("Keyspace1", dk.getKey());
+        rm = new Mutation(KEYSPACE1, dk.getKey());
         rm.delete("Standard1", 1);
         rm.apply();
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush2Test.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush2Test.java b/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush2Test.java
index 1e910ad..d2ba51f 100644
--- a/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush2Test.java
+++ b/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush2Test.java
@@ -18,6 +18,7 @@
 */
 package org.apache.cassandra.db;
 
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import static org.junit.Assert.assertNull;
@@ -25,25 +26,41 @@ import org.apache.cassandra.db.filter.QueryFilter;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
+import org.apache.cassandra.config.KSMetaData;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 
-public class RemoveColumnFamilyWithFlush2Test extends SchemaLoader
+public class RemoveColumnFamilyWithFlush2Test
 {
+    private static final String KEYSPACE1 = "RemoveColumnFamilyWithFlush2Test";
+    private static final String CF_STANDARD1 = "Standard1";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1));
+    }
+
     @Test
     public void testRemoveColumnFamilyWithFlush2()
     {
-        Keyspace keyspace = Keyspace.open("Keyspace1");
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
         ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard1");
         Mutation rm;
         DecoratedKey dk = Util.dk("key1");
 
         // add data
-        rm = new Mutation("Keyspace1", dk.getKey());
+        rm = new Mutation(KEYSPACE1, dk.getKey());
         rm.add("Standard1", Util.cellname("Column1"), ByteBufferUtil.bytes("asdf"), 0);
         rm.apply();
         // remove
-        rm = new Mutation("Keyspace1", dk.getKey());
+        rm = new Mutation(KEYSPACE1, dk.getKey());
         rm.delete("Standard1", 1);
         rm.apply();
         store.forceBlockingFlush();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/db/RemoveSubCellTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RemoveSubCellTest.java b/test/unit/org/apache/cassandra/db/RemoveSubCellTest.java
index cec1bce..db27bef 100644
--- a/test/unit/org/apache/cassandra/db/RemoveSubCellTest.java
+++ b/test/unit/org/apache/cassandra/db/RemoveSubCellTest.java
@@ -21,12 +21,17 @@ package org.apache.cassandra.db;
 import java.nio.ByteBuffer;
 import java.util.concurrent.TimeUnit;
 
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNull;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.db.composites.*;
 import org.apache.cassandra.db.filter.QueryFilter;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.locator.SimpleStrategy;
 
 import static org.apache.cassandra.Util.getBytes;
 import org.apache.cassandra.Util;
@@ -36,25 +41,38 @@ import org.apache.cassandra.utils.ByteBufferUtil;
 import com.google.common.util.concurrent.Uninterruptibles;
 
 
-public class RemoveSubCellTest extends SchemaLoader
+public class RemoveSubCellTest
 {
+    private static final String KEYSPACE1 = "RemoveSubCellTest";
+    private static final String CF_SUPER1 = "Super1";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.superCFMD(KEYSPACE1, CF_SUPER1, LongType.instance));
+    }
+
     @Test
     public void testRemoveSubColumn()
     {
-        Keyspace keyspace = Keyspace.open("Keyspace1");
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
         ColumnFamilyStore store = keyspace.getColumnFamilyStore("Super1");
         Mutation rm;
         DecoratedKey dk = Util.dk("key1");
 
         // add data
-        rm = new Mutation("Keyspace1", dk.getKey());
+        rm = new Mutation(KEYSPACE1, dk.getKey());
         Util.addMutation(rm, "Super1", "SC1", 1, "asdf", 0);
         rm.apply();
         store.forceBlockingFlush();
 
         CellName cname = CellNames.compositeDense(ByteBufferUtil.bytes("SC1"), getBytes(1L));
         // remove
-        rm = new Mutation("Keyspace1", dk.getKey());
+        rm = new Mutation(KEYSPACE1, dk.getKey());
         rm.delete("Super1", cname, 1);
         rm.apply();
 
@@ -66,13 +84,13 @@ public class RemoveSubCellTest extends SchemaLoader
     @Test
     public void testRemoveSubColumnAndContainer()
     {
-        Keyspace keyspace = Keyspace.open("Keyspace1");
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
         ColumnFamilyStore store = keyspace.getColumnFamilyStore("Super1");
         Mutation rm;
         DecoratedKey dk = Util.dk("key2");
 
         // add data
-        rm = new Mutation("Keyspace1", dk.getKey());
+        rm = new Mutation(KEYSPACE1, dk.getKey());
         Util.addMutation(rm, "Super1", "SC1", 1, "asdf", 0);
         rm.apply();
         store.forceBlockingFlush();
@@ -80,7 +98,7 @@ public class RemoveSubCellTest extends SchemaLoader
         // remove the SC
         ByteBuffer scName = ByteBufferUtil.bytes("SC1");
         CellName cname = CellNames.compositeDense(scName, getBytes(1L));
-        rm = new Mutation("Keyspace1", dk.getKey());
+        rm = new Mutation(KEYSPACE1, dk.getKey());
         rm.deleteRange("Super1", SuperColumns.startOf(scName), SuperColumns.endOf(scName), 1);
         rm.apply();
 
@@ -90,7 +108,7 @@ public class RemoveSubCellTest extends SchemaLoader
         Uninterruptibles.sleepUninterruptibly(1, TimeUnit.SECONDS);
 
         // remove the column itself
-        rm = new Mutation("Keyspace1", dk.getKey());
+        rm = new Mutation(KEYSPACE1, dk.getKey());
         rm.delete("Super1", cname, 2);
         rm.apply();
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/db/RowCacheTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RowCacheTest.java b/test/unit/org/apache/cassandra/db/RowCacheTest.java
index 7b6ff99..4bed2ad 100644
--- a/test/unit/org/apache/cassandra/db/RowCacheTest.java
+++ b/test/unit/org/apache/cassandra/db/RowCacheTest.java
@@ -23,30 +23,51 @@ import java.nio.ByteBuffer;
 import java.util.Collection;
 
 import org.junit.AfterClass;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
+import org.apache.cassandra.cache.CachingOptions;
 import org.apache.cassandra.cache.RowCacheKey;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.db.composites.*;
 import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.db.filter.QueryFilter;
+import org.apache.cassandra.db.marshal.IntegerType;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.dht.BytesToken;
 import org.apache.cassandra.locator.TokenMetadata;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.service.CacheService;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import static org.junit.Assert.assertEquals;
 
-public class RowCacheTest extends SchemaLoader
+public class RowCacheTest
 {
-    private String KEYSPACE = "RowCacheSpace";
-    private String COLUMN_FAMILY = "CachedCF";
+    private static final String KEYSPACE_CACHED = "RowCacheTest";
+    private static final String CF_CACHED = "CachedCF";
+    private static final String CF_CACHEDINT = "CachedIntCF";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE_CACHED,
+                SimpleStrategy.class,
+                KSMetaData.optsWithRF(1),
+                SchemaLoader.standardCFMD(KEYSPACE_CACHED, CF_CACHED).caching(CachingOptions.ALL),
+                SchemaLoader.standardCFMD(KEYSPACE_CACHED, CF_CACHEDINT)
+                            .defaultValidator(IntegerType.instance)
+                            .caching(new CachingOptions(new CachingOptions.KeyCache(CachingOptions.KeyCache.Type.ALL),
+                                     new CachingOptions.RowCache(CachingOptions.RowCache.Type.HEAD, 100))));
+    }
 
     @AfterClass
     public static void cleanup()
     {
-        cleanupSavedCaches();
+        SchemaLoader.cleanupSavedCaches();
     }
 
     @Test
@@ -54,8 +75,8 @@ public class RowCacheTest extends SchemaLoader
     {
         CompactionManager.instance.disableAutoCompaction();
 
-        Keyspace keyspace = Keyspace.open(KEYSPACE);
-        ColumnFamilyStore cachedStore  = keyspace.getColumnFamilyStore(COLUMN_FAMILY);
+        Keyspace keyspace = Keyspace.open(KEYSPACE_CACHED);
+        ColumnFamilyStore cachedStore  = keyspace.getColumnFamilyStore(CF_CACHED);
 
         // empty the row cache
         CacheService.instance.invalidateRowCache();
@@ -64,7 +85,7 @@ public class RowCacheTest extends SchemaLoader
         CacheService.instance.setRowCacheCapacityInMB(1);
 
         // inserting 100 rows into both column families
-        insertData(KEYSPACE, COLUMN_FAMILY, 0, 100);
+        SchemaLoader.insertData(KEYSPACE_CACHED, CF_CACHED, 0, 100);
 
         // now reading rows one by one and checking if row change grows
         for (int i = 0; i < 100; i++)
@@ -87,7 +108,7 @@ public class RowCacheTest extends SchemaLoader
         }
 
         // insert 10 more keys
-        insertData(KEYSPACE, COLUMN_FAMILY, 100, 10);
+        SchemaLoader.insertData(KEYSPACE_CACHED, CF_CACHED, 100, 10);
 
         for (int i = 100; i < 110; i++)
         {
@@ -134,7 +155,7 @@ public class RowCacheTest extends SchemaLoader
         CacheService.instance.setRowCacheCapacityInMB(1);
         rowCacheLoad(100, Integer.MAX_VALUE, 1000);
 
-        ColumnFamilyStore store = Keyspace.open(KEYSPACE).getColumnFamilyStore(COLUMN_FAMILY);
+        ColumnFamilyStore store = Keyspace.open(KEYSPACE_CACHED).getColumnFamilyStore(CF_CACHED);
         assertEquals(CacheService.instance.rowCache.getKeySet().size(), 100);
         store.cleanupCache();
         assertEquals(CacheService.instance.rowCache.getKeySet().size(), 100);
@@ -161,7 +182,7 @@ public class RowCacheTest extends SchemaLoader
     {
         CompactionManager.instance.disableAutoCompaction();
 
-        Keyspace keyspace = Keyspace.open(KEYSPACE);
+        Keyspace keyspace = Keyspace.open(KEYSPACE_CACHED);
         String cf = "CachedIntCF";
         ColumnFamilyStore cachedStore  = keyspace.getColumnFamilyStore(cf);
         long startRowCacheHits = cachedStore.metric.rowCacheHit.count();
@@ -175,7 +196,7 @@ public class RowCacheTest extends SchemaLoader
         ByteBuffer key = ByteBufferUtil.bytes("rowcachekey");
         DecoratedKey dk = cachedStore.partitioner.decorateKey(key);
         RowCacheKey rck = new RowCacheKey(cachedStore.metadata.cfId, dk);
-        Mutation mutation = new Mutation(KEYSPACE, key);
+        Mutation mutation = new Mutation(KEYSPACE_CACHED, key);
         for (int i = 0; i < 200; i++)
             mutation.add(cf, Util.cellname(i), ByteBufferUtil.bytes("val" + i), System.currentTimeMillis());
         mutation.applyUnsafe();
@@ -234,15 +255,15 @@ public class RowCacheTest extends SchemaLoader
     {
         CompactionManager.instance.disableAutoCompaction();
 
-        ColumnFamilyStore store = Keyspace.open(KEYSPACE).getColumnFamilyStore(COLUMN_FAMILY);
+        ColumnFamilyStore store = Keyspace.open(KEYSPACE_CACHED).getColumnFamilyStore(CF_CACHED);
 
         // empty the cache
         CacheService.instance.invalidateRowCache();
         assert CacheService.instance.rowCache.size() == 0;
 
         // insert data and fill the cache
-        insertData(KEYSPACE, COLUMN_FAMILY, offset, totalKeys);
-        readData(KEYSPACE, COLUMN_FAMILY, offset, totalKeys);
+        SchemaLoader.insertData(KEYSPACE_CACHED, CF_CACHED, offset, totalKeys);
+        SchemaLoader.readData(KEYSPACE_CACHED, CF_CACHED, offset, totalKeys);
         assert CacheService.instance.rowCache.size() == totalKeys;
 
         // force the cache to disk

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/db/RowIterationTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RowIterationTest.java b/test/unit/org/apache/cassandra/db/RowIterationTest.java
index 8464a35..10d8903 100644
--- a/test/unit/org/apache/cassandra/db/RowIterationTest.java
+++ b/test/unit/org/apache/cassandra/db/RowIterationTest.java
@@ -25,20 +25,36 @@ import java.util.HashSet;
 
 import org.apache.cassandra.Util;
 
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.db.composites.*;
+import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.utils.FBUtilities;
 import static org.junit.Assert.assertEquals;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 
-public class RowIterationTest extends SchemaLoader
+public class RowIterationTest
 {
-    public static final String KEYSPACE1 = "Keyspace2";
+    public static final String KEYSPACE1 = "RowIterationTest";
     public static final InetAddress LOCAL = FBUtilities.getBroadcastAddress();
 
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, "Standard3"),
+                                    SchemaLoader.superCFMD(KEYSPACE1, "Super3", LongType.instance));
+    }
+
     @Test
     public void testRowIteration()
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/db/RowTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RowTest.java b/test/unit/org/apache/cassandra/db/RowTest.java
index 22e112e..910f9e1 100644
--- a/test/unit/org/apache/cassandra/db/RowTest.java
+++ b/test/unit/org/apache/cassandra/db/RowTest.java
@@ -22,10 +22,14 @@ import java.util.Arrays;
 import java.util.concurrent.TimeUnit;
 
 import com.google.common.util.concurrent.Uninterruptibles;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.db.composites.CellNames;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 import static org.apache.cassandra.Util.column;
@@ -34,15 +38,28 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
-public class RowTest extends SchemaLoader
+public class RowTest
 {
+    private static final String KEYSPACE1 = "RowTest";
+    private static final String CF_STANDARD1 = "Standard1";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1));
+    }
+
     @Test
     public void testDiffColumnFamily()
     {
-        ColumnFamily cf1 = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        ColumnFamily cf1 = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Standard1");
         cf1.addColumn(column("one", "onev", 0));
 
-        ColumnFamily cf2 = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        ColumnFamily cf2 = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Standard1");
         DeletionInfo delInfo = new DeletionInfo(0, 0);
         cf2.delete(delInfo);
 
@@ -86,10 +103,10 @@ public class RowTest extends SchemaLoader
     @Test
     public void testResolve()
     {
-        ColumnFamily cf1 = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        ColumnFamily cf1 = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Standard1");
         cf1.addColumn(column("one", "A", 0));
 
-        ColumnFamily cf2 = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        ColumnFamily cf2 = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Standard1");
         cf2.addColumn(column("one", "B", 1));
         cf2.addColumn(column("two", "C", 1));
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/db/ScrubTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/ScrubTest.java b/test/unit/org/apache/cassandra/db/ScrubTest.java
index e0bbb64..24a6787 100644
--- a/test/unit/org/apache/cassandra/db/ScrubTest.java
+++ b/test/unit/org/apache/cassandra/db/ScrubTest.java
@@ -30,10 +30,17 @@ import java.util.List;
 import java.util.Set;
 import java.util.concurrent.ExecutionException;
 
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.cql3.QueryProcessor;
+import org.apache.cassandra.db.marshal.BytesType;
+import org.apache.cassandra.db.marshal.CounterColumnType;
+import org.apache.cassandra.db.marshal.UUIDType;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.RequestExecutionException;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.utils.UUIDGen;
 import org.apache.commons.lang3.StringUtils;
+import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 
@@ -58,12 +65,28 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
 
 @RunWith(OrderedJUnit4ClassRunner.class)
-public class ScrubTest extends SchemaLoader
+public class ScrubTest
 {
-    public String KEYSPACE = "Keyspace1";
-    public String CF = "Standard1";
-    public String CF3 = "Standard2";
-    public String COUNTER_CF = "Counter1";
+    public static final String KEYSPACE = "Keyspace1";
+    public static final String CF = "Standard1";
+    public static final String CF2 = "Standard2";
+    public static final String CF3 = "Standard3";
+    public static final String COUNTER_CF = "Counter1";
+    public static final String CF_UUID = "UUIDKeys";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.loadSchema();
+        SchemaLoader.createKeyspace(KEYSPACE,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE, CF),
+                                    SchemaLoader.standardCFMD(KEYSPACE, CF2),
+                                    SchemaLoader.standardCFMD(KEYSPACE, CF3),
+                                    CFMetaData.denseCFMetaData(KEYSPACE, COUNTER_CF, BytesType.instance).defaultValidator(CounterColumnType.instance),
+                                    SchemaLoader.standardCFMD(KEYSPACE, CF_UUID).keyValidator(UUIDType.instance));
+    }
 
     @Test
     public void testScrubOneRow() throws ExecutionException, InterruptedException
@@ -138,10 +161,10 @@ public class ScrubTest extends SchemaLoader
     {
         CompactionManager.instance.disableAutoCompaction();
         Keyspace keyspace = Keyspace.open(KEYSPACE);
-        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF3);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF2);
         cfs.clearUnsafe();
 
-        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(KEYSPACE, CF3);
+        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(KEYSPACE, CF2);
         cf.delete(new DeletionInfo(0, 1)); // expired tombstone
         Mutation rm = new Mutation(KEYSPACE, ByteBufferUtil.bytes(1), cf);
         rm.applyUnsafe();
@@ -178,7 +201,7 @@ public class ScrubTest extends SchemaLoader
     {
         CompactionManager.instance.disableAutoCompaction();
         Keyspace keyspace = Keyspace.open(KEYSPACE);
-        String columnFamily = "Standard3";
+        String columnFamily = CF3;
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(columnFamily);
         cfs.clearUnsafe();
 
@@ -283,12 +306,12 @@ public class ScrubTest extends SchemaLoader
     @Test
     public void testScrubColumnValidation() throws InterruptedException, RequestExecutionException, ExecutionException
     {
-        QueryProcessor.process("CREATE TABLE \"Keyspace1\".test_compact_static_columns (a bigint, b timeuuid, c boolean static, d text, PRIMARY KEY (a, b))", ConsistencyLevel.ONE);
+        QueryProcessor.process(String.format("CREATE TABLE \"%s\".test_compact_static_columns (a bigint, b timeuuid, c boolean static, d text, PRIMARY KEY (a, b))", KEYSPACE), ConsistencyLevel.ONE);
 
-        Keyspace keyspace = Keyspace.open("Keyspace1");
+        Keyspace keyspace = Keyspace.open(KEYSPACE);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("test_compact_static_columns");
 
-        QueryProcessor.executeInternal("INSERT INTO \"Keyspace1\".test_compact_static_columns (a, b, c, d) VALUES (123, c3db07e8-b602-11e3-bc6b-e0b9a54a6d93, true, 'foobar')");
+        QueryProcessor.executeInternal(String.format("INSERT INTO \"%s\".test_compact_static_columns (a, b, c, d) VALUES (123, c3db07e8-b602-11e3-bc6b-e0b9a54a6d93, true, 'foobar')", KEYSPACE));
         cfs.forceBlockingFlush();
         CompactionManager.instance.performScrub(cfs, false);
     }
@@ -299,12 +322,12 @@ public class ScrubTest extends SchemaLoader
     @Test
     public void testColumnNameEqualToDefaultKeyAlias() throws ExecutionException, InterruptedException
     {
-        Keyspace keyspace = Keyspace.open("Keyspace1");
-        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("UUIDKeys");
+        Keyspace keyspace = Keyspace.open(KEYSPACE);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF_UUID);
 
-        ColumnFamily cf = ArrayBackedSortedColumns.factory.create("Keyspace1", "UUIDKeys");
+        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(KEYSPACE, CF_UUID);
         cf.addColumn(column(CFMetaData.DEFAULT_KEY_ALIAS, "not a uuid", 1L));
-        Mutation mutation = new Mutation("Keyspace1", ByteBufferUtil.bytes(UUIDGen.getTimeUUID()), cf);
+        Mutation mutation = new Mutation(KEYSPACE, ByteBufferUtil.bytes(UUIDGen.getTimeUUID()), cf);
         mutation.applyUnsafe();
         cfs.forceBlockingFlush();
         CompactionManager.instance.performScrub(cfs, false);
@@ -319,19 +342,19 @@ public class ScrubTest extends SchemaLoader
     @Test
     public void testValidationCompactStorage() throws Exception
     {
-        QueryProcessor.process("CREATE TABLE \"Keyspace1\".test_compact_dynamic_columns (a int, b text, c text, PRIMARY KEY (a, b)) WITH COMPACT STORAGE", ConsistencyLevel.ONE);
+        QueryProcessor.process(String.format("CREATE TABLE \"%s\".test_compact_dynamic_columns (a int, b text, c text, PRIMARY KEY (a, b)) WITH COMPACT STORAGE", KEYSPACE), ConsistencyLevel.ONE);
 
-        Keyspace keyspace = Keyspace.open("Keyspace1");
+        Keyspace keyspace = Keyspace.open(KEYSPACE);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("test_compact_dynamic_columns");
 
-        QueryProcessor.executeInternal("INSERT INTO \"Keyspace1\".test_compact_dynamic_columns (a, b, c) VALUES (0, 'a', 'foo')");
-        QueryProcessor.executeInternal("INSERT INTO \"Keyspace1\".test_compact_dynamic_columns (a, b, c) VALUES (0, 'b', 'bar')");
-        QueryProcessor.executeInternal("INSERT INTO \"Keyspace1\".test_compact_dynamic_columns (a, b, c) VALUES (0, 'c', 'boo')");
+        QueryProcessor.executeInternal(String.format("INSERT INTO \"%s\".test_compact_dynamic_columns (a, b, c) VALUES (0, 'a', 'foo')", KEYSPACE));
+        QueryProcessor.executeInternal(String.format("INSERT INTO \"%s\".test_compact_dynamic_columns (a, b, c) VALUES (0, 'b', 'bar')", KEYSPACE));
+        QueryProcessor.executeInternal(String.format("INSERT INTO \"%s\".test_compact_dynamic_columns (a, b, c) VALUES (0, 'c', 'boo')", KEYSPACE));
         cfs.forceBlockingFlush();
         CompactionManager.instance.performScrub(cfs, true);
 
         // Scrub is silent, but it will remove broken records. So reading everything back to make sure nothing to "scrubbed away"
-        UntypedResultSet rs = QueryProcessor.executeInternal("SELECT * FROM \"Keyspace1\".test_compact_dynamic_columns");
+        UntypedResultSet rs = QueryProcessor.executeInternal(String.format("SELECT * FROM \"%s\".test_compact_dynamic_columns", KEYSPACE));
         assertEquals(3, rs.size());
 
         Iterator<UntypedResultSet.Row> iter = rs.iterator();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/db/SerializationsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/SerializationsTest.java b/test/unit/org/apache/cassandra/db/SerializationsTest.java
index 0dd6b8f..a50bbae 100644
--- a/test/unit/org/apache/cassandra/db/SerializationsTest.java
+++ b/test/unit/org/apache/cassandra/db/SerializationsTest.java
@@ -19,16 +19,21 @@
 package org.apache.cassandra.db;
 
 import org.apache.cassandra.AbstractSerializationsTester;
+import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.db.composites.*;
 import org.apache.cassandra.db.filter.*;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.marshal.BytesType;
+import org.apache.cassandra.db.marshal.LongType;
 import org.apache.cassandra.dht.AbstractBounds;
 import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.util.DataOutputStreamAndChannel;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.net.CallbackInfo;
 import org.apache.cassandra.net.MessageIn;
 import org.apache.cassandra.net.MessageOut;
@@ -36,6 +41,7 @@ import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import java.io.DataInputStream;
@@ -47,6 +53,7 @@ public class SerializationsTest extends AbstractSerializationsTester
 {
     Statics statics = new Statics();
 
+    private static final String KEYSPACE1 = "Keyspace1";
     private ByteBuffer startCol = ByteBufferUtil.bytes("Start");
     private ByteBuffer stopCol = ByteBufferUtil.bytes("Stop");
     private Composite emptyCol = Composites.EMPTY;
@@ -56,6 +63,17 @@ public class SerializationsTest extends AbstractSerializationsTester
     public SliceQueryFilter nonEmptyRangePred = new SliceQueryFilter(CellNames.simpleDense(startCol), CellNames.simpleDense(stopCol), true, 100);
     public SliceQueryFilter nonEmptyRangeSCPred = new SliceQueryFilter(CellNames.compositeDense(statics.SC, startCol), CellNames.compositeDense(statics.SC, stopCol), true, 100);
 
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, "Standard1"),
+                                    SchemaLoader.superCFMD(KEYSPACE1, "Super1", LongType.instance));
+    }
+
     private void testRangeSliceCommandWrite() throws IOException
     {
         IPartitioner part = StorageService.getPartitioner();
@@ -339,7 +357,7 @@ public class SerializationsTest extends AbstractSerializationsTester
 
     private static class Statics
     {
-        private final String KS = "Keyspace1";
+        private final String KS = KEYSPACE1;
         private final ByteBuffer Key = ByteBufferUtil.bytes("Key01");
         private final SortedSet<CellName> NamedCols = new TreeSet<CellName>(new SimpleDenseCellNameType(BytesType.instance))
         {{


[5/6] Reduce unit test times due to schema loading

Posted by ty...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java b/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
index dda9b65..467a451 100644
--- a/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
+++ b/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
@@ -45,17 +45,14 @@ import com.google.common.collect.Iterables;
 import com.google.common.collect.Sets;
 import org.apache.commons.lang3.ArrayUtils;
 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.SchemaLoader;
 import org.apache.cassandra.Util;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.IndexType;
-import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.config.*;
 import org.apache.cassandra.db.columniterator.IdentityQueryFilter;
 import org.apache.cassandra.db.composites.CellName;
 import org.apache.cassandra.db.composites.CellNameType;
@@ -67,13 +64,16 @@ import org.apache.cassandra.db.filter.NamesQueryFilter;
 import org.apache.cassandra.db.filter.QueryFilter;
 import org.apache.cassandra.db.filter.SliceQueryFilter;
 import org.apache.cassandra.db.index.SecondaryIndex;
+import org.apache.cassandra.db.marshal.IntegerType;
 import org.apache.cassandra.db.marshal.LexicalUUIDType;
 import org.apache.cassandra.db.marshal.LongType;
+import org.apache.cassandra.db.marshal.UTF8Type;
 import org.apache.cassandra.dht.Bounds;
 import org.apache.cassandra.dht.ExcludingBounds;
 import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.dht.IncludingExcludingBounds;
 import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.sstable.Component;
 import org.apache.cassandra.io.sstable.Descriptor;
 import org.apache.cassandra.io.sstable.SSTableDeletingTask;
@@ -82,6 +82,7 @@ import org.apache.cassandra.io.sstable.SSTableSimpleWriter;
 import org.apache.cassandra.io.sstable.SSTableWriter;
 import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
 import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.service.ActiveRepairService;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.thrift.SlicePredicate;
@@ -91,6 +92,7 @@ import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.Pair;
 import org.apache.cassandra.utils.WrappedRunnable;
+import org.apache.thrift.TException;
 
 import static org.apache.cassandra.Util.cellname;
 import static org.apache.cassandra.Util.column;
@@ -104,9 +106,22 @@ import static org.junit.Assert.assertSame;
 import static org.junit.Assert.assertTrue;
 
 @RunWith(OrderedJUnit4ClassRunner.class)
-public class ColumnFamilyStoreTest extends SchemaLoader
+public class ColumnFamilyStoreTest
 {
     static byte[] bytes1, bytes2;
+    public static final String KEYSPACE1 = "ColumnFamilyStoreTest1";
+    public static final String KEYSPACE2 = "ColumnFamilyStoreTest2";
+    public static final String KEYSPACE3 = "ColumnFamilyStoreTest3";
+    public static final String CF_STANDARD1 = "Standard1";
+    public static final String CF_STANDARD2 = "Standard2";
+    public static final String CF_STANDARD3 = "Standard3";
+    public static final String CF_STANDARD4 = "Standard4";
+    public static final String CF_STANDARDINT = "StandardInteger1";
+    public static final String CF_SUPER1 = "Super1";
+    public static final String CF_SUPER6 = "Super6";
+    public static final String CF_INDEX1 = "Indexed1";
+    public static final String CF_INDEX2 = "Indexed2";
+    public static final String CF_INDEX3 = "Indexed3";
 
     static
     {
@@ -117,22 +132,51 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         random.nextBytes(bytes2);
     }
 
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException, IOException, TException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD2),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD3),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD4),
+                                    SchemaLoader.indexCFMD(KEYSPACE1, CF_INDEX1, true),
+                                    SchemaLoader.indexCFMD(KEYSPACE1, CF_INDEX2, false),
+                                    SchemaLoader.superCFMD(KEYSPACE1, CF_SUPER1, LongType.instance),
+                                    SchemaLoader.superCFMD(KEYSPACE1, CF_SUPER6, LexicalUUIDType.instance, UTF8Type.instance),
+                                    CFMetaData.denseCFMetaData(KEYSPACE1, CF_STANDARDINT, IntegerType.instance));
+        SchemaLoader.createKeyspace(KEYSPACE2,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE2, CF_STANDARD1),
+                                    SchemaLoader.indexCFMD(KEYSPACE2, CF_INDEX1, true),
+                                    SchemaLoader.compositeIndexCFMD(KEYSPACE2, CF_INDEX2, true),
+                                    SchemaLoader.compositeIndexCFMD(KEYSPACE2, CF_INDEX3, true).gcGraceSeconds(0));
+        SchemaLoader.createKeyspace(KEYSPACE3,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(5),
+                                    SchemaLoader.indexCFMD(KEYSPACE3, CF_INDEX1, true));
+    }
+
     @Test
     // create two sstables, and verify that we only deserialize data from the most recent one
     public void testTimeSortedQuery()
     {
-        Keyspace keyspace = Keyspace.open("Keyspace1");
-        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("Standard1");
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF_STANDARD1);
         cfs.truncateBlocking();
 
         Mutation rm;
-        rm = new Mutation("Keyspace1", ByteBufferUtil.bytes("key1"));
-        rm.add("Standard1", cellname("Column1"), ByteBufferUtil.bytes("asdf"), 0);
+        rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes("key1"));
+        rm.add(CF_STANDARD1, cellname("Column1"), ByteBufferUtil.bytes("asdf"), 0);
         rm.apply();
         cfs.forceBlockingFlush();
 
-        rm = new Mutation("Keyspace1", ByteBufferUtil.bytes("key1"));
-        rm.add("Standard1", cellname("Column1"), ByteBufferUtil.bytes("asdf"), 1);
+        rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes("key1"));
+        rm.add(CF_STANDARD1, cellname("Column1"), ByteBufferUtil.bytes("asdf"), 1);
         rm.apply();
         cfs.forceBlockingFlush();
 
@@ -144,40 +188,40 @@ public class ColumnFamilyStoreTest extends SchemaLoader
     @Test
     public void testGetColumnWithWrongBF()
     {
-        Keyspace keyspace = Keyspace.open("Keyspace1");
-        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("Standard1");
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF_STANDARD1);
         cfs.truncateBlocking();
 
         List<Mutation> rms = new LinkedList<>();
-        Mutation rm = new Mutation("Keyspace1", ByteBufferUtil.bytes("key1"));
-        rm.add("Standard1", cellname("Column1"), ByteBufferUtil.bytes("asdf"), 0);
-        rm.add("Standard1", cellname("Column2"), ByteBufferUtil.bytes("asdf"), 0);
+        Mutation rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes("key1"));
+        rm.add(CF_STANDARD1, cellname("Column1"), ByteBufferUtil.bytes("asdf"), 0);
+        rm.add(CF_STANDARD1, cellname("Column2"), ByteBufferUtil.bytes("asdf"), 0);
         rms.add(rm);
         Util.writeColumnFamily(rms);
 
         List<SSTableReader> ssTables = keyspace.getAllSSTables();
         assertEquals(1, ssTables.size());
         ssTables.get(0).forceFilterFailures();
-        ColumnFamily cf = cfs.getColumnFamily(QueryFilter.getIdentityFilter(Util.dk("key2"), "Standard1", System.currentTimeMillis()));
+        ColumnFamily cf = cfs.getColumnFamily(QueryFilter.getIdentityFilter(Util.dk("key2"), CF_STANDARD1, System.currentTimeMillis()));
         assertNull(cf);
     }
 
     @Test
     public void testEmptyRow() throws Exception
     {
-        Keyspace keyspace = Keyspace.open("Keyspace1");
-        final ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard2");
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
+        final ColumnFamilyStore store = keyspace.getColumnFamilyStore(CF_STANDARD2);
         Mutation rm;
 
-        rm = new Mutation("Keyspace1", ByteBufferUtil.bytes("key1"));
-        rm.delete("Standard2", System.currentTimeMillis());
+        rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes("key1"));
+        rm.delete(CF_STANDARD2, System.currentTimeMillis());
         rm.apply();
 
         Runnable r = new WrappedRunnable()
         {
             public void runMayThrow() throws IOException
             {
-                QueryFilter sliceFilter = QueryFilter.getSliceFilter(Util.dk("key1"), "Standard2", Composites.EMPTY, Composites.EMPTY, false, 1, System.currentTimeMillis());
+                QueryFilter sliceFilter = QueryFilter.getSliceFilter(Util.dk("key1"), CF_STANDARD2, Composites.EMPTY, Composites.EMPTY, false, 1, System.currentTimeMillis());
                 ColumnFamily cf = store.getColumnFamily(sliceFilter);
                 assertTrue(cf.isMarkedForDelete());
                 assertFalse(cf.hasColumns());
@@ -209,29 +253,29 @@ public class ColumnFamilyStoreTest extends SchemaLoader
     @Test
     public void testIndexScan()
     {
-        ColumnFamilyStore cfs = Keyspace.open("Keyspace1").getColumnFamilyStore("Indexed1");
+        ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF_INDEX1);
         Mutation rm;
         CellName nobirthdate = cellname("notbirthdate");
         CellName birthdate = cellname("birthdate");
 
-        rm = new Mutation("Keyspace1", ByteBufferUtil.bytes("k1"));
-        rm.add("Indexed1", nobirthdate, ByteBufferUtil.bytes(1L), 0);
-        rm.add("Indexed1", birthdate, ByteBufferUtil.bytes(1L), 0);
+        rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes("k1"));
+        rm.add(CF_INDEX1, nobirthdate, ByteBufferUtil.bytes(1L), 0);
+        rm.add(CF_INDEX1, birthdate, ByteBufferUtil.bytes(1L), 0);
         rm.apply();
 
-        rm = new Mutation("Keyspace1", ByteBufferUtil.bytes("k2"));
-        rm.add("Indexed1", nobirthdate, ByteBufferUtil.bytes(2L), 0);
-        rm.add("Indexed1", birthdate, ByteBufferUtil.bytes(2L), 0);
+        rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes("k2"));
+        rm.add(CF_INDEX1, nobirthdate, ByteBufferUtil.bytes(2L), 0);
+        rm.add(CF_INDEX1, birthdate, ByteBufferUtil.bytes(2L), 0);
         rm.apply();
 
-        rm = new Mutation("Keyspace1", ByteBufferUtil.bytes("k3"));
-        rm.add("Indexed1", nobirthdate, ByteBufferUtil.bytes(2L), 0);
-        rm.add("Indexed1", birthdate, ByteBufferUtil.bytes(1L), 0);
+        rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes("k3"));
+        rm.add(CF_INDEX1, nobirthdate, ByteBufferUtil.bytes(2L), 0);
+        rm.add(CF_INDEX1, birthdate, ByteBufferUtil.bytes(1L), 0);
         rm.apply();
 
-        rm = new Mutation("Keyspace1", ByteBufferUtil.bytes("k4aaaa"));
-        rm.add("Indexed1", nobirthdate, ByteBufferUtil.bytes(2L), 0);
-        rm.add("Indexed1", birthdate, ByteBufferUtil.bytes(3L), 0);
+        rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes("k4aaaa"));
+        rm.add(CF_INDEX1, nobirthdate, ByteBufferUtil.bytes(2L), 0);
+        rm.add(CF_INDEX1, birthdate, ByteBufferUtil.bytes(3L), 0);
         rm.apply();
 
         // basic single-expression query
@@ -294,12 +338,12 @@ public class ColumnFamilyStoreTest extends SchemaLoader
     public void testLargeScan()
     {
         Mutation rm;
-        ColumnFamilyStore cfs = Keyspace.open("Keyspace1").getColumnFamilyStore("Indexed1");
+        ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF_INDEX1);
         for (int i = 0; i < 100; i++)
         {
-            rm = new Mutation("Keyspace1", ByteBufferUtil.bytes("key" + i));
-            rm.add("Indexed1", cellname("birthdate"), ByteBufferUtil.bytes(34L), 0);
-            rm.add("Indexed1", cellname("notbirthdate"), ByteBufferUtil.bytes((long) (i % 2)), 0);
+            rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes("key" + i));
+            rm.add(CF_INDEX1, cellname("birthdate"), ByteBufferUtil.bytes(34L), 0);
+            rm.add(CF_INDEX1, cellname("notbirthdate"), ByteBufferUtil.bytes((long) (i % 2)), 0);
             rm.applyUnsafe();
         }
 
@@ -322,11 +366,11 @@ public class ColumnFamilyStoreTest extends SchemaLoader
     @Test
     public void testIndexDeletions() throws IOException
     {
-        ColumnFamilyStore cfs = Keyspace.open("Keyspace3").getColumnFamilyStore("Indexed1");
+        ColumnFamilyStore cfs = Keyspace.open(KEYSPACE3).getColumnFamilyStore(CF_INDEX1);
         Mutation rm;
 
-        rm = new Mutation("Keyspace3", ByteBufferUtil.bytes("k1"));
-        rm.add("Indexed1", cellname("birthdate"), ByteBufferUtil.bytes(1L), 0);
+        rm = new Mutation(KEYSPACE3, ByteBufferUtil.bytes("k1"));
+        rm.add(CF_INDEX1, cellname("birthdate"), ByteBufferUtil.bytes(1L), 0);
         rm.apply();
 
         IndexExpression expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), IndexExpression.Operator.EQ, ByteBufferUtil.bytes(1L));
@@ -339,8 +383,8 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         assert "k1".equals( key );
 
         // delete the column directly
-        rm = new Mutation("Keyspace3", ByteBufferUtil.bytes("k1"));
-        rm.delete("Indexed1", cellname("birthdate"), 1);
+        rm = new Mutation(KEYSPACE3, ByteBufferUtil.bytes("k1"));
+        rm.delete(CF_INDEX1, cellname("birthdate"), 1);
         rm.apply();
         rows = cfs.search(range, clause, filter, 100);
         assert rows.isEmpty();
@@ -354,8 +398,8 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         assert rows.isEmpty();
 
         // resurrect w/ a newer timestamp
-        rm = new Mutation("Keyspace3", ByteBufferUtil.bytes("k1"));
-        rm.add("Indexed1", cellname("birthdate"), ByteBufferUtil.bytes(1L), 2);
+        rm = new Mutation(KEYSPACE3, ByteBufferUtil.bytes("k1"));
+        rm.add(CF_INDEX1, cellname("birthdate"), ByteBufferUtil.bytes(1L), 2);
         rm.apply();
         rows = cfs.search(range, clause, filter, 100);
         assert rows.size() == 1 : StringUtils.join(rows, ",");
@@ -363,8 +407,8 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         assert "k1".equals( key );
 
         // verify that row and delete w/ older timestamp does nothing
-        rm = new Mutation("Keyspace3", ByteBufferUtil.bytes("k1"));
-        rm.delete("Indexed1", 1);
+        rm = new Mutation(KEYSPACE3, ByteBufferUtil.bytes("k1"));
+        rm.delete(CF_INDEX1, 1);
         rm.apply();
         rows = cfs.search(range, clause, filter, 100);
         assert rows.size() == 1 : StringUtils.join(rows, ",");
@@ -372,8 +416,8 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         assert "k1".equals( key );
 
         // similarly, column delete w/ older timestamp should do nothing
-        rm = new Mutation("Keyspace3", ByteBufferUtil.bytes("k1"));
-        rm.delete("Indexed1", cellname("birthdate"), 1);
+        rm = new Mutation(KEYSPACE3, ByteBufferUtil.bytes("k1"));
+        rm.delete(CF_INDEX1, cellname("birthdate"), 1);
         rm.apply();
         rows = cfs.search(range, clause, filter, 100);
         assert rows.size() == 1 : StringUtils.join(rows, ",");
@@ -381,31 +425,31 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         assert "k1".equals( key );
 
         // delete the entire row (w/ newer timestamp this time)
-        rm = new Mutation("Keyspace3", ByteBufferUtil.bytes("k1"));
-        rm.delete("Indexed1", 3);
+        rm = new Mutation(KEYSPACE3, ByteBufferUtil.bytes("k1"));
+        rm.delete(CF_INDEX1, 3);
         rm.apply();
         rows = cfs.search(range, clause, filter, 100);
         assert rows.isEmpty() : StringUtils.join(rows, ",");
 
         // make sure obsolete mutations don't generate an index entry
-        rm = new Mutation("Keyspace3", ByteBufferUtil.bytes("k1"));
-        rm.add("Indexed1", cellname("birthdate"), ByteBufferUtil.bytes(1L), 3);
+        rm = new Mutation(KEYSPACE3, ByteBufferUtil.bytes("k1"));
+        rm.add(CF_INDEX1, cellname("birthdate"), ByteBufferUtil.bytes(1L), 3);
         rm.apply();
         rows = cfs.search(range, clause, filter, 100);
         assert rows.isEmpty() : StringUtils.join(rows, ",");
 
         // try insert followed by row delete in the same mutation
-        rm = new Mutation("Keyspace3", ByteBufferUtil.bytes("k1"));
-        rm.add("Indexed1", cellname("birthdate"), ByteBufferUtil.bytes(1L), 1);
-        rm.delete("Indexed1", 2);
+        rm = new Mutation(KEYSPACE3, ByteBufferUtil.bytes("k1"));
+        rm.add(CF_INDEX1, cellname("birthdate"), ByteBufferUtil.bytes(1L), 1);
+        rm.delete(CF_INDEX1, 2);
         rm.apply();
         rows = cfs.search(range, clause, filter, 100);
         assert rows.isEmpty() : StringUtils.join(rows, ",");
 
         // try row delete followed by insert in the same mutation
-        rm = new Mutation("Keyspace3", ByteBufferUtil.bytes("k1"));
-        rm.delete("Indexed1", 3);
-        rm.add("Indexed1", cellname("birthdate"), ByteBufferUtil.bytes(1L), 4);
+        rm = new Mutation(KEYSPACE3, ByteBufferUtil.bytes("k1"));
+        rm.delete(CF_INDEX1, 3);
+        rm.add(CF_INDEX1, cellname("birthdate"), ByteBufferUtil.bytes(1L), 4);
         rm.apply();
         rows = cfs.search(range, clause, filter, 100);
         assert rows.size() == 1 : StringUtils.join(rows, ",");
@@ -416,17 +460,17 @@ public class ColumnFamilyStoreTest extends SchemaLoader
     @Test
     public void testIndexUpdate() throws IOException
     {
-        Keyspace keyspace = Keyspace.open("Keyspace2");
-        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("Indexed1");
+        Keyspace keyspace = Keyspace.open(KEYSPACE2);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF_INDEX1);
         CellName birthdate = cellname("birthdate");
 
         // create a row and update the birthdate value, test that the index query fetches the new version
         Mutation rm;
-        rm = new Mutation("Keyspace2", ByteBufferUtil.bytes("k1"));
-        rm.add("Indexed1", birthdate, ByteBufferUtil.bytes(1L), 1);
+        rm = new Mutation(KEYSPACE2, ByteBufferUtil.bytes("k1"));
+        rm.add(CF_INDEX1, birthdate, ByteBufferUtil.bytes(1L), 1);
         rm.apply();
-        rm = new Mutation("Keyspace2", ByteBufferUtil.bytes("k1"));
-        rm.add("Indexed1", birthdate, ByteBufferUtil.bytes(2L), 2);
+        rm = new Mutation(KEYSPACE2, ByteBufferUtil.bytes("k1"));
+        rm.add(CF_INDEX1, birthdate, ByteBufferUtil.bytes(2L), 2);
         rm.apply();
 
         IndexExpression expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), IndexExpression.Operator.EQ, ByteBufferUtil.bytes(1L));
@@ -438,16 +482,16 @@ public class ColumnFamilyStoreTest extends SchemaLoader
 
         expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), IndexExpression.Operator.EQ, ByteBufferUtil.bytes(2L));
         clause = Arrays.asList(expr);
-        rows = keyspace.getColumnFamilyStore("Indexed1").search(range, clause, filter, 100);
+        rows = keyspace.getColumnFamilyStore(CF_INDEX1).search(range, clause, filter, 100);
         String key = ByteBufferUtil.string(rows.get(0).key.getKey());
         assert "k1".equals( key );
 
         // update the birthdate value with an OLDER timestamp, and test that the index ignores this
-        rm = new Mutation("Keyspace2", ByteBufferUtil.bytes("k1"));
-        rm.add("Indexed1", birthdate, ByteBufferUtil.bytes(3L), 0);
+        rm = new Mutation(KEYSPACE2, ByteBufferUtil.bytes("k1"));
+        rm.add(CF_INDEX1, birthdate, ByteBufferUtil.bytes(3L), 0);
         rm.apply();
 
-        rows = keyspace.getColumnFamilyStore("Indexed1").search(range, clause, filter, 100);
+        rows = keyspace.getColumnFamilyStore(CF_INDEX1).search(range, clause, filter, 100);
         key = ByteBufferUtil.string(rows.get(0).key.getKey());
         assert "k1".equals( key );
 
@@ -457,11 +501,11 @@ public class ColumnFamilyStoreTest extends SchemaLoader
     public void testIndexUpdateOverwritingExpiringColumns() throws Exception
     {
         // see CASSANDRA-7268
-        Keyspace keyspace = Keyspace.open("Keyspace2");
+        Keyspace keyspace = Keyspace.open(KEYSPACE2);
 
         // create a row and update the birthdate value with an expiring column
         Mutation rm;
-        rm = new Mutation("Keyspace2", ByteBufferUtil.bytes("k100"));
+        rm = new Mutation(KEYSPACE2, ByteBufferUtil.bytes("k100"));
         rm.add("Indexed1", cellname("birthdate"), ByteBufferUtil.bytes(100L), 1, 1000);
         rm.apply();
 
@@ -476,7 +520,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         TimeUnit.SECONDS.sleep(1);
 
         // now overwrite with the same name/value/ttl, but the local expiry time will be different
-        rm = new Mutation("Keyspace2", ByteBufferUtil.bytes("k100"));
+        rm = new Mutation(KEYSPACE2, ByteBufferUtil.bytes("k100"));
         rm.add("Indexed1", cellname("birthdate"), ByteBufferUtil.bytes(100L), 1, 1000);
         rm.apply();
 
@@ -484,7 +528,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         assertEquals(1, rows.size());
 
         // check that modifying the indexed value using the same timestamp behaves as expected
-        rm = new Mutation("Keyspace2", ByteBufferUtil.bytes("k101"));
+        rm = new Mutation(KEYSPACE2, ByteBufferUtil.bytes("k101"));
         rm.add("Indexed1", cellname("birthdate"), ByteBufferUtil.bytes(101L), 1, 1000);
         rm.apply();
 
@@ -494,7 +538,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         assertEquals(1, rows.size());
 
         TimeUnit.SECONDS.sleep(1);
-        rm = new Mutation("Keyspace2", ByteBufferUtil.bytes("k101"));
+        rm = new Mutation(KEYSPACE2, ByteBufferUtil.bytes("k101"));
         rm.add("Indexed1", cellname("birthdate"), ByteBufferUtil.bytes(102L), 1, 1000);
         rm.apply();
         // search for the old value
@@ -510,8 +554,8 @@ public class ColumnFamilyStoreTest extends SchemaLoader
     @Test
     public void testDeleteOfInconsistentValuesInKeysIndex() throws Exception
     {
-        String keySpace = "Keyspace2";
-        String cfName = "Indexed1";
+        String keySpace = KEYSPACE2;
+        String cfName = CF_INDEX1;
 
         Keyspace keyspace = Keyspace.open(keySpace);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfName);
@@ -573,8 +617,8 @@ public class ColumnFamilyStoreTest extends SchemaLoader
     @Test
     public void testDeleteOfInconsistentValuesFromCompositeIndex() throws Exception
     {
-        String keySpace = "Keyspace2";
-        String cfName = "Indexed2";
+        String keySpace = KEYSPACE2;
+        String cfName = CF_INDEX2;
 
         Keyspace keyspace = Keyspace.open(keySpace);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfName);
@@ -646,8 +690,8 @@ public class ColumnFamilyStoreTest extends SchemaLoader
     @Test
     public void testDeleteCompositeIndex() throws Exception
     {
-        String keySpace = "Keyspace2";
-        String cfName = "Indexed3"; // has gcGrace 0
+        String keySpace = KEYSPACE2;
+        String cfName = CF_INDEX3; // has gcGrace 0
 
         Keyspace keyspace = Keyspace.open(keySpace);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfName);
@@ -692,30 +736,30 @@ public class ColumnFamilyStoreTest extends SchemaLoader
     @Test
     public void testIndexScanWithLimitOne()
     {
-        ColumnFamilyStore cfs = Keyspace.open("Keyspace1").getColumnFamilyStore("Indexed1");
+        ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF_INDEX1);
         Mutation rm;
 
         CellName nobirthdate = cellname("notbirthdate");
         CellName birthdate = cellname("birthdate");
 
-        rm = new Mutation("Keyspace1", ByteBufferUtil.bytes("kk1"));
-        rm.add("Indexed1", nobirthdate, ByteBufferUtil.bytes(1L), 0);
-        rm.add("Indexed1", birthdate, ByteBufferUtil.bytes(1L), 0);
+        rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes("kk1"));
+        rm.add(CF_INDEX1, nobirthdate, ByteBufferUtil.bytes(1L), 0);
+        rm.add(CF_INDEX1, birthdate, ByteBufferUtil.bytes(1L), 0);
         rm.apply();
 
-        rm = new Mutation("Keyspace1", ByteBufferUtil.bytes("kk2"));
-        rm.add("Indexed1", nobirthdate, ByteBufferUtil.bytes(2L), 0);
-        rm.add("Indexed1", birthdate, ByteBufferUtil.bytes(1L), 0);
+        rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes("kk2"));
+        rm.add(CF_INDEX1, nobirthdate, ByteBufferUtil.bytes(2L), 0);
+        rm.add(CF_INDEX1, birthdate, ByteBufferUtil.bytes(1L), 0);
         rm.apply();
 
-        rm = new Mutation("Keyspace1", ByteBufferUtil.bytes("kk3"));
-        rm.add("Indexed1", nobirthdate, ByteBufferUtil.bytes(2L), 0);
-        rm.add("Indexed1", birthdate, ByteBufferUtil.bytes(1L), 0);
+        rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes("kk3"));
+        rm.add(CF_INDEX1, nobirthdate, ByteBufferUtil.bytes(2L), 0);
+        rm.add(CF_INDEX1, birthdate, ByteBufferUtil.bytes(1L), 0);
         rm.apply();
 
-        rm = new Mutation("Keyspace1", ByteBufferUtil.bytes("kk4"));
-        rm.add("Indexed1", nobirthdate, ByteBufferUtil.bytes(2L), 0);
-        rm.add("Indexed1", birthdate, ByteBufferUtil.bytes(1L), 0);
+        rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes("kk4"));
+        rm.add(CF_INDEX1, nobirthdate, ByteBufferUtil.bytes(2L), 0);
+        rm.add(CF_INDEX1, birthdate, ByteBufferUtil.bytes(1L), 0);
         rm.apply();
 
         // basic single-expression query
@@ -733,13 +777,13 @@ public class ColumnFamilyStoreTest extends SchemaLoader
     @Test
     public void testIndexCreate() throws IOException, InterruptedException, ExecutionException
     {
-        Keyspace keyspace = Keyspace.open("Keyspace1");
-        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("Indexed2");
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF_INDEX2);
 
         // create a row and update the birthdate value, test that the index query fetches the new version
         Mutation rm;
-        rm = new Mutation("Keyspace1", ByteBufferUtil.bytes("k1"));
-        rm.add("Indexed2", cellname("birthdate"), ByteBufferUtil.bytes(1L), 1);
+        rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes("k1"));
+        rm.add(CF_INDEX2, cellname("birthdate"), ByteBufferUtil.bytes(1L), 1);
         rm.apply();
 
         ColumnDefinition old = cfs.metadata.getColumnDefinition(ByteBufferUtil.bytes("birthdate"));
@@ -767,7 +811,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         IndexExpression expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), IndexExpression.Operator.EQ, ByteBufferUtil.bytes(1L));
         List<IndexExpression> clause = Arrays.asList(expr);
         IDiskAtomFilter filter = new IdentityQueryFilter();
-        List<Row> rows = keyspace.getColumnFamilyStore("Indexed2").search(Util.range("", ""), clause, filter, 100);
+        List<Row> rows = keyspace.getColumnFamilyStore(CF_INDEX2).search(Util.range("", ""), clause, filter, 100);
         assert rows.size() == 1 : StringUtils.join(rows, ",");
         assertEquals("k1", ByteBufferUtil.string(rows.get(0).key.getKey()));
     }
@@ -775,19 +819,19 @@ public class ColumnFamilyStoreTest extends SchemaLoader
     @Test
     public void testCassandra6778() throws CharacterCodingException
     {
-        String cfname = "StandardInteger1";
-        Keyspace keyspace = Keyspace.open("Keyspace1");
+        String cfname = CF_STANDARDINT;
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfname);
 
         // insert two columns that represent the same integer but have different binary forms (the
         // second one is padded with extra zeros)
-        Mutation rm = new Mutation("Keyspace1", ByteBufferUtil.bytes("k1"));
+        Mutation rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes("k1"));
         CellName column1 = cellname(ByteBuffer.wrap(new byte[]{1}));
         rm.add(cfname, column1, ByteBufferUtil.bytes("data1"), 1);
         rm.apply();
         cfs.forceBlockingFlush();
 
-        rm = new Mutation("Keyspace1", ByteBufferUtil.bytes("k1"));
+        rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes("k1"));
         CellName column2 = cellname(ByteBuffer.wrap(new byte[]{0, 0, 1}));
         rm.add(cfname, column2, ByteBufferUtil.bytes("data2"), 2);
         rm.apply();
@@ -795,7 +839,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
 
         // fetch by the first column name; we should get the second version of the column value
         SliceByNamesReadCommand cmd = new SliceByNamesReadCommand(
-            "Keyspace1", ByteBufferUtil.bytes("k1"), cfname, System.currentTimeMillis(),
+            KEYSPACE1, ByteBufferUtil.bytes("k1"), cfname, System.currentTimeMillis(),
             new NamesQueryFilter(FBUtilities.singleton(column1, cfs.getComparator())));
 
         ColumnFamily cf = cmd.getRow(keyspace).cf;
@@ -806,7 +850,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
 
         // fetch by the second column name; we should get the second version of the column value
         cmd = new SliceByNamesReadCommand(
-            "Keyspace1", ByteBufferUtil.bytes("k1"), cfname, System.currentTimeMillis(),
+            KEYSPACE1, ByteBufferUtil.bytes("k1"), cfname, System.currentTimeMillis(),
             new NamesQueryFilter(FBUtilities.singleton(column2, cfs.getComparator())));
 
         cf = cmd.getRow(keyspace).cf;
@@ -832,8 +876,8 @@ public class ColumnFamilyStoreTest extends SchemaLoader
     @Test
     public void testDeleteSuperRowSticksAfterFlush() throws Throwable
     {
-        String keyspaceName = "Keyspace1";
-        String cfName= "Super1";
+        String keyspaceName = KEYSPACE1;
+        String cfName= CF_SUPER1;
         ByteBuffer scfName = ByteBufferUtil.bytes("SuperDuper");
         Keyspace keyspace = Keyspace.open(keyspaceName);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfName);
@@ -933,8 +977,8 @@ public class ColumnFamilyStoreTest extends SchemaLoader
     public void testDeleteStandardRowSticksAfterFlush() throws Throwable
     {
         // test to make sure flushing after a delete doesn't resurrect delted cols.
-        String keyspaceName = "Keyspace1";
-        String cfName = "Standard1";
+        String keyspaceName = KEYSPACE1;
+        String cfName = CF_STANDARD1;
         Keyspace keyspace = Keyspace.open(keyspaceName);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfName);
         DecoratedKey key = Util.dk("f-flush-resurrection");
@@ -988,16 +1032,16 @@ public class ColumnFamilyStoreTest extends SchemaLoader
 
     private ColumnFamilyStore insertKey1Key2()
     {
-        ColumnFamilyStore cfs = Keyspace.open("Keyspace2").getColumnFamilyStore("Standard1");
+        ColumnFamilyStore cfs = Keyspace.open(KEYSPACE2).getColumnFamilyStore(CF_STANDARD1);
         List<Mutation> rms = new LinkedList<>();
         Mutation rm;
-        rm = new Mutation("Keyspace2", ByteBufferUtil.bytes("key1"));
-        rm.add("Standard1", cellname("Column1"), ByteBufferUtil.bytes("asdf"), 0);
+        rm = new Mutation(KEYSPACE2, ByteBufferUtil.bytes("key1"));
+        rm.add(CF_STANDARD1, cellname("Column1"), ByteBufferUtil.bytes("asdf"), 0);
         rms.add(rm);
         Util.writeColumnFamily(rms);
 
-        rm = new Mutation("Keyspace2", ByteBufferUtil.bytes("key2"));
-        rm.add("Standard1", cellname("Column1"), ByteBufferUtil.bytes("asdf"), 0);
+        rm = new Mutation(KEYSPACE2, ByteBufferUtil.bytes("key2"));
+        rm.add(CF_STANDARD1, cellname("Column1"), ByteBufferUtil.bytes("asdf"), 0);
         rms.add(rm);
         return Util.writeColumnFamily(rms);
     }
@@ -1009,8 +1053,8 @@ public class ColumnFamilyStoreTest extends SchemaLoader
 
         for (int version = 1; version <= 2; ++version)
         {
-            Descriptor existing = new Descriptor(cfs.directories.getDirectoryForNewSSTables(), "Keyspace2", "Standard1", version, Descriptor.Type.FINAL);
-            Descriptor desc = new Descriptor(Directories.getBackupsDirectory(existing), "Keyspace2", "Standard1", version, Descriptor.Type.FINAL);
+            Descriptor existing = new Descriptor(cfs.directories.getDirectoryForNewSSTables(), KEYSPACE2, CF_STANDARD1, version, Descriptor.Type.FINAL);
+            Descriptor desc = new Descriptor(Directories.getBackupsDirectory(existing), KEYSPACE2, CF_STANDARD1, version, Descriptor.Type.FINAL);
             for (Component c : new Component[]{ Component.DATA, Component.PRIMARY_INDEX, Component.FILTER, Component.STATS })
                 assertTrue("can not find backedup file:" + desc.filenameFor(c), new File(desc.filenameFor(c)).exists());
         }
@@ -1020,8 +1064,8 @@ public class ColumnFamilyStoreTest extends SchemaLoader
     @Test
     public void testSliceByNamesCommandOnUUIDTypeSCF() throws Throwable
     {
-        String keyspaceName = "Keyspace1";
-        String cfName = "Super6";
+        String keyspaceName = KEYSPACE1;
+        String cfName = CF_SUPER6;
         ByteBuffer superColName = LexicalUUIDType.instance.fromString("a4ed3562-0e8e-4b41-bdfd-c45a2774682d");
         Keyspace keyspace = Keyspace.open(keyspaceName);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfName);
@@ -1051,8 +1095,8 @@ public class ColumnFamilyStoreTest extends SchemaLoader
     @Test
     public void testSliceByNamesCommandOldMetadata() throws Throwable
     {
-        String keyspaceName = "Keyspace1";
-        String cfName= "Standard1";
+        String keyspaceName = KEYSPACE1;
+        String cfName= CF_STANDARD1;
         DecoratedKey key = Util.dk("slice-name-old-metadata");
         CellName cname = cellname("c1");
         Keyspace keyspace = Keyspace.open(keyspaceName);
@@ -1081,7 +1125,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         Cell cell = cf.getColumn(cname);
         assert cell.value().equals(ByteBufferUtil.bytes("a")) : "expecting a, got " + ByteBufferUtil.string(cell.value());
 
-        Keyspace.clear("Keyspace1"); // CASSANDRA-7195
+        Keyspace.clear(KEYSPACE1); // CASSANDRA-7195
     }
 
     private static void assertTotalColCount(Collection<Row> rows, int expectedCount)
@@ -1098,8 +1142,8 @@ public class ColumnFamilyStoreTest extends SchemaLoader
     @Test
     public void testRangeSliceColumnsLimit() throws Throwable
     {
-        String keyspaceName = "Keyspace1";
-        String cfName = "Standard1";
+        String keyspaceName = KEYSPACE1;
+        String cfName = CF_STANDARD1;
         Keyspace keyspace = Keyspace.open(keyspaceName);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfName);
         cfs.clearUnsafe();
@@ -1214,8 +1258,8 @@ public class ColumnFamilyStoreTest extends SchemaLoader
     @Test
     public void testRangeSlicePaging() throws Throwable
     {
-        String keyspaceName = "Keyspace1";
-        String cfName = "Standard1";
+        String keyspaceName = KEYSPACE1;
+        String cfName = CF_STANDARD1;
         Keyspace keyspace = Keyspace.open(keyspaceName);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfName);
         cfs.clearUnsafe();
@@ -1350,8 +1394,8 @@ public class ColumnFamilyStoreTest extends SchemaLoader
     @Test
     public void testRangeSliceInclusionExclusion() throws Throwable
     {
-        String keyspaceName = "Keyspace1";
-        String cfName = "Standard1";
+        String keyspaceName = KEYSPACE1;
+        String cfName = CF_STANDARD1;
         Keyspace keyspace = Keyspace.open(keyspaceName);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfName);
         cfs.clearUnsafe();
@@ -1404,16 +1448,16 @@ public class ColumnFamilyStoreTest extends SchemaLoader
     public void testKeysSearcher() throws Exception
     {
         // Create secondary index and flush to disk
-        Keyspace keyspace = Keyspace.open("Keyspace1");
-        ColumnFamilyStore store = keyspace.getColumnFamilyStore("Indexed1");
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
+        ColumnFamilyStore store = keyspace.getColumnFamilyStore(CF_INDEX1);
 
         store.truncateBlocking();
 
         for (int i = 0; i < 10; i++)
         {
             ByteBuffer key = ByteBufferUtil.bytes(String.valueOf("k" + i));
-            Mutation rm = new Mutation("Keyspace1", key);
-            rm.add("Indexed1", cellname("birthdate"), LongType.instance.decompose(1L), System.currentTimeMillis());
+            Mutation rm = new Mutation(KEYSPACE1, key);
+            rm.add(CF_INDEX1, cellname("birthdate"), LongType.instance.decompose(1L), System.currentTimeMillis());
             rm.apply();
         }
 
@@ -1446,8 +1490,8 @@ public class ColumnFamilyStoreTest extends SchemaLoader
                 new ColumnSlice(cellname("colE"), cellname("colC")),
                 new ColumnSlice(cellname("colA"), Composites.EMPTY) };
 
-        String tableName = "Keyspace1";
-        String cfName = "Standard1";
+        String tableName = KEYSPACE1;
+        String cfName = CF_STANDARD1;
         Keyspace table = Keyspace.open(tableName);
         ColumnFamilyStore cfs = table.getColumnFamilyStore(cfName);
         cfs.clearUnsafe();
@@ -1495,8 +1539,8 @@ public class ColumnFamilyStoreTest extends SchemaLoader
                 new ColumnSlice(cellname("colE"), cellname("colC")),
                 new ColumnSlice(cellname("colA"), Composites.EMPTY) };
 
-        String tableName = "Keyspace1";
-        String cfName = "Standard1";
+        String tableName = KEYSPACE1;
+        String cfName = CF_STANDARD1;
         Keyspace table = Keyspace.open(tableName);
         ColumnFamilyStore cfs = table.getColumnFamilyStore(cfName);
         cfs.clearUnsafe();
@@ -1544,8 +1588,8 @@ public class ColumnFamilyStoreTest extends SchemaLoader
                 new ColumnSlice(cellname("colE"), cellname("colC")),
                 new ColumnSlice(cellname("colA"), Composites.EMPTY) };
 
-        String tableName = "Keyspace1";
-        String cfName = "Standard1";
+        String tableName = KEYSPACE1;
+        String cfName = CF_STANDARD1;
         Keyspace table = Keyspace.open(tableName);
         ColumnFamilyStore cfs = table.getColumnFamilyStore(cfName);
         cfs.clearUnsafe();
@@ -1594,8 +1638,8 @@ public class ColumnFamilyStoreTest extends SchemaLoader
                 new ColumnSlice(cellname("colE"), cellname("colC")),
                 new ColumnSlice(cellname("colA"), Composites.EMPTY) };
 
-        String tableName = "Keyspace1";
-        String cfName = "Standard1";
+        String tableName = KEYSPACE1;
+        String cfName = CF_STANDARD1;
         Keyspace table = Keyspace.open(tableName);
         ColumnFamilyStore cfs = table.getColumnFamilyStore(cfName);
         cfs.clearUnsafe();
@@ -1642,8 +1686,8 @@ public class ColumnFamilyStoreTest extends SchemaLoader
                 new ColumnSlice(cellname("colE"), cellname("colC")),
                 new ColumnSlice(cellname("colA"), Composites.EMPTY) };
 
-        String keyspaceName = "Keyspace1";
-        String cfName = "Standard1";
+        String keyspaceName = KEYSPACE1;
+        String cfName = CF_STANDARD1;
         Keyspace keyspace = Keyspace.open(keyspaceName);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfName);
         cfs.clearUnsafe();
@@ -1709,8 +1753,8 @@ public class ColumnFamilyStoreTest extends SchemaLoader
     @Test
     public void testRemoveUnfinishedCompactionLeftovers() throws Throwable
     {
-        String ks = "Keyspace1";
-        String cf = "Standard3"; // should be empty
+        String ks = KEYSPACE1;
+        String cf = CF_STANDARD3; // should be empty
 
         final CFMetaData cfmeta = Schema.instance.getCFMetaData(ks, cf);
         Directories dir = new Directories(cfmeta);
@@ -1775,8 +1819,8 @@ public class ColumnFamilyStoreTest extends SchemaLoader
     @Test
     public void testFailedToRemoveUnfinishedCompactionLeftovers() throws Throwable
     {
-        final String ks = "Keyspace1";
-        final String cf = "Standard4"; // should be empty
+        final String ks = KEYSPACE1;
+        final String cf = CF_STANDARD4; // should be empty
 
         final CFMetaData cfmeta = Schema.instance.getCFMetaData(ks, cf);
         Directories dir = new Directories(cfmeta);
@@ -1827,8 +1871,8 @@ public class ColumnFamilyStoreTest extends SchemaLoader
     @Test
     public void testLoadNewSSTablesAvoidsOverwrites() throws Throwable
     {
-        String ks = "Keyspace1";
-        String cf = "Standard1";
+        String ks = KEYSPACE1;
+        String cf = CF_STANDARD1;
         ColumnFamilyStore cfs = Keyspace.open(ks).getColumnFamilyStore(cf);
         cfs.truncateBlocking();
         SSTableDeletingTask.waitForDeletions();
@@ -1903,8 +1947,8 @@ public class ColumnFamilyStoreTest extends SchemaLoader
 
     private ColumnFamilyStore prepareMultiRangeSlicesTest(int valueSize, boolean flush) throws Throwable
     {
-        String keyspaceName = "Keyspace1";
-        String cfName = "Standard1";
+        String keyspaceName = KEYSPACE1;
+        String cfName = CF_STANDARD1;
         Keyspace keyspace = Keyspace.open(keyspaceName);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfName);
         cfs.clearUnsafe();
@@ -2167,7 +2211,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
     {
         DecoratedKey ROW = Util.dk(rowKey);
         System.err.println("Original:");
-        ColumnFamily cf = cfs.getColumnFamily(QueryFilter.getIdentityFilter(ROW, "Standard1", System.currentTimeMillis()));
+        ColumnFamily cf = cfs.getColumnFamily(QueryFilter.getIdentityFilter(ROW, CF_STANDARD1, System.currentTimeMillis()));
         System.err.println("Row key: " + rowKey + " Cols: "
                 + Iterables.transform(cf.getSortedColumns(), new Function<Cell, String>()
                 {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/db/ColumnFamilyTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/ColumnFamilyTest.java b/test/unit/org/apache/cassandra/db/ColumnFamilyTest.java
index 69a851e..2cae90a 100644
--- a/test/unit/org/apache/cassandra/db/ColumnFamilyTest.java
+++ b/test/unit/org/apache/cassandra/db/ColumnFamilyTest.java
@@ -25,13 +25,20 @@ import java.nio.ByteBuffer;
 import java.util.TreeMap;
 
 import com.google.common.collect.Iterables;
+import org.apache.cassandra.config.CFMetaData;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.KSMetaData;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.db.composites.CellName;
 import org.apache.cassandra.db.context.CounterContext;
+import org.apache.cassandra.db.marshal.BytesType;
+import org.apache.cassandra.db.marshal.CounterColumnType;
 import org.apache.cassandra.io.sstable.ColumnStats;
 import org.apache.cassandra.io.util.DataOutputBuffer;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.CounterId;
@@ -45,9 +52,24 @@ import static org.apache.cassandra.Util.tombstone;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 
-public class ColumnFamilyTest extends SchemaLoader
+public class ColumnFamilyTest
 {
     static int version = MessagingService.current_version;
+    private static final String KEYSPACE1 = "Keyspace1";
+    private static final String CF_STANDARD1 = "Standard1";
+    private static final String CF_COUNTER1 = "Counter1";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1),
+                                    CFMetaData.denseCFMetaData(KEYSPACE1, CF_COUNTER1, BytesType.instance)
+                                              .defaultValidator(CounterColumnType.instance));
+    }
 
     // TODO test SuperColumns more
 
@@ -56,7 +78,7 @@ public class ColumnFamilyTest extends SchemaLoader
     {
         ColumnFamily cf;
 
-        cf = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        cf = ArrayBackedSortedColumns.factory.create(KEYSPACE1, CF_STANDARD1);
         cf.addColumn(column("C", "v", 1));
         DataOutputBuffer bufOut = new DataOutputBuffer();
         ColumnFamily.serializer.serialize(cf, bufOut, version);
@@ -64,7 +86,7 @@ public class ColumnFamilyTest extends SchemaLoader
         ByteArrayInputStream bufIn = new ByteArrayInputStream(bufOut.getData(), 0, bufOut.getLength());
         cf = ColumnFamily.serializer.deserialize(new DataInputStream(bufIn), version);
         assert cf != null;
-        assert cf.metadata().cfName.equals("Standard1");
+        assert cf.metadata().cfName.equals(CF_STANDARD1);
         assert cf.getSortedColumns().size() == 1;
     }
 
@@ -80,7 +102,7 @@ public class ColumnFamilyTest extends SchemaLoader
         }
 
         // write
-        cf = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        cf = ArrayBackedSortedColumns.factory.create(KEYSPACE1, CF_STANDARD1);
         DataOutputBuffer bufOut = new DataOutputBuffer();
         for (String cName : map.navigableKeySet())
         {
@@ -102,7 +124,7 @@ public class ColumnFamilyTest extends SchemaLoader
     @Test
     public void testGetColumnCount()
     {
-        ColumnFamily cf = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(KEYSPACE1, CF_STANDARD1);
 
         cf.addColumn(column("col1", "", 1));
         cf.addColumn(column("col2", "", 2));
@@ -115,8 +137,8 @@ public class ColumnFamilyTest extends SchemaLoader
     @Test
     public void testDigest()
     {
-        ColumnFamily cf = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
-        ColumnFamily cf2 = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(KEYSPACE1, CF_STANDARD1);
+        ColumnFamily cf2 = ArrayBackedSortedColumns.factory.create(KEYSPACE1, CF_STANDARD1);
 
         ByteBuffer digest = ColumnFamily.digest(cf);
 
@@ -157,7 +179,7 @@ public class ColumnFamilyTest extends SchemaLoader
     @Test
     public void testTimestamp()
     {
-        ColumnFamily cf = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(KEYSPACE1, CF_STANDARD1);
 
         cf.addColumn(column("col1", "val1", 2));
         cf.addColumn(column("col1", "val2", 2)); // same timestamp, new value
@@ -169,9 +191,9 @@ public class ColumnFamilyTest extends SchemaLoader
     @Test
     public void testMergeAndAdd()
     {
-        ColumnFamily cf_new = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
-        ColumnFamily cf_old = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
-        ColumnFamily cf_result = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        ColumnFamily cf_new = ArrayBackedSortedColumns.factory.create(KEYSPACE1, CF_STANDARD1);
+        ColumnFamily cf_old = ArrayBackedSortedColumns.factory.create(KEYSPACE1, CF_STANDARD1);
+        ColumnFamily cf_result = ArrayBackedSortedColumns.factory.create(KEYSPACE1, CF_STANDARD1);
         ByteBuffer val = ByteBufferUtil.bytes("sample value");
         ByteBuffer val2 = ByteBufferUtil.bytes("x value ");
 
@@ -207,7 +229,7 @@ public class ColumnFamilyTest extends SchemaLoader
         long timestamp = System.currentTimeMillis();
         int localDeletionTime = (int) (System.currentTimeMillis() / 1000);
 
-        ColumnFamily cf = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(KEYSPACE1, CF_STANDARD1);
         cf.delete(new DeletionInfo(timestamp, localDeletionTime));
         ColumnStats stats = cf.getColumnStats();
         assertEquals(timestamp, stats.maxTimestamp);
@@ -240,14 +262,14 @@ public class ColumnFamilyTest extends SchemaLoader
         assertTrue(counter.reconcile(tombstone) == tombstone);
 
         // check that a range tombstone overrides the counter cell, even with a lower timestamp than the counter
-        ColumnFamily cf0 = ArrayBackedSortedColumns.factory.create("Keyspace1", "Counter1");
+        ColumnFamily cf0 = ArrayBackedSortedColumns.factory.create(KEYSPACE1, CF_COUNTER1);
         cf0.addColumn(counter);
         cf0.delete(new RangeTombstone(cellname("counter0"), cellname("counter2"), 0L, (int) (System.currentTimeMillis() / 1000)));
         assertTrue(cf0.deletionInfo().isDeleted(counter));
         assertTrue(cf0.deletionInfo().inOrderTester(false).isDeleted(counter));
 
         // check that a top-level deletion info overrides the counter cell, even with a lower timestamp than the counter
-        ColumnFamily cf1 = ArrayBackedSortedColumns.factory.create("Keyspace1", "Counter1");
+        ColumnFamily cf1 = ArrayBackedSortedColumns.factory.create(KEYSPACE1, CF_COUNTER1);
         cf1.addColumn(counter);
         cf1.delete(new DeletionInfo(0L, (int) (System.currentTimeMillis() / 1000)));
         assertTrue(cf1.deletionInfo().isDeleted(counter));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/db/CommitLogTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/CommitLogTest.java b/test/unit/org/apache/cassandra/db/CommitLogTest.java
index c4a1fe1..7e30867 100644
--- a/test/unit/org/apache/cassandra/db/CommitLogTest.java
+++ b/test/unit/org/apache/cassandra/db/CommitLogTest.java
@@ -26,22 +26,41 @@ import java.util.zip.CRC32;
 import java.util.zip.Checksum;
 
 import org.junit.Assert;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
 import org.apache.cassandra.config.Config;
 import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.db.commitlog.CommitLog;
 import org.apache.cassandra.db.commitlog.CommitLogDescriptor;
 import org.apache.cassandra.db.commitlog.CommitLogSegment;
 import org.apache.cassandra.db.composites.CellName;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.net.MessagingService;
 
 import static org.apache.cassandra.utils.ByteBufferUtil.bytes;
 
-public class CommitLogTest extends SchemaLoader
+public class CommitLogTest
 {
+    private static final String KEYSPACE1 = "CommitLogTest";
+    private static final String CF1 = "Standard1";
+    private static final String CF2 = "Standard2";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF2));
+    }
+
     @Test
     public void testRecoveryWithEmptyLog() throws Exception
     {
@@ -102,8 +121,8 @@ public class CommitLogTest extends SchemaLoader
     {
         CommitLog.instance.resetUnsafe();
         // Roughly 32 MB mutation
-        Mutation rm = new Mutation("Keyspace1", bytes("k"));
-        rm.add("Standard1", Util.cellname("c1"), ByteBuffer.allocate(DatabaseDescriptor.getCommitLogSegmentSize()/4), 0);
+        Mutation rm = new Mutation(KEYSPACE1, bytes("k"));
+        rm.add(CF1, Util.cellname("c1"), ByteBuffer.allocate(DatabaseDescriptor.getCommitLogSegmentSize()/4), 0);
 
         // Adding it 5 times
         CommitLog.instance.add(rm);
@@ -113,8 +132,8 @@ public class CommitLogTest extends SchemaLoader
         CommitLog.instance.add(rm);
 
         // Adding new mutation on another CF
-        Mutation rm2 = new Mutation("Keyspace1", bytes("k"));
-        rm2.add("Standard2", Util.cellname("c1"), ByteBuffer.allocate(4), 0);
+        Mutation rm2 = new Mutation(KEYSPACE1, bytes("k"));
+        rm2.add(CF2, Util.cellname("c1"), ByteBuffer.allocate(4), 0);
         CommitLog.instance.add(rm2);
 
         assert CommitLog.instance.activeSegments() == 2 : "Expecting 2 segments, got " + CommitLog.instance.activeSegments();
@@ -132,8 +151,8 @@ public class CommitLogTest extends SchemaLoader
         DatabaseDescriptor.getCommitLogSegmentSize();
         CommitLog.instance.resetUnsafe();
         // Roughly 32 MB mutation
-        Mutation rm = new Mutation("Keyspace1", bytes("k"));
-        rm.add("Standard1", Util.cellname("c1"), ByteBuffer.allocate((DatabaseDescriptor.getCommitLogSegmentSize()/4) - 1), 0);
+        Mutation rm = new Mutation(KEYSPACE1, bytes("k"));
+        rm.add(CF1, Util.cellname("c1"), ByteBuffer.allocate((DatabaseDescriptor.getCommitLogSegmentSize()/4) - 1), 0);
 
         // Adding it twice (won't change segment)
         CommitLog.instance.add(rm);
@@ -149,8 +168,8 @@ public class CommitLogTest extends SchemaLoader
         assert CommitLog.instance.activeSegments() == 1 : "Expecting 1 segment, got " + CommitLog.instance.activeSegments();
 
         // Adding new mutation on another CF, large enough (including CL entry overhead) that a new segment is created
-        Mutation rm2 = new Mutation("Keyspace1", bytes("k"));
-        rm2.add("Standard2", Util.cellname("c1"), ByteBuffer.allocate((DatabaseDescriptor.getCommitLogSegmentSize()/2) - 100), 0);
+        Mutation rm2 = new Mutation(KEYSPACE1, bytes("k"));
+        rm2.add(CF2, Util.cellname("c1"), ByteBuffer.allocate((DatabaseDescriptor.getCommitLogSegmentSize()/2) - 100), 0);
         CommitLog.instance.add(rm2);
         // also forces a new segment, since each entry-with-overhead is just under half the CL size
         CommitLog.instance.add(rm2);
@@ -172,7 +191,7 @@ public class CommitLogTest extends SchemaLoader
     private static int getMaxRecordDataSize(String keyspace, ByteBuffer key, String table, CellName column)
     {
         Mutation rm = new Mutation(keyspace, bytes("k"));
-        rm.add("Standard1", Util.cellname("c1"), ByteBuffer.allocate(0), 0);
+        rm.add(CF1, Util.cellname("c1"), ByteBuffer.allocate(0), 0);
 
         int max = (DatabaseDescriptor.getCommitLogSegmentSize() / 2);
         max -= CommitLogSegment.ENTRY_OVERHEAD_SIZE; // log entry overhead
@@ -181,7 +200,7 @@ public class CommitLogTest extends SchemaLoader
 
     private static int getMaxRecordDataSize()
     {
-        return getMaxRecordDataSize("Keyspace1", bytes("k"), "Standard1", Util.cellname("c1"));
+        return getMaxRecordDataSize(KEYSPACE1, bytes("k"), CF1, Util.cellname("c1"));
     }
 
     // CASSANDRA-3615
@@ -190,8 +209,8 @@ public class CommitLogTest extends SchemaLoader
     {
         CommitLog.instance.resetUnsafe();
 
-        Mutation rm = new Mutation("Keyspace1", bytes("k"));
-        rm.add("Standard1", Util.cellname("c1"), ByteBuffer.allocate(getMaxRecordDataSize()), 0);
+        Mutation rm = new Mutation(KEYSPACE1, bytes("k"));
+        rm.add(CF1, Util.cellname("c1"), ByteBuffer.allocate(getMaxRecordDataSize()), 0);
         CommitLog.instance.add(rm);
     }
 
@@ -201,8 +220,8 @@ public class CommitLogTest extends SchemaLoader
         CommitLog.instance.resetUnsafe();
         try
         {
-            Mutation rm = new Mutation("Keyspace1", bytes("k"));
-            rm.add("Standard1", Util.cellname("c1"), ByteBuffer.allocate(1 + getMaxRecordDataSize()), 0);
+            Mutation rm = new Mutation(KEYSPACE1, bytes("k"));
+            rm.add(CF1, Util.cellname("c1"), ByteBuffer.allocate(1 + getMaxRecordDataSize()), 0);
             CommitLog.instance.add(rm);
             throw new AssertionError("mutation larger than limit was accepted");
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/db/CounterCacheTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/CounterCacheTest.java b/test/unit/org/apache/cassandra/db/CounterCacheTest.java
index cb2d97a..71f8b20 100644
--- a/test/unit/org/apache/cassandra/db/CounterCacheTest.java
+++ b/test/unit/org/apache/cassandra/db/CounterCacheTest.java
@@ -20,10 +20,15 @@ package org.apache.cassandra.db;
 import java.util.concurrent.ExecutionException;
 
 import org.junit.AfterClass;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.KSMetaData;
+import org.apache.cassandra.db.marshal.CounterColumnType;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.WriteTimeoutException;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.service.CacheService;
 import org.apache.cassandra.utils.FBUtilities;
 
@@ -33,21 +38,31 @@ import static org.junit.Assert.assertNull;
 import static org.apache.cassandra.Util.cellname;
 import static org.apache.cassandra.utils.ByteBufferUtil.bytes;
 
-public class CounterCacheTest extends SchemaLoader
+public class CounterCacheTest
 {
-    private static final String KS = "CounterCacheSpace";
+    private static final String KEYSPACE1 = "CounterCacheTest";
     private static final String CF = "Counter1";
 
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF).defaultValidator(CounterColumnType.instance));
+    }
+
     @AfterClass
     public static void cleanup()
     {
-        cleanupSavedCaches();
+        SchemaLoader.cleanupSavedCaches();
     }
 
     @Test
     public void testReadWrite()
     {
-        ColumnFamilyStore cfs = Keyspace.open(KS).getColumnFamilyStore(CF);
+        ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF);
         CacheService.instance.invalidateCounterCache();
 
         assertEquals(0, CacheService.instance.counterCache.size());
@@ -71,14 +86,14 @@ public class CounterCacheTest extends SchemaLoader
     @Test
     public void testSaveLoad() throws ExecutionException, InterruptedException, WriteTimeoutException
     {
-        ColumnFamilyStore cfs = Keyspace.open(KS).getColumnFamilyStore(CF);
+        ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF);
         CacheService.instance.invalidateCounterCache();
 
         ColumnFamily cells = ArrayBackedSortedColumns.factory.create(cfs.metadata);
         cells.addColumn(new BufferCounterUpdateCell(cellname(1), 1L, FBUtilities.timestampMicros()));
         cells.addColumn(new BufferCounterUpdateCell(cellname(2), 2L, FBUtilities.timestampMicros()));
-        new CounterMutation(new Mutation(KS, bytes(1), cells), ConsistencyLevel.ONE).apply();
-        new CounterMutation(new Mutation(KS, bytes(2), cells), ConsistencyLevel.ONE).apply();
+        new CounterMutation(new Mutation(KEYSPACE1, bytes(1), cells), ConsistencyLevel.ONE).apply();
+        new CounterMutation(new Mutation(KEYSPACE1, bytes(2), cells), ConsistencyLevel.ONE).apply();
 
         // flush the counter cache and invalidate
         CacheService.instance.counterCache.submitWrite(Integer.MAX_VALUE).get();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/db/CounterCellTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/CounterCellTest.java b/test/unit/org/apache/cassandra/db/CounterCellTest.java
index 21826d2..6b4c636 100644
--- a/test/unit/org/apache/cassandra/db/CounterCellTest.java
+++ b/test/unit/org/apache/cassandra/db/CounterCellTest.java
@@ -41,7 +41,7 @@ import org.apache.cassandra.utils.*;
 import static org.apache.cassandra.Util.cellname;
 import static org.apache.cassandra.db.context.CounterContext.ContextState;
 
-public class CounterCellTest extends SchemaLoader
+public class CounterCellTest
 {
     private static final CounterContext cc = new CounterContext();
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/db/CounterMutationTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/CounterMutationTest.java b/test/unit/org/apache/cassandra/db/CounterMutationTest.java
index 431531c..0aa33c5 100644
--- a/test/unit/org/apache/cassandra/db/CounterMutationTest.java
+++ b/test/unit/org/apache/cassandra/db/CounterMutationTest.java
@@ -19,12 +19,17 @@ package org.apache.cassandra.db;
 
 import java.nio.ByteBuffer;
 
+import org.junit.BeforeClass;
 import org.junit.Test;
 
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.db.context.CounterContext;
 import org.apache.cassandra.db.filter.QueryFilter;
+import org.apache.cassandra.db.marshal.CounterColumnType;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.WriteTimeoutException;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.utils.FBUtilities;
 
 import static org.junit.Assert.assertEquals;
@@ -34,36 +39,47 @@ import static org.apache.cassandra.Util.cellname;
 import static org.apache.cassandra.Util.dk;
 import static org.apache.cassandra.utils.ByteBufferUtil.bytes;
 
-public class CounterMutationTest extends SchemaLoader
+public class CounterMutationTest
 {
-    private static final String KS = "CounterCacheSpace";
+    private static final String KEYSPACE1 = "CounterMutationTest";
     private static final String CF1 = "Counter1";
     private static final String CF2 = "Counter2";
 
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF1).defaultValidator(CounterColumnType.instance),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF2).defaultValidator(CounterColumnType.instance));
+    }
+
     @Test
     public void testSingleCell() throws WriteTimeoutException
     {
-        ColumnFamilyStore cfs = Keyspace.open(KS).getColumnFamilyStore(CF1);
+        ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF1);
         cfs.truncateBlocking();
 
         // Do the initial update (+1)
         ColumnFamily cells = ArrayBackedSortedColumns.factory.create(cfs.metadata);
         cells.addCounter(cellname(1), 1L);
-        new CounterMutation(new Mutation(KS, bytes(1), cells), ConsistencyLevel.ONE).apply();
+        new CounterMutation(new Mutation(KEYSPACE1, bytes(1), cells), ConsistencyLevel.ONE).apply();
         ColumnFamily current = cfs.getColumnFamily(QueryFilter.getIdentityFilter(dk(bytes(1)), CF1, System.currentTimeMillis()));
         assertEquals(1L, CounterContext.instance().total(current.getColumn(cellname(1)).value()));
 
         // Make another increment (+2)
         cells = ArrayBackedSortedColumns.factory.create(cfs.metadata);
         cells.addCounter(cellname(1), 2L);
-        new CounterMutation(new Mutation(KS, bytes(1), cells), ConsistencyLevel.ONE).apply();
+        new CounterMutation(new Mutation(KEYSPACE1, bytes(1), cells), ConsistencyLevel.ONE).apply();
         current = cfs.getColumnFamily(QueryFilter.getIdentityFilter(dk(bytes(1)), CF1, System.currentTimeMillis()));
         assertEquals(3L, CounterContext.instance().total(current.getColumn(cellname(1)).value()));
 
         // Decrement to 0 (-3)
         cells = ArrayBackedSortedColumns.factory.create(cfs.metadata);
         cells.addCounter(cellname(1), -3L);
-        new CounterMutation(new Mutation(KS, bytes(1), cells), ConsistencyLevel.ONE).apply();
+        new CounterMutation(new Mutation(KEYSPACE1, bytes(1), cells), ConsistencyLevel.ONE).apply();
         current = cfs.getColumnFamily(QueryFilter.getIdentityFilter(dk(bytes(1)), CF1, System.currentTimeMillis()));
         assertEquals(0L, CounterContext.instance().total(current.getColumn(cellname(1)).value()));
         assertEquals(ClockAndCount.create(3L, 0L), cfs.getCachedCounter(bytes(1), cellname(1)));
@@ -72,14 +88,14 @@ public class CounterMutationTest extends SchemaLoader
     @Test
     public void testTwoCells() throws WriteTimeoutException
     {
-        ColumnFamilyStore cfs = Keyspace.open(KS).getColumnFamilyStore(CF1);
+        ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF1);
         cfs.truncateBlocking();
 
         // Do the initial update (+1, -1)
         ColumnFamily cells = ArrayBackedSortedColumns.factory.create(cfs.metadata);
         cells.addCounter(cellname(1), 1L);
         cells.addCounter(cellname(2), -1L);
-        new CounterMutation(new Mutation(KS, bytes(1), cells), ConsistencyLevel.ONE).apply();
+        new CounterMutation(new Mutation(KEYSPACE1, bytes(1), cells), ConsistencyLevel.ONE).apply();
         ColumnFamily current = cfs.getColumnFamily(QueryFilter.getIdentityFilter(dk(bytes(1)), CF1, System.currentTimeMillis()));
         assertEquals(1L, CounterContext.instance().total(current.getColumn(cellname(1)).value()));
         assertEquals(-1L, CounterContext.instance().total(current.getColumn(cellname(2)).value()));
@@ -88,7 +104,7 @@ public class CounterMutationTest extends SchemaLoader
         cells = ArrayBackedSortedColumns.factory.create(cfs.metadata);
         cells.addCounter(cellname(1), 2L);
         cells.addCounter(cellname(2), -2L);
-        new CounterMutation(new Mutation(KS, bytes(1), cells), ConsistencyLevel.ONE).apply();
+        new CounterMutation(new Mutation(KEYSPACE1, bytes(1), cells), ConsistencyLevel.ONE).apply();
         current = cfs.getColumnFamily(QueryFilter.getIdentityFilter(dk(bytes(1)), CF1, System.currentTimeMillis()));
         assertEquals(3L, CounterContext.instance().total(current.getColumn(cellname(1)).value()));
 
@@ -96,7 +112,7 @@ public class CounterMutationTest extends SchemaLoader
         cells = ArrayBackedSortedColumns.factory.create(cfs.metadata);
         cells.addCounter(cellname(1), -3L);
         cells.addCounter(cellname(2), 3L);
-        new CounterMutation(new Mutation(KS, bytes(1), cells), ConsistencyLevel.ONE).apply();
+        new CounterMutation(new Mutation(KEYSPACE1, bytes(1), cells), ConsistencyLevel.ONE).apply();
         current = cfs.getColumnFamily(QueryFilter.getIdentityFilter(dk(bytes(1)), CF1, System.currentTimeMillis()));
         assertEquals(0L, CounterContext.instance().total(current.getColumn(cellname(1)).value()));
         assertEquals(0L, CounterContext.instance().total(current.getColumn(cellname(2)).value()));
@@ -109,8 +125,8 @@ public class CounterMutationTest extends SchemaLoader
     @Test
     public void testBatch() throws WriteTimeoutException
     {
-        ColumnFamilyStore cfs1 = Keyspace.open(KS).getColumnFamilyStore(CF1);
-        ColumnFamilyStore cfs2 = Keyspace.open(KS).getColumnFamilyStore(CF2);
+        ColumnFamilyStore cfs1 = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF1);
+        ColumnFamilyStore cfs2 = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF2);
 
         cfs1.truncateBlocking();
         cfs2.truncateBlocking();
@@ -124,7 +140,7 @@ public class CounterMutationTest extends SchemaLoader
         cells2.addCounter(cellname(1), 2L);
         cells2.addCounter(cellname(2), -2L);
 
-        Mutation mutation = new Mutation(KS, bytes(1));
+        Mutation mutation = new Mutation(KEYSPACE1, bytes(1));
         mutation.add(cells1);
         mutation.add(cells2);
 
@@ -149,14 +165,14 @@ public class CounterMutationTest extends SchemaLoader
     @Test
     public void testDeletes() throws WriteTimeoutException
     {
-        ColumnFamilyStore cfs = Keyspace.open(KS).getColumnFamilyStore(CF1);
+        ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF1);
         cfs.truncateBlocking();
 
         // Do the initial update (+1, -1)
         ColumnFamily cells = ArrayBackedSortedColumns.factory.create(cfs.metadata);
         cells.addCounter(cellname(1), 1L);
         cells.addCounter(cellname(2), 1L);
-        new CounterMutation(new Mutation(KS, bytes(1), cells), ConsistencyLevel.ONE).apply();
+        new CounterMutation(new Mutation(KEYSPACE1, bytes(1), cells), ConsistencyLevel.ONE).apply();
         ColumnFamily current = cfs.getColumnFamily(QueryFilter.getIdentityFilter(dk(bytes(1)), CF1, System.currentTimeMillis()));
         assertEquals(1L, CounterContext.instance().total(current.getColumn(cellname(1)).value()));
         assertEquals(1L, CounterContext.instance().total(current.getColumn(cellname(2)).value()));
@@ -165,7 +181,7 @@ public class CounterMutationTest extends SchemaLoader
         cells = ArrayBackedSortedColumns.factory.create(cfs.metadata);
         cells.addTombstone(cellname(1), (int) System.currentTimeMillis() / 1000, FBUtilities.timestampMicros());
         cells.addCounter(cellname(2), 1L);
-        new CounterMutation(new Mutation(KS, bytes(1), cells), ConsistencyLevel.ONE).apply();
+        new CounterMutation(new Mutation(KEYSPACE1, bytes(1), cells), ConsistencyLevel.ONE).apply();
         current = cfs.getColumnFamily(QueryFilter.getIdentityFilter(dk(bytes(1)), CF1, System.currentTimeMillis()));
         assertNull(current.getColumn(cellname(1)));
         assertEquals(2L, CounterContext.instance().total(current.getColumn(cellname(2)).value()));
@@ -173,12 +189,12 @@ public class CounterMutationTest extends SchemaLoader
         // Increment the first counter, make sure it's still shadowed by the tombstone
         cells = ArrayBackedSortedColumns.factory.create(cfs.metadata);
         cells.addCounter(cellname(1), 1L);
-        new CounterMutation(new Mutation(KS, bytes(1), cells), ConsistencyLevel.ONE).apply();
+        new CounterMutation(new Mutation(KEYSPACE1, bytes(1), cells), ConsistencyLevel.ONE).apply();
         current = cfs.getColumnFamily(QueryFilter.getIdentityFilter(dk(bytes(1)), CF1, System.currentTimeMillis()));
         assertNull(current.getColumn(cellname(1)));
 
         // Get rid of the complete partition
-        Mutation mutation = new Mutation(KS, bytes(1));
+        Mutation mutation = new Mutation(KEYSPACE1, bytes(1));
         mutation.delete(CF1, FBUtilities.timestampMicros());
         new CounterMutation(mutation, ConsistencyLevel.ONE).apply();
         current = cfs.getColumnFamily(QueryFilter.getIdentityFilter(dk(bytes(1)), CF1, System.currentTimeMillis()));
@@ -189,7 +205,7 @@ public class CounterMutationTest extends SchemaLoader
         cells = ArrayBackedSortedColumns.factory.create(cfs.metadata);
         cells.addCounter(cellname(1), 1L);
         cells.addCounter(cellname(2), 1L);
-        new CounterMutation(new Mutation(KS, bytes(1), cells), ConsistencyLevel.ONE).apply();
+        new CounterMutation(new Mutation(KEYSPACE1, bytes(1), cells), ConsistencyLevel.ONE).apply();
         current = cfs.getColumnFamily(QueryFilter.getIdentityFilter(dk(bytes(1)), CF1, System.currentTimeMillis()));
         assertNull(current.getColumn(cellname(1)));
         assertNull(current.getColumn(cellname(2)));
@@ -198,7 +214,7 @@ public class CounterMutationTest extends SchemaLoader
     @Test
     public void testDuplicateCells() throws WriteTimeoutException
     {
-        ColumnFamilyStore cfs = Keyspace.open(KS).getColumnFamilyStore(CF1);
+        ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF1);
         cfs.truncateBlocking();
 
         ColumnFamily cells = ArrayBackedSortedColumns.factory.create(cfs.metadata);
@@ -206,7 +222,7 @@ public class CounterMutationTest extends SchemaLoader
         cells.addCounter(cellname(1), 2L);
         cells.addCounter(cellname(1), 3L);
         cells.addCounter(cellname(1), 4L);
-        new CounterMutation(new Mutation(KS, bytes(1), cells), ConsistencyLevel.ONE).apply();
+        new CounterMutation(new Mutation(KEYSPACE1, bytes(1), cells), ConsistencyLevel.ONE).apply();
 
         ColumnFamily current = cfs.getColumnFamily(QueryFilter.getIdentityFilter(dk(bytes(1)), CF1, System.currentTimeMillis()));
         ByteBuffer context = current.getColumn(cellname(1)).value();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/db/HintedHandOffTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/HintedHandOffTest.java b/test/unit/org/apache/cassandra/db/HintedHandOffTest.java
index 9bc0724..e0a78a9 100644
--- a/test/unit/org/apache/cassandra/db/HintedHandOffTest.java
+++ b/test/unit/org/apache/cassandra/db/HintedHandOffTest.java
@@ -25,30 +25,44 @@ import java.util.Map;
 import java.util.UUID;
 import java.util.concurrent.TimeUnit;
 
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import com.google.common.collect.Iterators;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.cql3.UntypedResultSet;
 import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy;
 import org.apache.cassandra.db.marshal.Int32Type;
 import org.apache.cassandra.db.marshal.UUIDType;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 
 import static org.junit.Assert.assertEquals;
 import static org.apache.cassandra.cql3.QueryProcessor.executeInternal;
 
-public class HintedHandOffTest extends SchemaLoader
+public class HintedHandOffTest
 {
 
-    public static final String KEYSPACE4 = "Keyspace4";
+    public static final String KEYSPACE4 = "HintedHandOffTest4";
     public static final String STANDARD1_CF = "Standard1";
     public static final String COLUMN1 = "column1";
 
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE4,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE4, STANDARD1_CF));
+    }
+
     // Test compaction of hints column family. It shouldn't remove all columns on compaction.
     @Test
     public void testCompactionOfHintsCF() throws Exception

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/db/KeyCacheTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/KeyCacheTest.java b/test/unit/org/apache/cassandra/db/KeyCacheTest.java
index c0560ab..0d3fa63 100644
--- a/test/unit/org/apache/cassandra/db/KeyCacheTest.java
+++ b/test/unit/org/apache/cassandra/db/KeyCacheTest.java
@@ -26,31 +26,46 @@ import java.util.concurrent.TimeUnit;
 
 import com.google.common.util.concurrent.Uninterruptibles;
 import org.junit.AfterClass;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
 import org.apache.cassandra.cache.KeyCacheKey;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.db.composites.*;
 import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.db.filter.QueryFilter;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.sstable.SSTableReader;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.service.CacheService;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 import static org.junit.Assert.assertEquals;
 
-public class KeyCacheTest extends SchemaLoader
+public class KeyCacheTest
 {
-    private static final String KEYSPACE1 = "KeyCacheSpace";
+    private static final String KEYSPACE1 = "KeyCacheTest1";
     private static final String COLUMN_FAMILY1 = "Standard1";
     private static final String COLUMN_FAMILY2 = "Standard2";
 
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, COLUMN_FAMILY1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, COLUMN_FAMILY2));
+    }
+
     @AfterClass
     public static void cleanup()
     {
-        cleanupSavedCaches();
+        SchemaLoader.cleanupSavedCaches();
     }
 
     @Test
@@ -65,11 +80,11 @@ public class KeyCacheTest extends SchemaLoader
         assertKeyCacheSize(0, KEYSPACE1, COLUMN_FAMILY2);
 
         // insert data and force to disk
-        insertData(KEYSPACE1, COLUMN_FAMILY2, 0, 100);
+        SchemaLoader.insertData(KEYSPACE1, COLUMN_FAMILY2, 0, 100);
         store.forceBlockingFlush();
 
         // populate the cache
-        readData(KEYSPACE1, COLUMN_FAMILY2, 0, 100);
+        SchemaLoader.readData(KEYSPACE1, COLUMN_FAMILY2, 0, 100);
         assertKeyCacheSize(100, KEYSPACE1, COLUMN_FAMILY2);
 
         // really? our caches don't implement the map interface? (hence no .addAll)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/db/KeyCollisionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/KeyCollisionTest.java b/test/unit/org/apache/cassandra/db/KeyCollisionTest.java
index 1869872..23d8abc 100644
--- a/test/unit/org/apache/cassandra/db/KeyCollisionTest.java
+++ b/test/unit/org/apache/cassandra/db/KeyCollisionTest.java
@@ -22,6 +22,7 @@ import java.math.BigInteger;
 import java.nio.ByteBuffer;
 import java.util.*;
 
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
@@ -31,6 +32,8 @@ import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.marshal.IntegerType;
 import org.apache.cassandra.dht.*;
 import org.apache.cassandra.config.*;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.*;
 import static org.apache.cassandra.Util.dk;
@@ -43,12 +46,22 @@ import static org.apache.cassandra.Util.dk;
  * length partitioner that takes the length of the key as token, making
  * collision easy and predictable.
  */
-public class KeyCollisionTest extends SchemaLoader
+public class KeyCollisionTest
 {
     IPartitioner oldPartitioner;
-    private static final String KEYSPACE = "Keyspace1";
+    private static final String KEYSPACE1 = "KeyCollisionTest1";
     private static final String CF = "Standard1";
 
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF));
+    }
+
     protected void setUp()
     {
         oldPartitioner = DatabaseDescriptor.getPartitioner();
@@ -63,7 +76,7 @@ public class KeyCollisionTest extends SchemaLoader
     @Test
     public void testGetSliceWithCollision() throws Exception
     {
-        Keyspace keyspace = Keyspace.open(KEYSPACE);
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF);
         cfs.clearUnsafe();
 
@@ -88,7 +101,7 @@ public class KeyCollisionTest extends SchemaLoader
     private void insert(String key)
     {
         Mutation rm;
-        rm = new Mutation(KEYSPACE, ByteBufferUtil.bytes(key));
+        rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes(key));
         rm.add(CF, Util.cellname("column"), ByteBufferUtil.bytes("asdf"), 0);
         rm.apply();
     }


[2/6] Reduce unit test times due to schema loading

Posted by ty...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java b/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
index 6997162..01ca340 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
@@ -34,11 +34,13 @@ import java.util.concurrent.ScheduledThreadPoolExecutor;
 import java.util.concurrent.ThreadPoolExecutor;
 
 import com.google.common.collect.Sets;
+import org.apache.cassandra.cache.CachingOptions;
+import org.apache.cassandra.config.KSMetaData;
+import org.apache.cassandra.locator.SimpleStrategy;
 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 org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.SchemaLoader;
@@ -75,19 +77,39 @@ import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
 @RunWith(OrderedJUnit4ClassRunner.class)
-public class SSTableReaderTest extends SchemaLoader
+public class SSTableReaderTest
 {
-    private static final Logger logger = LoggerFactory.getLogger(SSTableReaderTest.class);
+    public static final String KEYSPACE1 = "SSTableReaderTest";
+    public static final String CF_STANDARD = "Standard1";
+    public static final String CF_STANDARD2 = "Standard2";
+    public static final String CF_INDEXED = "Indexed1";
+    public static final String CF_STANDARDLOWINDEXINTERVAL = "StandardLowIndexInterval";
 
     static Token t(int i)
     {
         return StorageService.getPartitioner().getToken(ByteBufferUtil.bytes(String.valueOf(i)));
     }
 
+    @BeforeClass
+    public static void defineSchema() throws Exception
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD2),
+                                    SchemaLoader.indexCFMD(KEYSPACE1, CF_INDEXED, true),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARDLOWINDEXINTERVAL)
+                                                .minIndexInterval(8)
+                                                .maxIndexInterval(256)
+                                                .caching(CachingOptions.NONE));
+    }
+
     @Test
     public void testGetPositionsForRanges() throws ExecutionException, InterruptedException
     {
-        Keyspace keyspace = Keyspace.open("Keyspace1");
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
         ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard2");
 
         // insert data and compact to a single sstable
@@ -95,7 +117,7 @@ public class SSTableReaderTest extends SchemaLoader
         for (int j = 0; j < 10; j++)
         {
             ByteBuffer key = ByteBufferUtil.bytes(String.valueOf(j));
-            Mutation rm = new Mutation("Keyspace1", key);
+            Mutation rm = new Mutation(KEYSPACE1, key);
             rm.add("Standard2", cellname("0"), ByteBufferUtil.EMPTY_BYTE_BUFFER, j);
             rm.apply();
         }
@@ -128,7 +150,7 @@ public class SSTableReaderTest extends SchemaLoader
     {
         MmappedSegmentedFile.MAX_SEGMENT_SIZE = 40; // each index entry is ~11 bytes, so this will generate lots of segments
 
-        Keyspace keyspace = Keyspace.open("Keyspace1");
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
         ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard1");
 
         // insert a bunch of data and compact to a single sstable
@@ -136,7 +158,7 @@ public class SSTableReaderTest extends SchemaLoader
         for (int j = 0; j < 100; j += 2)
         {
             ByteBuffer key = ByteBufferUtil.bytes(String.valueOf(j));
-            Mutation rm = new Mutation("Keyspace1", key);
+            Mutation rm = new Mutation(KEYSPACE1, key);
             rm.add("Standard1", cellname("0"), ByteBufferUtil.EMPTY_BYTE_BUFFER, j);
             rm.apply();
         }
@@ -165,13 +187,13 @@ public class SSTableReaderTest extends SchemaLoader
     public void testPersistentStatistics()
     {
 
-        Keyspace keyspace = Keyspace.open("Keyspace1");
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
         ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard1");
 
         for (int j = 0; j < 100; j += 2)
         {
             ByteBuffer key = ByteBufferUtil.bytes(String.valueOf(j));
-            Mutation rm = new Mutation("Keyspace1", key);
+            Mutation rm = new Mutation(KEYSPACE1, key);
             rm.add("Standard1", cellname("0"), ByteBufferUtil.EMPTY_BYTE_BUFFER, j);
             rm.apply();
         }
@@ -190,7 +212,7 @@ public class SSTableReaderTest extends SchemaLoader
     @Test
     public void testGetPositionsForRangesWithKeyCache() throws ExecutionException, InterruptedException
     {
-        Keyspace keyspace = Keyspace.open("Keyspace1");
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
         ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard2");
         CacheService.instance.keyCache.setCapacity(100);
 
@@ -199,7 +221,7 @@ public class SSTableReaderTest extends SchemaLoader
         for (int j = 0; j < 10; j++)
         {
             ByteBuffer key = ByteBufferUtil.bytes(String.valueOf(j));
-            Mutation rm = new Mutation("Keyspace1", key);
+            Mutation rm = new Mutation(KEYSPACE1, key);
             rm.add("Standard2", cellname("0"), ByteBufferUtil.EMPTY_BYTE_BUFFER, j);
             rm.apply();
         }
@@ -225,10 +247,10 @@ public class SSTableReaderTest extends SchemaLoader
     public void testPersistentStatisticsWithSecondaryIndex()
     {
         // Create secondary index and flush to disk
-        Keyspace keyspace = Keyspace.open("Keyspace1");
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
         ColumnFamilyStore store = keyspace.getColumnFamilyStore("Indexed1");
         ByteBuffer key = ByteBufferUtil.bytes(String.valueOf("k1"));
-        Mutation rm = new Mutation("Keyspace1", key);
+        Mutation rm = new Mutation(KEYSPACE1, key);
         rm.add("Indexed1", cellname("birthdate"), ByteBufferUtil.bytes(1L), System.currentTimeMillis());
         rm.apply();
         store.forceBlockingFlush();
@@ -240,7 +262,7 @@ public class SSTableReaderTest extends SchemaLoader
     @Test
     public void testOpeningSSTable() throws Exception
     {
-        String ks = "Keyspace1";
+        String ks = KEYSPACE1;
         String cf = "Standard1";
 
         // clear and create just one sstable for this test
@@ -281,10 +303,10 @@ public class SSTableReaderTest extends SchemaLoader
     @Test
     public void testLoadingSummaryUsesCorrectPartitioner() throws Exception
     {
-        Keyspace keyspace = Keyspace.open("Keyspace1");
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
         ColumnFamilyStore store = keyspace.getColumnFamilyStore("Indexed1");
         ByteBuffer key = ByteBufferUtil.bytes(String.valueOf("k1"));
-        Mutation rm = new Mutation("Keyspace1", key);
+        Mutation rm = new Mutation(KEYSPACE1, key);
         rm.add("Indexed1", cellname("birthdate"), ByteBufferUtil.bytes(1L), System.currentTimeMillis());
         rm.apply();
         store.forceBlockingFlush();
@@ -308,10 +330,10 @@ public class SSTableReaderTest extends SchemaLoader
     @Test
     public void testGetScannerForNoIntersectingRanges()
     {
-        Keyspace keyspace = Keyspace.open("Keyspace1");
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
         ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard1");
         ByteBuffer key = ByteBufferUtil.bytes(String.valueOf("k1"));
-        Mutation rm = new Mutation("Keyspace1", key);
+        Mutation rm = new Mutation(KEYSPACE1, key);
         rm.add("Standard1", cellname("xyz"), ByteBufferUtil.bytes("abc"), 0);
         rm.apply();
         store.forceBlockingFlush();
@@ -328,7 +350,7 @@ public class SSTableReaderTest extends SchemaLoader
     @Test
     public void testGetPositionsForRangesFromTableOpenedForBulkLoading() throws IOException, ExecutionException, InterruptedException
     {
-        Keyspace keyspace = Keyspace.open("Keyspace1");
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
         ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard2");
 
         // insert data and compact to a single sstable. The
@@ -338,7 +360,7 @@ public class SSTableReaderTest extends SchemaLoader
         for (int j = 0; j < 130; j++)
         {
             ByteBuffer key = ByteBufferUtil.bytes(String.valueOf(j));
-            Mutation rm = new Mutation("Keyspace1", key);
+            Mutation rm = new Mutation(KEYSPACE1, key);
             rm.add("Standard2", cellname("0"), ByteBufferUtil.EMPTY_BYTE_BUFFER, j);
             rm.apply();
         }
@@ -364,7 +386,7 @@ public class SSTableReaderTest extends SchemaLoader
     @Test
     public void testIndexSummaryReplacement() throws IOException, ExecutionException, InterruptedException
     {
-        Keyspace keyspace = Keyspace.open("Keyspace1");
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
         final ColumnFamilyStore store = keyspace.getColumnFamilyStore("StandardLowIndexInterval"); // index interval of 8, no key caching
         CompactionManager.instance.disableAutoCompaction();
 
@@ -372,7 +394,7 @@ public class SSTableReaderTest extends SchemaLoader
         for (int j = 0; j < NUM_ROWS; j++)
         {
             ByteBuffer key = ByteBufferUtil.bytes(String.format("%3d", j));
-            Mutation rm = new Mutation("Keyspace1", key);
+            Mutation rm = new Mutation(KEYSPACE1, key);
             rm.add("StandardLowIndexInterval", Util.cellname("0"), ByteBufferUtil.bytes(String.format("%3d", j)), j);
             rm.apply();
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/io/sstable/SSTableScannerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableScannerTest.java b/test/unit/org/apache/cassandra/io/sstable/SSTableScannerTest.java
index ff1a305..681efb69 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableScannerTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableScannerTest.java
@@ -21,10 +21,12 @@ package org.apache.cassandra.io.sstable;
 import java.util.ArrayList;
 import java.util.Collection;
 
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.columniterator.IdentityQueryFilter;
 import org.apache.cassandra.db.compaction.ICompactionScanner;
@@ -32,15 +34,26 @@ import org.apache.cassandra.dht.Bounds;
 import org.apache.cassandra.dht.BytesToken;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 import static org.junit.Assert.*;
 
-public class SSTableScannerTest extends SchemaLoader
+public class SSTableScannerTest
 {
-    public static final String KEYSPACE = "Keyspace1";
+    public static final String KEYSPACE = "SSTableScannerTest";
     public static final String TABLE = "Standard1";
 
+    @BeforeClass
+    public static void defineSchema() throws Exception
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE, TABLE));
+    }
+
     private static String toKey(int key)
     {
         return String.format("%03d", key);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/io/sstable/SSTableSimpleWriterTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableSimpleWriterTest.java b/test/unit/org/apache/cassandra/io/sstable/SSTableSimpleWriterTest.java
index 15980a4..df9c56b 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableSimpleWriterTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableSimpleWriterTest.java
@@ -20,27 +20,43 @@ package org.apache.cassandra.io.sstable;
 
 import java.io.File;
 
-import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.dht.IPartitioner;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
+import org.apache.cassandra.config.KSMetaData;
+import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.marshal.IntegerType;
+import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.service.StorageService;
 import static org.apache.cassandra.utils.ByteBufferUtil.bytes;
 import static org.apache.cassandra.utils.ByteBufferUtil.toInt;
 
-public class SSTableSimpleWriterTest extends SchemaLoader
+public class SSTableSimpleWriterTest
 {
+    public static final String KEYSPACE = "SSTableSimpleWriterTest";
+    public static final String CF_STANDARDINT = "StandardInteger1";
+
+    @BeforeClass
+    public static void defineSchema() throws Exception
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE, CF_STANDARDINT));
+    }
+
     @Test
     public void testSSTableSimpleUnsortedWriter() throws Exception
     {
         final int INC = 5;
         final int NBCOL = 10;
 
-        String keyspaceName = "Keyspace1";
+        String keyspaceName = KEYSPACE;
         String cfname = "StandardInteger1";
 
         Keyspace t = Keyspace.open(keyspaceName); // make sure we create the directory

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/io/sstable/SSTableUtils.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableUtils.java b/test/unit/org/apache/cassandra/io/sstable/SSTableUtils.java
index d39f968..32d07ac 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableUtils.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableUtils.java
@@ -37,9 +37,21 @@ public class SSTableUtils
     public static String KEYSPACENAME = "Keyspace1";
     public static String CFNAME = "Standard1";
 
+    public SSTableUtils(String ksname, String cfname)
+    {
+        KEYSPACENAME = ksname;
+        CFNAME = cfname;
+    }
+
+    /**/
     public static ColumnFamily createCF(long mfda, int ldt, Cell... cols)
     {
-        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(KEYSPACENAME, CFNAME);
+        return createCF(KEYSPACENAME, CFNAME, mfda, ldt, cols);
+    }
+
+    public static ColumnFamily createCF(String ksname, String cfname, long mfda, int ldt, Cell... cols)
+    {
+        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(ksname, cfname);
         cf.delete(new DeletionInfo(mfda, ldt));
         for (Cell col : cols)
             cf.addColumn(col);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/locator/OldNetworkTopologyStrategyTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/locator/OldNetworkTopologyStrategyTest.java b/test/unit/org/apache/cassandra/locator/OldNetworkTopologyStrategyTest.java
index 4ae21c0..517167f 100644
--- a/test/unit/org/apache/cassandra/locator/OldNetworkTopologyStrategyTest.java
+++ b/test/unit/org/apache/cassandra/locator/OldNetworkTopologyStrategyTest.java
@@ -31,7 +31,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
-import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.dht.BigIntegerToken;
 import org.apache.cassandra.dht.Range;
@@ -41,7 +40,7 @@ import org.apache.cassandra.utils.Pair;
 import org.junit.Before;
 import org.junit.Test;
 
-public class OldNetworkTopologyStrategyTest extends SchemaLoader
+public class OldNetworkTopologyStrategyTest
 {
     private List<Token> keyTokens;
     private TokenMetadata tmd;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/locator/ReplicationStrategyEndpointCacheTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/locator/ReplicationStrategyEndpointCacheTest.java b/test/unit/org/apache/cassandra/locator/ReplicationStrategyEndpointCacheTest.java
index 91f6111..2a41e50 100644
--- a/test/unit/org/apache/cassandra/locator/ReplicationStrategyEndpointCacheTest.java
+++ b/test/unit/org/apache/cassandra/locator/ReplicationStrategyEndpointCacheTest.java
@@ -22,28 +22,39 @@ package org.apache.cassandra.locator;
 import java.net.InetAddress;
 import java.util.*;
 
-import org.apache.cassandra.db.Keyspace;
-
 import org.apache.commons.lang3.StringUtils;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.KSMetaData;
+import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.dht.BigIntegerToken;
 import org.apache.cassandra.dht.Token;
 
-public class ReplicationStrategyEndpointCacheTest extends SchemaLoader
+public class ReplicationStrategyEndpointCacheTest
 {
     private TokenMetadata tmd;
     private Token searchToken;
     private AbstractReplicationStrategy strategy;
+    public static final String KEYSPACE = "ReplicationStrategyEndpointCacheTest";
+
+    @BeforeClass
+    public static void defineSchema() throws Exception
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(5));
+    }
 
     public void setup(Class stratClass, Map<String, String> strategyOptions) throws Exception
     {
         tmd = new TokenMetadata();
         searchToken = new BigIntegerToken(String.valueOf(15));
 
-        strategy = getStrategyWithNewTokenMetadata(Keyspace.open("Keyspace3").getReplicationStrategy(), tmd);
+        strategy = getStrategyWithNewTokenMetadata(Keyspace.open(KEYSPACE).getReplicationStrategy(), tmd);
 
         tmd.updateNormalToken(new BigIntegerToken(String.valueOf(10)), InetAddress.getByName("127.0.0.1"));
         tmd.updateNormalToken(new BigIntegerToken(String.valueOf(20)), InetAddress.getByName("127.0.0.2"));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/locator/SimpleStrategyTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/locator/SimpleStrategyTest.java b/test/unit/org/apache/cassandra/locator/SimpleStrategyTest.java
index d4978ae..87c8613 100644
--- a/test/unit/org/apache/cassandra/locator/SimpleStrategyTest.java
+++ b/test/unit/org/apache/cassandra/locator/SimpleStrategyTest.java
@@ -26,25 +26,37 @@ import java.util.Collection;
 import java.util.HashSet;
 import java.util.List;
 
-import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.service.PendingRangeCalculatorService;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.config.KSMetaData;
+import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.dht.*;
+import org.apache.cassandra.service.PendingRangeCalculatorService;
 import org.apache.cassandra.service.StorageServiceAccessor;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 import static org.junit.Assert.*;
 
-public class SimpleStrategyTest extends SchemaLoader
+public class SimpleStrategyTest
 {
+    public static final String KEYSPACE1 = "SimpleStrategyTest";
+
+    @BeforeClass
+    public static void defineSchema() throws Exception
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1));
+    }
+
     @Test
     public void tryValidKeyspace()
     {
-        assert Keyspace.open("Keyspace1").getReplicationStrategy() != null;
+        assert Keyspace.open(KEYSPACE1).getReplicationStrategy() != null;
     }
 
     @Test

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/repair/DifferencerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/repair/DifferencerTest.java b/test/unit/org/apache/cassandra/repair/DifferencerTest.java
index bc0f0de..e1ff26e 100644
--- a/test/unit/org/apache/cassandra/repair/DifferencerTest.java
+++ b/test/unit/org/apache/cassandra/repair/DifferencerTest.java
@@ -24,15 +24,18 @@ import java.util.Set;
 import java.util.UUID;
 
 import org.junit.After;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.dht.Murmur3Partitioner;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.net.MessageIn;
 import org.apache.cassandra.net.MessageOut;
 import org.apache.cassandra.net.MessagingService;
@@ -46,9 +49,21 @@ import org.apache.cassandra.utils.MerkleTree;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
-public class DifferencerTest extends SchemaLoader
+public class DifferencerTest
 {
     private static final IPartitioner partirioner = new Murmur3Partitioner();
+    public static final String KEYSPACE1 = "DifferencerTest";
+    public static final String CF_STANDARD = "Standard1";
+
+    @BeforeClass
+    public static void defineSchema() throws Exception
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD));
+    }
 
     @After
     public void tearDown()
@@ -86,7 +101,7 @@ public class DifferencerTest extends SchemaLoader
             }
         });
         Range<Token> range = new Range<>(partirioner.getMinimumToken(), partirioner.getRandomToken());
-        RepairJobDesc desc = new RepairJobDesc(UUID.randomUUID(), UUID.randomUUID(), "Keyspace1", "Standard1", range);
+        RepairJobDesc desc = new RepairJobDesc(UUID.randomUUID(), UUID.randomUUID(), KEYSPACE1, "Standard1", range);
 
         MerkleTree tree1 = createInitialTree(desc);
         MerkleTree tree2 = createInitialTree(desc);
@@ -106,12 +121,12 @@ public class DifferencerTest extends SchemaLoader
     {
         Range<Token> range = new Range<>(partirioner.getMinimumToken(), partirioner.getRandomToken());
         UUID parentRepairSession = UUID.randomUUID();
-        Keyspace keyspace = Keyspace.open("Keyspace1");
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("Standard1");
 
         ActiveRepairService.instance.registerParentRepairSession(parentRepairSession, Arrays.asList(cfs), Arrays.asList(range));
 
-        RepairJobDesc desc = new RepairJobDesc(parentRepairSession, UUID.randomUUID(), "Keyspace1", "Standard1", range);
+        RepairJobDesc desc = new RepairJobDesc(parentRepairSession, UUID.randomUUID(), KEYSPACE1, "Standard1", range);
 
         MerkleTree tree1 = createInitialTree(desc);
         MerkleTree tree2 = createInitialTree(desc);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/repair/ValidatorTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/repair/ValidatorTest.java b/test/unit/org/apache/cassandra/repair/ValidatorTest.java
index c3ce810..e18ce9b 100644
--- a/test/unit/org/apache/cassandra/repair/ValidatorTest.java
+++ b/test/unit/org/apache/cassandra/repair/ValidatorTest.java
@@ -23,9 +23,11 @@ import java.security.MessageDigest;
 import java.util.UUID;
 
 import org.junit.After;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.db.BufferDecoratedKey;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.DecoratedKey;
@@ -37,6 +39,7 @@ import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.io.sstable.ColumnStats;
 import org.apache.cassandra.io.util.DataOutputPlus;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.net.MessageIn;
 import org.apache.cassandra.net.MessageOut;
 import org.apache.cassandra.net.MessagingService;
@@ -50,12 +53,22 @@ import org.apache.cassandra.utils.concurrent.SimpleCondition;
 
 import static org.junit.Assert.*;
 
-public class ValidatorTest extends SchemaLoader
+public class ValidatorTest
 {
-    private final String keyspace = "Keyspace1";
-    private final String columnFamily = "Standard1";
+    private static final String keyspace = "ValidatorTest";
+    private static final String columnFamily = "Standard1";
     private final IPartitioner partitioner = StorageService.getPartitioner();
 
+    @BeforeClass
+    public static void defineSchema() throws Exception
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(keyspace,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(keyspace, columnFamily));
+    }
+
     @After
     public void tearDown()
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/service/AntiEntropyServiceCounterTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/AntiEntropyServiceCounterTest.java b/test/unit/org/apache/cassandra/service/AntiEntropyServiceCounterTest.java
index 4a82183..3f1c3aa 100644
--- a/test/unit/org/apache/cassandra/service/AntiEntropyServiceCounterTest.java
+++ b/test/unit/org/apache/cassandra/service/AntiEntropyServiceCounterTest.java
@@ -32,8 +32,8 @@ public class AntiEntropyServiceCounterTest extends AntiEntropyServiceTestAbstrac
 {
     public void init()
     {
-        keyspaceName = "Keyspace5";
-        cfname    = "Counter1";
+        keyspaceName = AntiEntropyServiceTestAbstract.KEYSPACE5;
+        cfname = AntiEntropyServiceTestAbstract.CF_COUNTER;;
     }
 
     public List<IMutation> getWriteData()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/service/AntiEntropyServiceStandardTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/AntiEntropyServiceStandardTest.java b/test/unit/org/apache/cassandra/service/AntiEntropyServiceStandardTest.java
index f4025cc..c83a730 100644
--- a/test/unit/org/apache/cassandra/service/AntiEntropyServiceStandardTest.java
+++ b/test/unit/org/apache/cassandra/service/AntiEntropyServiceStandardTest.java
@@ -31,8 +31,8 @@ public class AntiEntropyServiceStandardTest extends AntiEntropyServiceTestAbstra
 {
     public void init()
     {
-        keyspaceName = "Keyspace5";
-        cfname    = "Standard1";
+        keyspaceName = AntiEntropyServiceStandardTest.KEYSPACE5;
+        cfname    = AntiEntropyServiceStandardTest.CF_STANDRAD1;
     }
 
     public List<IMutation> getWriteData()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/service/AntiEntropyServiceTestAbstract.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/AntiEntropyServiceTestAbstract.java b/test/unit/org/apache/cassandra/service/AntiEntropyServiceTestAbstract.java
index ac39de6..efa47c1 100644
--- a/test/unit/org/apache/cassandra/service/AntiEntropyServiceTestAbstract.java
+++ b/test/unit/org/apache/cassandra/service/AntiEntropyServiceTestAbstract.java
@@ -27,9 +27,11 @@ import java.util.concurrent.TimeUnit;
 import com.google.common.collect.Sets;
 import org.junit.After;
 import org.junit.Before;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.concurrent.Stage;
 import org.apache.cassandra.concurrent.StageManager;
 import org.apache.cassandra.config.DatabaseDescriptor;
@@ -38,16 +40,18 @@ import org.apache.cassandra.db.IMutation;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.gms.Gossiper;
 import org.apache.cassandra.locator.AbstractReplicationStrategy;
 import org.apache.cassandra.locator.TokenMetadata;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.repair.RepairJobDesc;
 import org.apache.cassandra.utils.FBUtilities;
 
 import static org.junit.Assert.assertEquals;
 
-public abstract class AntiEntropyServiceTestAbstract extends SchemaLoader
+public abstract class AntiEntropyServiceTestAbstract
 {
     // keyspace and column family to test against
     public ActiveRepairService aes;
@@ -66,11 +70,27 @@ public abstract class AntiEntropyServiceTestAbstract extends SchemaLoader
 
     public abstract List<IMutation> getWriteData();
 
+    public static final String KEYSPACE5 = "Keyspace5";
+    public static final String CF_STANDRAD1 = "Standard1";
+    public static final String CF_COUNTER = "Counter1";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE5,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(2),
+                                    SchemaLoader.standardCFMD(KEYSPACE5, CF_COUNTER),
+                                    SchemaLoader.standardCFMD(KEYSPACE5, CF_STANDRAD1));
+    }
+
     @Before
     public void prepare() throws Exception
     {
         if (!initialized)
         {
+            SchemaLoader.startGossiper();
             initialized = true;
 
             init();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/service/EmbeddedCassandraServiceTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/EmbeddedCassandraServiceTest.java b/test/unit/org/apache/cassandra/service/EmbeddedCassandraServiceTest.java
index 24b5a74..1437886 100644
--- a/test/unit/org/apache/cassandra/service/EmbeddedCassandraServiceTest.java
+++ b/test/unit/org/apache/cassandra/service/EmbeddedCassandraServiceTest.java
@@ -27,7 +27,8 @@ import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.config.KSMetaData;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.thrift.*;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.thrift.TException;
@@ -47,10 +48,23 @@ import static org.junit.Assert.assertNotNull;
  * Tests connect to localhost:9160 when the embedded server is running.
  *
  */
-public class EmbeddedCassandraServiceTest extends SchemaLoader
+public class EmbeddedCassandraServiceTest
 {
 
     private static EmbeddedCassandraService cassandra;
+    private static final String KEYSPACE1 = "EmbeddedCassandraServiceTest";
+    private static final String CF_STANDARD = "Standard1";
+
+    @BeforeClass
+    public static void defineSchema() throws Exception
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD));
+        setup();
+    }
 
     /**
      * Set embedded cassandra up and spawn it in a new thread.
@@ -59,10 +73,9 @@ public class EmbeddedCassandraServiceTest extends SchemaLoader
      * @throws IOException
      * @throws InterruptedException
      */
-    @BeforeClass
     public static void setup() throws TTransportException, IOException, InterruptedException
     {
-        Schema.instance.clear(); // Schema are now written on disk and will be reloaded
+        // unique ks / cfs mean no need to clear the schema
         cassandra = new EmbeddedCassandraService();
         cassandra.start();
     }
@@ -72,7 +85,7 @@ public class EmbeddedCassandraServiceTest extends SchemaLoader
     throws AuthenticationException, AuthorizationException, InvalidRequestException, UnavailableException, TimedOutException, TException, NotFoundException, CharacterCodingException
     {
         Cassandra.Client client = getClient();
-        client.set_keyspace("Keyspace1");
+        client.set_keyspace(KEYSPACE1);
 
         ByteBuffer key_user_id = ByteBufferUtil.bytes("1");
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/service/LeaveAndBootstrapTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/LeaveAndBootstrapTest.java b/test/unit/org/apache/cassandra/service/LeaveAndBootstrapTest.java
index eef8c86..bf34ba6 100644
--- a/test/unit/org/apache/cassandra/service/LeaveAndBootstrapTest.java
+++ b/test/unit/org/apache/cassandra/service/LeaveAndBootstrapTest.java
@@ -23,9 +23,6 @@ import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.util.*;
 
-import org.apache.cassandra.db.SystemKeyspace;
-import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.config.Schema;
 import org.junit.Test;
 
 import static org.junit.Assert.*;
@@ -36,6 +33,8 @@ import com.google.common.collect.Multimap;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
 import org.apache.cassandra.config.KSMetaData;
+import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.db.SystemKeyspace;
 import org.apache.cassandra.dht.*;
 import org.apache.cassandra.gms.ApplicationState;
 import org.apache.cassandra.gms.Gossiper;
@@ -48,19 +47,23 @@ public class LeaveAndBootstrapTest
 {
     private static final IPartitioner partitioner = new RandomPartitioner();
     private static IPartitioner oldPartitioner;
+    private static final String KEYSPACE1 = "LeaveAndBootstrapTestKeyspace1";
+    private static final String KEYSPACE2 = "LeaveAndBootstrapTestKeyspace2";
+    private static final String KEYSPACE3 = "LeaveAndBootstrapTestKeyspace3";
+    private static final String KEYSPACE4 = "LeaveAndBootstrapTestKeyspace4";
 
     @BeforeClass
-    public static void setup() throws ConfigurationException
+    public static void defineSchema() throws Exception
     {
         oldPartitioner = StorageService.instance.setPartitionerUnsafe(partitioner);
         SchemaLoader.loadSchema();
+        SchemaLoader.schemaDefinition("LeaveAndBootstrapTest");
     }
 
     @AfterClass
     public static void tearDown()
     {
         StorageService.instance.setPartitionerUnsafe(oldPartitioner);
-        SchemaLoader.stopGossiper();
     }
 
     /**
@@ -185,55 +188,55 @@ public class LeaveAndBootstrapTest
         Map<String, AbstractReplicationStrategy> keyspaceStrategyMap = new HashMap<String, AbstractReplicationStrategy>();
         for (int i=1; i<=4; i++)
         {
-            keyspaceStrategyMap.put("Keyspace" + i, getStrategy("Keyspace" + i, tmd));
+            keyspaceStrategyMap.put("LeaveAndBootstrapTestKeyspace" + i, getStrategy("LeaveAndBootstrapTestKeyspace" + i, tmd));
         }
 
         // pre-calculate the results.
         Map<String, Multimap<Token, InetAddress>> expectedEndpoints = new HashMap<String, Multimap<Token, InetAddress>>();
-        expectedEndpoints.put("Keyspace1", HashMultimap.<Token, InetAddress>create());
-        expectedEndpoints.get("Keyspace1").putAll(new BigIntegerToken("5"), makeAddrs("127.0.0.2"));
-        expectedEndpoints.get("Keyspace1").putAll(new BigIntegerToken("15"), makeAddrs("127.0.0.3"));
-        expectedEndpoints.get("Keyspace1").putAll(new BigIntegerToken("25"), makeAddrs("127.0.0.4"));
-        expectedEndpoints.get("Keyspace1").putAll(new BigIntegerToken("35"), makeAddrs("127.0.0.5"));
-        expectedEndpoints.get("Keyspace1").putAll(new BigIntegerToken("45"), makeAddrs("127.0.0.6"));
-        expectedEndpoints.get("Keyspace1").putAll(new BigIntegerToken("55"), makeAddrs("127.0.0.7", "127.0.0.8", "127.0.1.1"));
-        expectedEndpoints.get("Keyspace1").putAll(new BigIntegerToken("65"), makeAddrs("127.0.0.8"));
-        expectedEndpoints.get("Keyspace1").putAll(new BigIntegerToken("75"), makeAddrs("127.0.0.9", "127.0.1.2", "127.0.0.1"));
-        expectedEndpoints.get("Keyspace1").putAll(new BigIntegerToken("85"), makeAddrs("127.0.0.10", "127.0.0.1"));
-        expectedEndpoints.get("Keyspace1").putAll(new BigIntegerToken("95"), makeAddrs("127.0.0.1"));
-        expectedEndpoints.put("Keyspace2", HashMultimap.<Token, InetAddress>create());
-        expectedEndpoints.get("Keyspace2").putAll(new BigIntegerToken("5"), makeAddrs("127.0.0.2"));
-        expectedEndpoints.get("Keyspace2").putAll(new BigIntegerToken("15"), makeAddrs("127.0.0.3"));
-        expectedEndpoints.get("Keyspace2").putAll(new BigIntegerToken("25"), makeAddrs("127.0.0.4"));
-        expectedEndpoints.get("Keyspace2").putAll(new BigIntegerToken("35"), makeAddrs("127.0.0.5"));
-        expectedEndpoints.get("Keyspace2").putAll(new BigIntegerToken("45"), makeAddrs("127.0.0.6"));
-        expectedEndpoints.get("Keyspace2").putAll(new BigIntegerToken("55"), makeAddrs("127.0.0.7", "127.0.0.8", "127.0.1.1"));
-        expectedEndpoints.get("Keyspace2").putAll(new BigIntegerToken("65"), makeAddrs("127.0.0.8"));
-        expectedEndpoints.get("Keyspace2").putAll(new BigIntegerToken("75"), makeAddrs("127.0.0.9", "127.0.1.2", "127.0.0.1"));
-        expectedEndpoints.get("Keyspace2").putAll(new BigIntegerToken("85"), makeAddrs("127.0.0.10", "127.0.0.1"));
-        expectedEndpoints.get("Keyspace2").putAll(new BigIntegerToken("95"), makeAddrs("127.0.0.1"));
-        expectedEndpoints.put("Keyspace3", HashMultimap.<Token, InetAddress>create());
-        expectedEndpoints.get("Keyspace3").putAll(new BigIntegerToken("5"), makeAddrs("127.0.0.2", "127.0.0.3", "127.0.0.4", "127.0.0.5", "127.0.0.6"));
-        expectedEndpoints.get("Keyspace3").putAll(new BigIntegerToken("15"), makeAddrs("127.0.0.3", "127.0.0.4", "127.0.0.5", "127.0.0.6", "127.0.0.7", "127.0.1.1", "127.0.0.8"));
-        expectedEndpoints.get("Keyspace3").putAll(new BigIntegerToken("25"), makeAddrs("127.0.0.4", "127.0.0.5", "127.0.0.6", "127.0.0.7", "127.0.0.8", "127.0.1.2", "127.0.0.1", "127.0.1.1"));
-        expectedEndpoints.get("Keyspace3").putAll(new BigIntegerToken("35"), makeAddrs("127.0.0.5", "127.0.0.6", "127.0.0.7", "127.0.0.8", "127.0.0.9", "127.0.1.2", "127.0.0.1", "127.0.0.2", "127.0.1.1"));
-        expectedEndpoints.get("Keyspace3").putAll(new BigIntegerToken("45"), makeAddrs("127.0.0.6", "127.0.0.7", "127.0.0.8", "127.0.0.9", "127.0.0.10", "127.0.1.2", "127.0.0.1", "127.0.0.2", "127.0.1.1", "127.0.0.3"));
-        expectedEndpoints.get("Keyspace3").putAll(new BigIntegerToken("55"), makeAddrs("127.0.0.7", "127.0.0.8", "127.0.0.9", "127.0.0.10", "127.0.0.1", "127.0.0.2", "127.0.0.3", "127.0.0.4", "127.0.1.1", "127.0.1.2"));
-        expectedEndpoints.get("Keyspace3").putAll(new BigIntegerToken("65"), makeAddrs("127.0.0.8", "127.0.0.9", "127.0.0.10", "127.0.0.1", "127.0.0.2", "127.0.1.2", "127.0.0.3", "127.0.0.4"));
-        expectedEndpoints.get("Keyspace3").putAll(new BigIntegerToken("75"), makeAddrs("127.0.0.9", "127.0.0.10", "127.0.0.1", "127.0.0.2", "127.0.0.3", "127.0.1.2", "127.0.0.4", "127.0.0.5"));
-        expectedEndpoints.get("Keyspace3").putAll(new BigIntegerToken("85"), makeAddrs("127.0.0.10", "127.0.0.1", "127.0.0.2", "127.0.0.3", "127.0.0.4", "127.0.0.5"));
-        expectedEndpoints.get("Keyspace3").putAll(new BigIntegerToken("95"), makeAddrs("127.0.0.1", "127.0.0.2", "127.0.0.3", "127.0.0.4", "127.0.0.5"));
-        expectedEndpoints.put("Keyspace4", HashMultimap.<Token, InetAddress>create());
-        expectedEndpoints.get("Keyspace4").putAll(new BigIntegerToken("5"), makeAddrs("127.0.0.2", "127.0.0.3", "127.0.0.4"));
-        expectedEndpoints.get("Keyspace4").putAll(new BigIntegerToken("15"), makeAddrs("127.0.0.3", "127.0.0.4", "127.0.0.5"));
-        expectedEndpoints.get("Keyspace4").putAll(new BigIntegerToken("25"), makeAddrs("127.0.0.4", "127.0.0.5", "127.0.0.6"));
-        expectedEndpoints.get("Keyspace4").putAll(new BigIntegerToken("35"), makeAddrs("127.0.0.5", "127.0.0.6", "127.0.0.7", "127.0.1.1", "127.0.0.8"));
-        expectedEndpoints.get("Keyspace4").putAll(new BigIntegerToken("45"), makeAddrs("127.0.0.6", "127.0.0.7", "127.0.0.8", "127.0.1.2", "127.0.0.1", "127.0.1.1"));
-        expectedEndpoints.get("Keyspace4").putAll(new BigIntegerToken("55"), makeAddrs("127.0.0.7", "127.0.0.8", "127.0.0.9", "127.0.0.1", "127.0.0.2", "127.0.1.1", "127.0.1.2"));
-        expectedEndpoints.get("Keyspace4").putAll(new BigIntegerToken("65"), makeAddrs("127.0.0.8", "127.0.0.9", "127.0.0.10", "127.0.1.2", "127.0.0.1", "127.0.0.2"));
-        expectedEndpoints.get("Keyspace4").putAll(new BigIntegerToken("75"), makeAddrs("127.0.0.9", "127.0.0.10", "127.0.0.1", "127.0.1.2", "127.0.0.2", "127.0.0.3"));
-        expectedEndpoints.get("Keyspace4").putAll(new BigIntegerToken("85"), makeAddrs("127.0.0.10", "127.0.0.1", "127.0.0.2", "127.0.0.3"));
-        expectedEndpoints.get("Keyspace4").putAll(new BigIntegerToken("95"), makeAddrs("127.0.0.1", "127.0.0.2", "127.0.0.3"));
+        expectedEndpoints.put(KEYSPACE1, HashMultimap.<Token, InetAddress>create());
+        expectedEndpoints.get(KEYSPACE1).putAll(new BigIntegerToken("5"), makeAddrs("127.0.0.2"));
+        expectedEndpoints.get(KEYSPACE1).putAll(new BigIntegerToken("15"), makeAddrs("127.0.0.3"));
+        expectedEndpoints.get(KEYSPACE1).putAll(new BigIntegerToken("25"), makeAddrs("127.0.0.4"));
+        expectedEndpoints.get(KEYSPACE1).putAll(new BigIntegerToken("35"), makeAddrs("127.0.0.5"));
+        expectedEndpoints.get(KEYSPACE1).putAll(new BigIntegerToken("45"), makeAddrs("127.0.0.6"));
+        expectedEndpoints.get(KEYSPACE1).putAll(new BigIntegerToken("55"), makeAddrs("127.0.0.7", "127.0.0.8", "127.0.1.1"));
+        expectedEndpoints.get(KEYSPACE1).putAll(new BigIntegerToken("65"), makeAddrs("127.0.0.8"));
+        expectedEndpoints.get(KEYSPACE1).putAll(new BigIntegerToken("75"), makeAddrs("127.0.0.9", "127.0.1.2", "127.0.0.1"));
+        expectedEndpoints.get(KEYSPACE1).putAll(new BigIntegerToken("85"), makeAddrs("127.0.0.10", "127.0.0.1"));
+        expectedEndpoints.get(KEYSPACE1).putAll(new BigIntegerToken("95"), makeAddrs("127.0.0.1"));
+        expectedEndpoints.put(KEYSPACE2, HashMultimap.<Token, InetAddress>create());
+        expectedEndpoints.get(KEYSPACE2).putAll(new BigIntegerToken("5"), makeAddrs("127.0.0.2"));
+        expectedEndpoints.get(KEYSPACE2).putAll(new BigIntegerToken("15"), makeAddrs("127.0.0.3"));
+        expectedEndpoints.get(KEYSPACE2).putAll(new BigIntegerToken("25"), makeAddrs("127.0.0.4"));
+        expectedEndpoints.get(KEYSPACE2).putAll(new BigIntegerToken("35"), makeAddrs("127.0.0.5"));
+        expectedEndpoints.get(KEYSPACE2).putAll(new BigIntegerToken("45"), makeAddrs("127.0.0.6"));
+        expectedEndpoints.get(KEYSPACE2).putAll(new BigIntegerToken("55"), makeAddrs("127.0.0.7", "127.0.0.8", "127.0.1.1"));
+        expectedEndpoints.get(KEYSPACE2).putAll(new BigIntegerToken("65"), makeAddrs("127.0.0.8"));
+        expectedEndpoints.get(KEYSPACE2).putAll(new BigIntegerToken("75"), makeAddrs("127.0.0.9", "127.0.1.2", "127.0.0.1"));
+        expectedEndpoints.get(KEYSPACE2).putAll(new BigIntegerToken("85"), makeAddrs("127.0.0.10", "127.0.0.1"));
+        expectedEndpoints.get(KEYSPACE2).putAll(new BigIntegerToken("95"), makeAddrs("127.0.0.1"));
+        expectedEndpoints.put(KEYSPACE3, HashMultimap.<Token, InetAddress>create());
+        expectedEndpoints.get(KEYSPACE3).putAll(new BigIntegerToken("5"), makeAddrs("127.0.0.2", "127.0.0.3", "127.0.0.4", "127.0.0.5", "127.0.0.6"));
+        expectedEndpoints.get(KEYSPACE3).putAll(new BigIntegerToken("15"), makeAddrs("127.0.0.3", "127.0.0.4", "127.0.0.5", "127.0.0.6", "127.0.0.7", "127.0.1.1", "127.0.0.8"));
+        expectedEndpoints.get(KEYSPACE3).putAll(new BigIntegerToken("25"), makeAddrs("127.0.0.4", "127.0.0.5", "127.0.0.6", "127.0.0.7", "127.0.0.8", "127.0.1.2", "127.0.0.1", "127.0.1.1"));
+        expectedEndpoints.get(KEYSPACE3).putAll(new BigIntegerToken("35"), makeAddrs("127.0.0.5", "127.0.0.6", "127.0.0.7", "127.0.0.8", "127.0.0.9", "127.0.1.2", "127.0.0.1", "127.0.0.2", "127.0.1.1"));
+        expectedEndpoints.get(KEYSPACE3).putAll(new BigIntegerToken("45"), makeAddrs("127.0.0.6", "127.0.0.7", "127.0.0.8", "127.0.0.9", "127.0.0.10", "127.0.1.2", "127.0.0.1", "127.0.0.2", "127.0.1.1", "127.0.0.3"));
+        expectedEndpoints.get(KEYSPACE3).putAll(new BigIntegerToken("55"), makeAddrs("127.0.0.7", "127.0.0.8", "127.0.0.9", "127.0.0.10", "127.0.0.1", "127.0.0.2", "127.0.0.3", "127.0.0.4", "127.0.1.1", "127.0.1.2"));
+        expectedEndpoints.get(KEYSPACE3).putAll(new BigIntegerToken("65"), makeAddrs("127.0.0.8", "127.0.0.9", "127.0.0.10", "127.0.0.1", "127.0.0.2", "127.0.1.2", "127.0.0.3", "127.0.0.4"));
+        expectedEndpoints.get(KEYSPACE3).putAll(new BigIntegerToken("75"), makeAddrs("127.0.0.9", "127.0.0.10", "127.0.0.1", "127.0.0.2", "127.0.0.3", "127.0.1.2", "127.0.0.4", "127.0.0.5"));
+        expectedEndpoints.get(KEYSPACE3).putAll(new BigIntegerToken("85"), makeAddrs("127.0.0.10", "127.0.0.1", "127.0.0.2", "127.0.0.3", "127.0.0.4", "127.0.0.5"));
+        expectedEndpoints.get(KEYSPACE3).putAll(new BigIntegerToken("95"), makeAddrs("127.0.0.1", "127.0.0.2", "127.0.0.3", "127.0.0.4", "127.0.0.5"));
+        expectedEndpoints.put(KEYSPACE4, HashMultimap.<Token, InetAddress>create());
+        expectedEndpoints.get(KEYSPACE4).putAll(new BigIntegerToken("5"), makeAddrs("127.0.0.2", "127.0.0.3", "127.0.0.4"));
+        expectedEndpoints.get(KEYSPACE4).putAll(new BigIntegerToken("15"), makeAddrs("127.0.0.3", "127.0.0.4", "127.0.0.5"));
+        expectedEndpoints.get(KEYSPACE4).putAll(new BigIntegerToken("25"), makeAddrs("127.0.0.4", "127.0.0.5", "127.0.0.6"));
+        expectedEndpoints.get(KEYSPACE4).putAll(new BigIntegerToken("35"), makeAddrs("127.0.0.5", "127.0.0.6", "127.0.0.7", "127.0.1.1", "127.0.0.8"));
+        expectedEndpoints.get(KEYSPACE4).putAll(new BigIntegerToken("45"), makeAddrs("127.0.0.6", "127.0.0.7", "127.0.0.8", "127.0.1.2", "127.0.0.1", "127.0.1.1"));
+        expectedEndpoints.get(KEYSPACE4).putAll(new BigIntegerToken("55"), makeAddrs("127.0.0.7", "127.0.0.8", "127.0.0.9", "127.0.0.1", "127.0.0.2", "127.0.1.1", "127.0.1.2"));
+        expectedEndpoints.get(KEYSPACE4).putAll(new BigIntegerToken("65"), makeAddrs("127.0.0.8", "127.0.0.9", "127.0.0.10", "127.0.1.2", "127.0.0.1", "127.0.0.2"));
+        expectedEndpoints.get(KEYSPACE4).putAll(new BigIntegerToken("75"), makeAddrs("127.0.0.9", "127.0.0.10", "127.0.0.1", "127.0.1.2", "127.0.0.2", "127.0.0.3"));
+        expectedEndpoints.get(KEYSPACE4).putAll(new BigIntegerToken("85"), makeAddrs("127.0.0.10", "127.0.0.1", "127.0.0.2", "127.0.0.3"));
+        expectedEndpoints.get(KEYSPACE4).putAll(new BigIntegerToken("95"), makeAddrs("127.0.0.1", "127.0.0.2", "127.0.0.3"));
 
         PendingRangeCalculatorService.instance.blockUntilFinished();
 
@@ -338,24 +341,24 @@ public class LeaveAndBootstrapTest
         ss.onChange(boot1, ApplicationState.STATUS, valueFactory.normal(Collections.singleton(keyTokens.get(5))));
 
         // adjust precalcuated results.  this changes what the epected endpoints are.
-        expectedEndpoints.get("Keyspace1").get(new BigIntegerToken("55")).removeAll(makeAddrs("127.0.0.7", "127.0.0.8"));
-        expectedEndpoints.get("Keyspace1").get(new BigIntegerToken("85")).removeAll(makeAddrs("127.0.0.10"));
-        expectedEndpoints.get("Keyspace2").get(new BigIntegerToken("55")).removeAll(makeAddrs("127.0.0.7", "127.0.0.8"));
-        expectedEndpoints.get("Keyspace2").get(new BigIntegerToken("85")).removeAll(makeAddrs("127.0.0.10"));
-        expectedEndpoints.get("Keyspace3").get(new BigIntegerToken("15")).removeAll(makeAddrs("127.0.0.7", "127.0.0.8"));
-        expectedEndpoints.get("Keyspace3").get(new BigIntegerToken("25")).removeAll(makeAddrs("127.0.0.7", "127.0.1.2", "127.0.0.1"));
-        expectedEndpoints.get("Keyspace3").get(new BigIntegerToken("35")).removeAll(makeAddrs("127.0.0.7", "127.0.0.2"));
-        expectedEndpoints.get("Keyspace3").get(new BigIntegerToken("45")).removeAll(makeAddrs("127.0.0.7", "127.0.0.10", "127.0.0.3"));
-        expectedEndpoints.get("Keyspace3").get(new BigIntegerToken("55")).removeAll(makeAddrs("127.0.0.7", "127.0.0.10", "127.0.0.4"));
-        expectedEndpoints.get("Keyspace3").get(new BigIntegerToken("65")).removeAll(makeAddrs("127.0.0.10"));
-        expectedEndpoints.get("Keyspace3").get(new BigIntegerToken("75")).removeAll(makeAddrs("127.0.0.10"));
-        expectedEndpoints.get("Keyspace3").get(new BigIntegerToken("85")).removeAll(makeAddrs("127.0.0.10"));
-        expectedEndpoints.get("Keyspace4").get(new BigIntegerToken("35")).removeAll(makeAddrs("127.0.0.7", "127.0.0.8"));
-        expectedEndpoints.get("Keyspace4").get(new BigIntegerToken("45")).removeAll(makeAddrs("127.0.0.7", "127.0.1.2", "127.0.0.1"));
-        expectedEndpoints.get("Keyspace4").get(new BigIntegerToken("55")).removeAll(makeAddrs("127.0.0.2", "127.0.0.7"));
-        expectedEndpoints.get("Keyspace4").get(new BigIntegerToken("65")).removeAll(makeAddrs("127.0.0.10"));
-        expectedEndpoints.get("Keyspace4").get(new BigIntegerToken("75")).removeAll(makeAddrs("127.0.0.10"));
-        expectedEndpoints.get("Keyspace4").get(new BigIntegerToken("85")).removeAll(makeAddrs("127.0.0.10"));
+        expectedEndpoints.get(KEYSPACE1).get(new BigIntegerToken("55")).removeAll(makeAddrs("127.0.0.7", "127.0.0.8"));
+        expectedEndpoints.get(KEYSPACE1).get(new BigIntegerToken("85")).removeAll(makeAddrs("127.0.0.10"));
+        expectedEndpoints.get(KEYSPACE2).get(new BigIntegerToken("55")).removeAll(makeAddrs("127.0.0.7", "127.0.0.8"));
+        expectedEndpoints.get(KEYSPACE2).get(new BigIntegerToken("85")).removeAll(makeAddrs("127.0.0.10"));
+        expectedEndpoints.get(KEYSPACE3).get(new BigIntegerToken("15")).removeAll(makeAddrs("127.0.0.7", "127.0.0.8"));
+        expectedEndpoints.get(KEYSPACE3).get(new BigIntegerToken("25")).removeAll(makeAddrs("127.0.0.7", "127.0.1.2", "127.0.0.1"));
+        expectedEndpoints.get(KEYSPACE3).get(new BigIntegerToken("35")).removeAll(makeAddrs("127.0.0.7", "127.0.0.2"));
+        expectedEndpoints.get(KEYSPACE3).get(new BigIntegerToken("45")).removeAll(makeAddrs("127.0.0.7", "127.0.0.10", "127.0.0.3"));
+        expectedEndpoints.get(KEYSPACE3).get(new BigIntegerToken("55")).removeAll(makeAddrs("127.0.0.7", "127.0.0.10", "127.0.0.4"));
+        expectedEndpoints.get(KEYSPACE3).get(new BigIntegerToken("65")).removeAll(makeAddrs("127.0.0.10"));
+        expectedEndpoints.get(KEYSPACE3).get(new BigIntegerToken("75")).removeAll(makeAddrs("127.0.0.10"));
+        expectedEndpoints.get(KEYSPACE3).get(new BigIntegerToken("85")).removeAll(makeAddrs("127.0.0.10"));
+        expectedEndpoints.get(KEYSPACE4).get(new BigIntegerToken("35")).removeAll(makeAddrs("127.0.0.7", "127.0.0.8"));
+        expectedEndpoints.get(KEYSPACE4).get(new BigIntegerToken("45")).removeAll(makeAddrs("127.0.0.7", "127.0.1.2", "127.0.0.1"));
+        expectedEndpoints.get(KEYSPACE4).get(new BigIntegerToken("55")).removeAll(makeAddrs("127.0.0.2", "127.0.0.7"));
+        expectedEndpoints.get(KEYSPACE4).get(new BigIntegerToken("65")).removeAll(makeAddrs("127.0.0.10"));
+        expectedEndpoints.get(KEYSPACE4).get(new BigIntegerToken("75")).removeAll(makeAddrs("127.0.0.10"));
+        expectedEndpoints.get(KEYSPACE4).get(new BigIntegerToken("85")).removeAll(makeAddrs("127.0.0.10"));
 
         PendingRangeCalculatorService.instance.blockUntilFinished();
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/service/MoveTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/MoveTest.java b/test/unit/org/apache/cassandra/service/MoveTest.java
index a3fd89c..c01f4af 100644
--- a/test/unit/org/apache/cassandra/service/MoveTest.java
+++ b/test/unit/org/apache/cassandra/service/MoveTest.java
@@ -48,6 +48,10 @@ public class MoveTest
 {
     private static final IPartitioner partitioner = new RandomPartitioner();
     private static IPartitioner oldPartitioner;
+    private static final String KEYSPACE1 = "MoveTestKeyspace1";
+    private static final String KEYSPACE2 = "MoveTestKeyspace2";
+    private static final String KEYSPACE3 = "MoveTestKeyspace3";
+    private static final String KEYSPACE4 = "MoveTestKeyspace4";
 
     /*
      * NOTE: the tests above uses RandomPartitioner, which is not the default
@@ -60,13 +64,13 @@ public class MoveTest
     {
         oldPartitioner = StorageService.instance.setPartitionerUnsafe(partitioner);
         SchemaLoader.loadSchema();
+        SchemaLoader.schemaDefinition("MoveTest");
     }
 
     @AfterClass
     public static void tearDown()
     {
         StorageService.instance.setPartitionerUnsafe(oldPartitioner);
-        SchemaLoader.stopGossiper();
     }
 
     /*
@@ -203,7 +207,7 @@ public class MoveTest
         Map<String, AbstractReplicationStrategy> keyspaceStrategyMap = new HashMap<String, AbstractReplicationStrategy>();
         for (int i = 1; i <= 4; i++)
         {
-            keyspaceStrategyMap.put("Keyspace" + i, getStrategy("Keyspace" + i, tmd));
+            keyspaceStrategyMap.put("MoveTestKeyspace" + i, getStrategy("MoveTestKeyspace" + i, tmd));
         }
 
        /**
@@ -222,7 +226,7 @@ public class MoveTest
         *  }
         */
 
-        Multimap<InetAddress, Range<Token>> keyspace1ranges = keyspaceStrategyMap.get("Keyspace1").getAddressRanges();
+        Multimap<InetAddress, Range<Token>> keyspace1ranges = keyspaceStrategyMap.get(KEYSPACE1).getAddressRanges();
         Collection<Range<Token>> ranges1 = keyspace1ranges.get(InetAddress.getByName("127.0.0.1"));
         assertEquals(1, collectionSize(ranges1));
         assertEquals(generateRange(97, 0), ranges1.iterator().next());
@@ -271,7 +275,7 @@ public class MoveTest
         * }
         */
 
-        Multimap<InetAddress, Range<Token>> keyspace3ranges = keyspaceStrategyMap.get("Keyspace3").getAddressRanges();
+        Multimap<InetAddress, Range<Token>> keyspace3ranges = keyspaceStrategyMap.get(KEYSPACE3).getAddressRanges();
         ranges1 = keyspace3ranges.get(InetAddress.getByName("127.0.0.1"));
         assertEquals(collectionSize(ranges1), 5);
         assertTrue(ranges1.equals(generateRanges(97, 0, 70, 87, 50, 67, 87, 97, 67, 70)));
@@ -319,7 +323,7 @@ public class MoveTest
          *      /127.0.0.10=[(70,87], (87,97], (67,70]]
          *  }
          */
-        Multimap<InetAddress, Range<Token>> keyspace4ranges = keyspaceStrategyMap.get("Keyspace4").getAddressRanges();
+        Multimap<InetAddress, Range<Token>> keyspace4ranges = keyspaceStrategyMap.get(KEYSPACE4).getAddressRanges();
         ranges1 = keyspace4ranges.get(InetAddress.getByName("127.0.0.1"));
         assertEquals(collectionSize(ranges1), 3);
         assertTrue(ranges1.equals(generateRanges(97, 0, 70, 87, 87, 97)));
@@ -353,50 +357,50 @@ public class MoveTest
 
         // pre-calculate the results.
         Map<String, Multimap<Token, InetAddress>> expectedEndpoints = new HashMap<String, Multimap<Token, InetAddress>>();
-        expectedEndpoints.put("Keyspace1", HashMultimap.<Token, InetAddress>create());
-        expectedEndpoints.get("Keyspace1").putAll(new BigIntegerToken("5"), makeAddrs("127.0.0.2"));
-        expectedEndpoints.get("Keyspace1").putAll(new BigIntegerToken("15"), makeAddrs("127.0.0.3"));
-        expectedEndpoints.get("Keyspace1").putAll(new BigIntegerToken("25"), makeAddrs("127.0.0.4"));
-        expectedEndpoints.get("Keyspace1").putAll(new BigIntegerToken("35"), makeAddrs("127.0.0.5"));
-        expectedEndpoints.get("Keyspace1").putAll(new BigIntegerToken("45"), makeAddrs("127.0.0.6"));
-        expectedEndpoints.get("Keyspace1").putAll(new BigIntegerToken("55"), makeAddrs("127.0.0.7", "127.0.1.1"));
-        expectedEndpoints.get("Keyspace1").putAll(new BigIntegerToken("65"), makeAddrs("127.0.0.7"));
-        expectedEndpoints.get("Keyspace1").putAll(new BigIntegerToken("75"), makeAddrs("127.0.0.9", "127.0.1.2"));
-        expectedEndpoints.get("Keyspace1").putAll(new BigIntegerToken("85"), makeAddrs("127.0.0.9"));
-        expectedEndpoints.get("Keyspace1").putAll(new BigIntegerToken("95"), makeAddrs("127.0.0.10"));
-        expectedEndpoints.put("Keyspace2", HashMultimap.<Token, InetAddress>create());
-        expectedEndpoints.get("Keyspace2").putAll(new BigIntegerToken("5"), makeAddrs("127.0.0.2"));
-        expectedEndpoints.get("Keyspace2").putAll(new BigIntegerToken("15"), makeAddrs("127.0.0.3"));
-        expectedEndpoints.get("Keyspace2").putAll(new BigIntegerToken("25"), makeAddrs("127.0.0.4"));
-        expectedEndpoints.get("Keyspace2").putAll(new BigIntegerToken("35"), makeAddrs("127.0.0.5"));
-        expectedEndpoints.get("Keyspace2").putAll(new BigIntegerToken("45"), makeAddrs("127.0.0.6"));
-        expectedEndpoints.get("Keyspace2").putAll(new BigIntegerToken("55"), makeAddrs("127.0.0.7", "127.0.1.1"));
-        expectedEndpoints.get("Keyspace2").putAll(new BigIntegerToken("65"), makeAddrs("127.0.0.7"));
-        expectedEndpoints.get("Keyspace2").putAll(new BigIntegerToken("75"), makeAddrs("127.0.0.9", "127.0.1.2"));
-        expectedEndpoints.get("Keyspace2").putAll(new BigIntegerToken("85"), makeAddrs("127.0.0.9"));
-        expectedEndpoints.get("Keyspace2").putAll(new BigIntegerToken("95"), makeAddrs("127.0.0.10"));
-        expectedEndpoints.put("Keyspace3", HashMultimap.<Token, InetAddress>create());
-        expectedEndpoints.get("Keyspace3").putAll(new BigIntegerToken("5"), makeAddrs("127.0.0.2", "127.0.0.3", "127.0.0.4", "127.0.0.5", "127.0.0.6"));
-        expectedEndpoints.get("Keyspace3").putAll(new BigIntegerToken("15"), makeAddrs("127.0.0.3", "127.0.0.4", "127.0.0.5", "127.0.0.6", "127.0.0.7", "127.0.1.1"));
-        expectedEndpoints.get("Keyspace3").putAll(new BigIntegerToken("25"), makeAddrs("127.0.0.4", "127.0.0.5", "127.0.0.6", "127.0.0.7", "127.0.0.8", "127.0.1.1"));
-        expectedEndpoints.get("Keyspace3").putAll(new BigIntegerToken("35"), makeAddrs("127.0.0.5", "127.0.0.6", "127.0.0.7", "127.0.0.8", "127.0.0.9", "127.0.1.1", "127.0.1.2"));
-        expectedEndpoints.get("Keyspace3").putAll(new BigIntegerToken("45"), makeAddrs("127.0.0.6", "127.0.0.7", "127.0.0.8", "127.0.0.9", "127.0.0.10", "127.0.1.1", "127.0.1.2"));
-        expectedEndpoints.get("Keyspace3").putAll(new BigIntegerToken("55"), makeAddrs("127.0.0.7", "127.0.0.8", "127.0.0.9", "127.0.0.10", "127.0.0.1", "127.0.1.1", "127.0.1.2"));
-        expectedEndpoints.get("Keyspace3").putAll(new BigIntegerToken("65"), makeAddrs("127.0.0.7", "127.0.0.8", "127.0.0.9", "127.0.0.10", "127.0.0.1", "127.0.1.2"));
-        expectedEndpoints.get("Keyspace3").putAll(new BigIntegerToken("75"), makeAddrs("127.0.0.9", "127.0.0.10", "127.0.0.1", "127.0.0.2", "127.0.0.3", "127.0.1.2"));
-        expectedEndpoints.get("Keyspace3").putAll(new BigIntegerToken("85"), makeAddrs("127.0.0.9", "127.0.0.10", "127.0.0.1", "127.0.0.2", "127.0.0.3"));
-        expectedEndpoints.get("Keyspace3").putAll(new BigIntegerToken("95"), makeAddrs("127.0.0.10", "127.0.0.1", "127.0.0.2", "127.0.0.3", "127.0.0.4"));
-        expectedEndpoints.put("Keyspace4", HashMultimap.<Token, InetAddress>create());
-        expectedEndpoints.get("Keyspace4").putAll(new BigIntegerToken("5"), makeAddrs("127.0.0.2", "127.0.0.3", "127.0.0.4"));
-        expectedEndpoints.get("Keyspace4").putAll(new BigIntegerToken("15"), makeAddrs("127.0.0.3", "127.0.0.4", "127.0.0.5"));
-        expectedEndpoints.get("Keyspace4").putAll(new BigIntegerToken("25"), makeAddrs("127.0.0.4", "127.0.0.5", "127.0.0.6"));
-        expectedEndpoints.get("Keyspace4").putAll(new BigIntegerToken("35"), makeAddrs("127.0.0.5", "127.0.0.6", "127.0.0.7", "127.0.1.1"));
-        expectedEndpoints.get("Keyspace4").putAll(new BigIntegerToken("45"), makeAddrs("127.0.0.6", "127.0.0.7", "127.0.0.8", "127.0.1.1"));
-        expectedEndpoints.get("Keyspace4").putAll(new BigIntegerToken("55"), makeAddrs("127.0.0.7", "127.0.0.8", "127.0.0.9", "127.0.1.1", "127.0.1.2"));
-        expectedEndpoints.get("Keyspace4").putAll(new BigIntegerToken("65"), makeAddrs("127.0.0.7", "127.0.0.8", "127.0.0.9", "127.0.1.2"));
-        expectedEndpoints.get("Keyspace4").putAll(new BigIntegerToken("75"), makeAddrs("127.0.0.9", "127.0.0.10", "127.0.0.1", "127.0.1.2"));
-        expectedEndpoints.get("Keyspace4").putAll(new BigIntegerToken("85"), makeAddrs("127.0.0.9", "127.0.0.10", "127.0.0.1"));
-        expectedEndpoints.get("Keyspace4").putAll(new BigIntegerToken("95"), makeAddrs("127.0.0.10", "127.0.0.1", "127.0.0.2"));
+        expectedEndpoints.put(KEYSPACE1, HashMultimap.<Token, InetAddress>create());
+        expectedEndpoints.get(KEYSPACE1).putAll(new BigIntegerToken("5"), makeAddrs("127.0.0.2"));
+        expectedEndpoints.get(KEYSPACE1).putAll(new BigIntegerToken("15"), makeAddrs("127.0.0.3"));
+        expectedEndpoints.get(KEYSPACE1).putAll(new BigIntegerToken("25"), makeAddrs("127.0.0.4"));
+        expectedEndpoints.get(KEYSPACE1).putAll(new BigIntegerToken("35"), makeAddrs("127.0.0.5"));
+        expectedEndpoints.get(KEYSPACE1).putAll(new BigIntegerToken("45"), makeAddrs("127.0.0.6"));
+        expectedEndpoints.get(KEYSPACE1).putAll(new BigIntegerToken("55"), makeAddrs("127.0.0.7", "127.0.1.1"));
+        expectedEndpoints.get(KEYSPACE1).putAll(new BigIntegerToken("65"), makeAddrs("127.0.0.7"));
+        expectedEndpoints.get(KEYSPACE1).putAll(new BigIntegerToken("75"), makeAddrs("127.0.0.9", "127.0.1.2"));
+        expectedEndpoints.get(KEYSPACE1).putAll(new BigIntegerToken("85"), makeAddrs("127.0.0.9"));
+        expectedEndpoints.get(KEYSPACE1).putAll(new BigIntegerToken("95"), makeAddrs("127.0.0.10"));
+        expectedEndpoints.put(KEYSPACE2, HashMultimap.<Token, InetAddress>create());
+        expectedEndpoints.get(KEYSPACE2).putAll(new BigIntegerToken("5"), makeAddrs("127.0.0.2"));
+        expectedEndpoints.get(KEYSPACE2).putAll(new BigIntegerToken("15"), makeAddrs("127.0.0.3"));
+        expectedEndpoints.get(KEYSPACE2).putAll(new BigIntegerToken("25"), makeAddrs("127.0.0.4"));
+        expectedEndpoints.get(KEYSPACE2).putAll(new BigIntegerToken("35"), makeAddrs("127.0.0.5"));
+        expectedEndpoints.get(KEYSPACE2).putAll(new BigIntegerToken("45"), makeAddrs("127.0.0.6"));
+        expectedEndpoints.get(KEYSPACE2).putAll(new BigIntegerToken("55"), makeAddrs("127.0.0.7", "127.0.1.1"));
+        expectedEndpoints.get(KEYSPACE2).putAll(new BigIntegerToken("65"), makeAddrs("127.0.0.7"));
+        expectedEndpoints.get(KEYSPACE2).putAll(new BigIntegerToken("75"), makeAddrs("127.0.0.9", "127.0.1.2"));
+        expectedEndpoints.get(KEYSPACE2).putAll(new BigIntegerToken("85"), makeAddrs("127.0.0.9"));
+        expectedEndpoints.get(KEYSPACE2).putAll(new BigIntegerToken("95"), makeAddrs("127.0.0.10"));
+        expectedEndpoints.put(KEYSPACE3, HashMultimap.<Token, InetAddress>create());
+        expectedEndpoints.get(KEYSPACE3).putAll(new BigIntegerToken("5"), makeAddrs("127.0.0.2", "127.0.0.3", "127.0.0.4", "127.0.0.5", "127.0.0.6"));
+        expectedEndpoints.get(KEYSPACE3).putAll(new BigIntegerToken("15"), makeAddrs("127.0.0.3", "127.0.0.4", "127.0.0.5", "127.0.0.6", "127.0.0.7", "127.0.1.1"));
+        expectedEndpoints.get(KEYSPACE3).putAll(new BigIntegerToken("25"), makeAddrs("127.0.0.4", "127.0.0.5", "127.0.0.6", "127.0.0.7", "127.0.0.8", "127.0.1.1"));
+        expectedEndpoints.get(KEYSPACE3).putAll(new BigIntegerToken("35"), makeAddrs("127.0.0.5", "127.0.0.6", "127.0.0.7", "127.0.0.8", "127.0.0.9", "127.0.1.1", "127.0.1.2"));
+        expectedEndpoints.get(KEYSPACE3).putAll(new BigIntegerToken("45"), makeAddrs("127.0.0.6", "127.0.0.7", "127.0.0.8", "127.0.0.9", "127.0.0.10", "127.0.1.1", "127.0.1.2"));
+        expectedEndpoints.get(KEYSPACE3).putAll(new BigIntegerToken("55"), makeAddrs("127.0.0.7", "127.0.0.8", "127.0.0.9", "127.0.0.10", "127.0.0.1", "127.0.1.1", "127.0.1.2"));
+        expectedEndpoints.get(KEYSPACE3).putAll(new BigIntegerToken("65"), makeAddrs("127.0.0.7", "127.0.0.8", "127.0.0.9", "127.0.0.10", "127.0.0.1", "127.0.1.2"));
+        expectedEndpoints.get(KEYSPACE3).putAll(new BigIntegerToken("75"), makeAddrs("127.0.0.9", "127.0.0.10", "127.0.0.1", "127.0.0.2", "127.0.0.3", "127.0.1.2"));
+        expectedEndpoints.get(KEYSPACE3).putAll(new BigIntegerToken("85"), makeAddrs("127.0.0.9", "127.0.0.10", "127.0.0.1", "127.0.0.2", "127.0.0.3"));
+        expectedEndpoints.get(KEYSPACE3).putAll(new BigIntegerToken("95"), makeAddrs("127.0.0.10", "127.0.0.1", "127.0.0.2", "127.0.0.3", "127.0.0.4"));
+        expectedEndpoints.put(KEYSPACE4, HashMultimap.<Token, InetAddress>create());
+        expectedEndpoints.get(KEYSPACE4).putAll(new BigIntegerToken("5"), makeAddrs("127.0.0.2", "127.0.0.3", "127.0.0.4"));
+        expectedEndpoints.get(KEYSPACE4).putAll(new BigIntegerToken("15"), makeAddrs("127.0.0.3", "127.0.0.4", "127.0.0.5"));
+        expectedEndpoints.get(KEYSPACE4).putAll(new BigIntegerToken("25"), makeAddrs("127.0.0.4", "127.0.0.5", "127.0.0.6"));
+        expectedEndpoints.get(KEYSPACE4).putAll(new BigIntegerToken("35"), makeAddrs("127.0.0.5", "127.0.0.6", "127.0.0.7", "127.0.1.1"));
+        expectedEndpoints.get(KEYSPACE4).putAll(new BigIntegerToken("45"), makeAddrs("127.0.0.6", "127.0.0.7", "127.0.0.8", "127.0.1.1"));
+        expectedEndpoints.get(KEYSPACE4).putAll(new BigIntegerToken("55"), makeAddrs("127.0.0.7", "127.0.0.8", "127.0.0.9", "127.0.1.1", "127.0.1.2"));
+        expectedEndpoints.get(KEYSPACE4).putAll(new BigIntegerToken("65"), makeAddrs("127.0.0.7", "127.0.0.8", "127.0.0.9", "127.0.1.2"));
+        expectedEndpoints.get(KEYSPACE4).putAll(new BigIntegerToken("75"), makeAddrs("127.0.0.9", "127.0.0.10", "127.0.0.1", "127.0.1.2"));
+        expectedEndpoints.get(KEYSPACE4).putAll(new BigIntegerToken("85"), makeAddrs("127.0.0.9", "127.0.0.10", "127.0.0.1"));
+        expectedEndpoints.get(KEYSPACE4).putAll(new BigIntegerToken("95"), makeAddrs("127.0.0.10", "127.0.0.1", "127.0.0.2"));
 
         for (Map.Entry<String, AbstractReplicationStrategy> keyspaceStrategy : keyspaceStrategyMap.entrySet())
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/service/QueryPagerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/QueryPagerTest.java b/test/unit/org/apache/cassandra/service/QueryPagerTest.java
index e71e97a..6d9ed60 100644
--- a/test/unit/org/apache/cassandra/service/QueryPagerTest.java
+++ b/test/unit/org/apache/cassandra/service/QueryPagerTest.java
@@ -29,11 +29,15 @@ import org.junit.runner.RunWith;
 import org.apache.cassandra.Util;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.OrderedJUnit4ClassRunner;
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.composites.*;
 import org.apache.cassandra.db.filter.*;
 import org.apache.cassandra.db.marshal.CompositeType;
 import org.apache.cassandra.dht.*;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.service.pager.*;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
@@ -43,10 +47,31 @@ import static org.apache.cassandra.Util.range;
 import static org.apache.cassandra.utils.ByteBufferUtil.bytes;
 
 @RunWith(OrderedJUnit4ClassRunner.class)
-public class QueryPagerTest extends SchemaLoader
+public class QueryPagerTest
 {
-    private static final String KS = "Keyspace1";
-    private static final String CF = "Standard1";
+    public static final String KEYSPACE1 = "QueryPagerTest";
+    public static final String CF_STANDARD = "Standard1";
+    public static final String KEYSPACE_CQL = "cql_keyspace";
+    public static final String CF_CQL = "table2";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD));
+        SchemaLoader.createKeyspace(KEYSPACE_CQL,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    CFMetaData.compile("CREATE TABLE " + CF_CQL + " ("
+                                                     + "k text,"
+                                                     + "c text,"
+                                                     + "v text,"
+                                                     + "PRIMARY KEY (k, c))", KEYSPACE_CQL));
+        addData();
+    }
 
     private static String string(CellName name)
     {
@@ -65,7 +90,6 @@ public class QueryPagerTest extends SchemaLoader
         }
     }
 
-    @BeforeClass
     public static void addData()
     {
         cfs().clearUnsafe();
@@ -81,8 +105,8 @@ public class QueryPagerTest extends SchemaLoader
          */
         for (int i = 0; i < nbKeys; i++)
         {
-            Mutation rm = new Mutation(KS, bytes("k" + i));
-            ColumnFamily cf = rm.addOrGet(CF);
+            Mutation rm = new Mutation(KEYSPACE1, bytes("k" + i));
+            ColumnFamily cf = rm.addOrGet(CF_STANDARD);
 
             for (int j = 0; j < nbCols; j++)
                 cf.addColumn(Util.column("c" + j, "", 0));
@@ -93,7 +117,7 @@ public class QueryPagerTest extends SchemaLoader
 
     private static ColumnFamilyStore cfs()
     {
-        return Keyspace.open(KS).getColumnFamilyStore(CF);
+        return Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF_STANDARD);
     }
 
     private static String toString(List<Row> rows)
@@ -120,7 +144,7 @@ public class QueryPagerTest extends SchemaLoader
         SortedSet<CellName> s = new TreeSet<CellName>(cfs().metadata.comparator);
         for (String name : names)
             s.add(CellNames.simpleDense(bytes(name)));
-        return new SliceByNamesReadCommand(KS, bytes(key), CF, System.currentTimeMillis(), new NamesQueryFilter(s, true));
+        return new SliceByNamesReadCommand(KEYSPACE1, bytes(key), CF_STANDARD, System.currentTimeMillis(), new NamesQueryFilter(s, true));
     }
 
     private static ReadCommand sliceQuery(String key, String start, String end, int count)
@@ -132,7 +156,7 @@ public class QueryPagerTest extends SchemaLoader
     {
         SliceQueryFilter filter = new SliceQueryFilter(CellNames.simpleDense(bytes(start)), CellNames.simpleDense(bytes(end)), reversed, count);
         // Note: for MultiQueryTest, we need the same timestamp/expireBefore for all queries, so we just use 0 as it doesn't matter here.
-        return new SliceFromReadCommand(KS, bytes(key), CF, 0, filter);
+        return new SliceFromReadCommand(KEYSPACE1, bytes(key), CF_STANDARD, 0, filter);
     }
 
     private static RangeSliceCommand rangeNamesQuery(AbstractBounds<RowPosition> range, int count, String... names)
@@ -140,13 +164,13 @@ public class QueryPagerTest extends SchemaLoader
         SortedSet<CellName> s = new TreeSet<CellName>(cfs().metadata.comparator);
         for (String name : names)
             s.add(CellNames.simpleDense(bytes(name)));
-        return new RangeSliceCommand(KS, CF, System.currentTimeMillis(), new NamesQueryFilter(s, true), range, count);
+        return new RangeSliceCommand(KEYSPACE1, CF_STANDARD, System.currentTimeMillis(), new NamesQueryFilter(s, true), range, count);
     }
 
     private static RangeSliceCommand rangeSliceQuery(AbstractBounds<RowPosition> range, int count, String start, String end)
     {
         SliceQueryFilter filter = new SliceQueryFilter(CellNames.simpleDense(bytes(start)), CellNames.simpleDense(bytes(end)), false, Integer.MAX_VALUE);
-        return new RangeSliceCommand(KS, CF, System.currentTimeMillis(), filter, range, count);
+        return new RangeSliceCommand(KEYSPACE1, CF_STANDARD, System.currentTimeMillis(), filter, range, count);
     }
 
     private static void assertRow(Row r, String key, String... names)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/service/RelocateTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/RelocateTest.java b/test/unit/org/apache/cassandra/service/RelocateTest.java
index e8f66ad..22a992c 100644
--- a/test/unit/org/apache/cassandra/service/RelocateTest.java
+++ b/test/unit/org/apache/cassandra/service/RelocateTest.java
@@ -82,7 +82,6 @@ public class RelocateTest
     public static void tearDown() throws Exception
     {
         StorageService.instance.setPartitionerUnsafe(oldPartitioner);
-        SchemaLoader.stopGossiper();
     }
 
     /** Setup a virtual node ring */

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/service/RemoveTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/RemoveTest.java b/test/unit/org/apache/cassandra/service/RemoveTest.java
index 649a3f2..2574dcf 100644
--- a/test/unit/org/apache/cassandra/service/RemoveTest.java
+++ b/test/unit/org/apache/cassandra/service/RemoveTest.java
@@ -69,7 +69,6 @@ public class RemoveTest
     public static void tearDownClass()
     {
         StorageService.instance.setPartitionerUnsafe(oldPartitioner);
-        SchemaLoader.stopGossiper();
     }
 
     @Before

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/service/RowResolverTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/RowResolverTest.java b/test/unit/org/apache/cassandra/service/RowResolverTest.java
index 286d037..825944c 100644
--- a/test/unit/org/apache/cassandra/service/RowResolverTest.java
+++ b/test/unit/org/apache/cassandra/service/RowResolverTest.java
@@ -23,26 +23,43 @@ package org.apache.cassandra.service;
 
 import java.util.Arrays;
 
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.db.ArrayBackedSortedColumns;
 import org.apache.cassandra.db.ColumnFamily;
 import org.apache.cassandra.db.DeletionInfo;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.locator.SimpleStrategy;
 
 import static org.junit.Assert.*;
 import static org.apache.cassandra.Util.column;
 import static org.apache.cassandra.db.KeyspaceTest.*;
 
-public class RowResolverTest extends SchemaLoader
+public class RowResolverTest
 {
+    public static final String KEYSPACE1 = "Keyspace1";
+    public static final String CF_STANDARD = "Standard1";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD));
+    }
+    
     @Test
     public void testResolveSupersetNewer()
     {
-        ColumnFamily cf1 = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        ColumnFamily cf1 = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Standard1");
         cf1.addColumn(column("c1", "v1", 0));
 
-        ColumnFamily cf2 = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        ColumnFamily cf2 = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Standard1");
         cf2.addColumn(column("c1", "v2", 1));
 
         ColumnFamily resolved = RowDataResolver.resolveSuperset(Arrays.asList(cf1, cf2), System.currentTimeMillis());
@@ -54,10 +71,10 @@ public class RowResolverTest extends SchemaLoader
     @Test
     public void testResolveSupersetDisjoint()
     {
-        ColumnFamily cf1 = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        ColumnFamily cf1 = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Standard1");
         cf1.addColumn(column("c1", "v1", 0));
 
-        ColumnFamily cf2 = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        ColumnFamily cf2 = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Standard1");
         cf2.addColumn(column("c2", "v2", 1));
 
         ColumnFamily resolved = RowDataResolver.resolveSuperset(Arrays.asList(cf1, cf2), System.currentTimeMillis());
@@ -69,7 +86,7 @@ public class RowResolverTest extends SchemaLoader
     @Test
     public void testResolveSupersetNullOne()
     {
-        ColumnFamily cf2 = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        ColumnFamily cf2 = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Standard1");
         cf2.addColumn(column("c2", "v2", 1));
 
         ColumnFamily resolved = RowDataResolver.resolveSuperset(Arrays.asList(null, cf2), System.currentTimeMillis());
@@ -81,7 +98,7 @@ public class RowResolverTest extends SchemaLoader
     @Test
     public void testResolveSupersetNullTwo()
     {
-        ColumnFamily cf1 = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        ColumnFamily cf1 = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Standard1");
         cf1.addColumn(column("c1", "v1", 0));
 
         ColumnFamily resolved = RowDataResolver.resolveSuperset(Arrays.asList(cf1, null), System.currentTimeMillis());
@@ -100,10 +117,10 @@ public class RowResolverTest extends SchemaLoader
     public void testResolveDeleted()
     {
         // one CF with columns timestamped before a delete in another cf
-        ColumnFamily cf1 = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        ColumnFamily cf1 = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Standard1");
         cf1.addColumn(column("one", "A", 0));
 
-        ColumnFamily cf2 = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        ColumnFamily cf2 = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Standard1");
         cf2.delete(new DeletionInfo(1L, (int) (System.currentTimeMillis() / 1000)));
 
         ColumnFamily resolved = RowDataResolver.resolveSuperset(Arrays.asList(cf1, cf2), System.currentTimeMillis());
@@ -118,19 +135,19 @@ public class RowResolverTest extends SchemaLoader
     {
         // deletes and columns with interleaved timestamp, with out of order return sequence
 
-        ColumnFamily cf1 = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        ColumnFamily cf1 = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Standard1");
         cf1.delete(new DeletionInfo(0L, (int) (System.currentTimeMillis() / 1000)));
 
         // these columns created after the previous deletion
-        ColumnFamily cf2 = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        ColumnFamily cf2 = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Standard1");
         cf2.addColumn(column("one", "A", 1));
         cf2.addColumn(column("two", "A", 1));
 
         //this column created after the next delete
-        ColumnFamily cf3 = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        ColumnFamily cf3 = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Standard1");
         cf3.addColumn(column("two", "B", 3));
 
-        ColumnFamily cf4 = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+        ColumnFamily cf4 = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Standard1");
         cf4.delete(new DeletionInfo(2L, (int) (System.currentTimeMillis() / 1000)));
 
         ColumnFamily resolved = RowDataResolver.resolveSuperset(Arrays.asList(cf1, cf2, cf3, cf4), System.currentTimeMillis());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/service/StorageProxyTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/StorageProxyTest.java b/test/unit/org/apache/cassandra/service/StorageProxyTest.java
index 6fbc80a..c8afac0 100644
--- a/test/unit/org/apache/cassandra/service/StorageProxyTest.java
+++ b/test/unit/org/apache/cassandra/service/StorageProxyTest.java
@@ -25,7 +25,6 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 import static org.junit.Assert.assertEquals;
 
-import org.apache.cassandra.SchemaLoader;
 import static org.apache.cassandra.Util.token;
 import static org.apache.cassandra.Util.rp;
 
@@ -39,7 +38,7 @@ import org.apache.cassandra.dht.IncludingExcludingBounds;
 import org.apache.cassandra.locator.TokenMetadata;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
-public class StorageProxyTest extends SchemaLoader
+public class StorageProxyTest
 {
     private static Range<RowPosition> range(RowPosition left, RowPosition right)
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/streaming/StreamTransferTaskTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/streaming/StreamTransferTaskTest.java b/test/unit/org/apache/cassandra/streaming/StreamTransferTaskTest.java
index c6c04b0..f9ae82e 100644
--- a/test/unit/org/apache/cassandra/streaming/StreamTransferTaskTest.java
+++ b/test/unit/org/apache/cassandra/streaming/StreamTransferTaskTest.java
@@ -22,25 +22,42 @@ import java.util.List;
 import java.util.concurrent.ScheduledFuture;
 import java.util.concurrent.TimeUnit;
 
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.sstable.SSTableReader;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.utils.FBUtilities;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNull;
 
-public class StreamTransferTaskTest extends SchemaLoader
+public class StreamTransferTaskTest
 {
+    public static final String KEYSPACE1 = "StreamTransferTaskTest";
+    public static final String CF_STANDARD = "Standard1";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD));
+    }
+
     @Test
     public void testScheduleTimeout() throws Exception
     {
-        String ks = "Keyspace1";
+        String ks = KEYSPACE1;
         String cf = "Standard1";
 
         StreamSession session = new StreamSession(FBUtilities.getBroadcastAddress(), 0);
@@ -49,7 +66,7 @@ public class StreamTransferTaskTest extends SchemaLoader
         // create two sstables
         for (int i = 0; i < 2; i++)
         {
-            insertData(ks, cf, i, 1);
+            SchemaLoader.insertData(ks, cf, i, 1);
             cfs.forceBlockingFlush();
         }
 


[6/6] git commit: Reduce unit test times due to schema loading

Posted by ty...@apache.org.
Reduce unit test times due to schema loading

Patch by Lyuben Todorov; reviewed by Tyler Hobbs for CASSANDRA-6968


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/d2a3827a
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/d2a3827a
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/d2a3827a

Branch: refs/heads/trunk
Commit: d2a3827a65f208eae7aed129dcb6dc582ac1f3d5
Parents: 5101687
Author: lyubent <lt...@dundee.ac.uk>
Authored: Fri Jun 20 12:55:35 2014 -0500
Committer: Tyler Hobbs <ty...@datastax.com>
Committed: Fri Jun 20 12:55:35 2014 -0500

----------------------------------------------------------------------
 .../cassandra/db/LongFlushMemtableTest.java     |  22 +-
 .../apache/cassandra/db/LongKeyspaceTest.java   |  26 +-
 .../cassandra/db/commitlog/ComitLogStress.java  |   1 +
 .../db/compaction/LongCompactionsTest.java      |  24 +-
 .../LongLeveledCompactionStrategyTest.java      |  29 +-
 .../cassandra/AbstractSerializationsTester.java |   2 +-
 .../unit/org/apache/cassandra/SchemaLoader.java |  83 +++--
 .../cassandra/cache/AutoSavingCacheTest.java    |  25 +-
 .../cassandra/cache/CacheProviderTest.java      |  22 +-
 test/unit/org/apache/cassandra/cli/CliTest.java |  36 +-
 .../apache/cassandra/config/CFMetaDataTest.java |  27 +-
 .../org/apache/cassandra/config/DefsTest.java   |  58 +++-
 .../org/apache/cassandra/cql3/DeleteTest.java   |   4 +-
 .../db/ArrayBackedSortedColumnsTest.java        |  22 +-
 .../cassandra/db/BatchlogManagerTest.java       |  45 ++-
 .../org/apache/cassandra/db/CleanupTest.java    |  19 +-
 .../cassandra/db/CollationControllerTest.java   |  29 +-
 .../cassandra/db/ColumnFamilyStoreTest.java     | 346 +++++++++++--------
 .../apache/cassandra/db/ColumnFamilyTest.java   |  50 ++-
 .../org/apache/cassandra/db/CommitLogTest.java  |  49 ++-
 .../apache/cassandra/db/CounterCacheTest.java   |  29 +-
 .../apache/cassandra/db/CounterCellTest.java    |   2 +-
 .../cassandra/db/CounterMutationTest.java       |  58 ++--
 .../apache/cassandra/db/HintedHandOffTest.java  |  18 +-
 .../org/apache/cassandra/db/KeyCacheTest.java   |  25 +-
 .../apache/cassandra/db/KeyCollisionTest.java   |  21 +-
 .../org/apache/cassandra/db/KeyspaceTest.java   | 120 ++++---
 .../org/apache/cassandra/db/MultitableTest.java |  38 +-
 .../org/apache/cassandra/db/NameSortTest.java   |  32 +-
 .../apache/cassandra/db/RangeTombstoneTest.java |  21 +-
 .../apache/cassandra/db/ReadMessageTest.java    |  44 ++-
 .../cassandra/db/RecoveryManager2Test.java      |  27 +-
 .../cassandra/db/RecoveryManager3Test.java      |  38 +-
 .../cassandra/db/RecoveryManagerTest.java       |  55 ++-
 .../db/RecoveryManagerTruncateTest.java         |  25 +-
 .../org/apache/cassandra/db/RemoveCellTest.java |  25 +-
 .../cassandra/db/RemoveColumnFamilyTest.java    |  25 +-
 .../db/RemoveColumnFamilyWithFlush1Test.java    |  25 +-
 .../db/RemoveColumnFamilyWithFlush2Test.java    |  25 +-
 .../apache/cassandra/db/RemoveSubCellTest.java  |  34 +-
 .../org/apache/cassandra/db/RowCacheTest.java   |  49 ++-
 .../apache/cassandra/db/RowIterationTest.java   |  20 +-
 test/unit/org/apache/cassandra/db/RowTest.java  |  27 +-
 .../unit/org/apache/cassandra/db/ScrubTest.java |  65 ++--
 .../apache/cassandra/db/SerializationsTest.java |  20 +-
 .../org/apache/cassandra/db/TimeSortTest.java   |  53 ++-
 .../db/compaction/AntiCompactionTest.java       |  21 +-
 .../compaction/BlacklistingCompactionsTest.java |  22 +-
 .../db/compaction/CompactionsPurgeTest.java     |  51 ++-
 .../db/compaction/CompactionsTest.java          |  60 +++-
 .../LeveledCompactionStrategyTest.java          |  55 ++-
 .../db/compaction/OneCompactionTest.java        |  29 +-
 .../SizeTieredCompactionStrategyTest.java       |  24 +-
 .../cassandra/db/compaction/TTLExpiryTest.java  |  39 ++-
 .../db/index/PerRowSecondaryIndexTest.java      |  26 +-
 .../cassandra/db/marshal/CompositeTypeTest.java |  37 +-
 .../db/marshal/DynamicCompositeTypeTest.java    |  35 +-
 .../apache/cassandra/dht/BootStrapperTest.java  |  12 +-
 .../cassandra/io/BloomFilterTrackerTest.java    |   3 +-
 .../io/sstable/IndexSummaryManagerTest.java     |  56 ++-
 .../cassandra/io/sstable/LegacySSTableTest.java |  17 +-
 .../cassandra/io/sstable/SSTableLoaderTest.java |  26 +-
 .../io/sstable/SSTableMetadataTest.java         |  59 +++-
 .../cassandra/io/sstable/SSTableReaderTest.java |  68 ++--
 .../io/sstable/SSTableScannerTest.java          |  17 +-
 .../io/sstable/SSTableSimpleWriterTest.java     |  24 +-
 .../cassandra/io/sstable/SSTableUtils.java      |  14 +-
 .../locator/OldNetworkTopologyStrategyTest.java |   3 +-
 .../ReplicationStrategyEndpointCacheTest.java   |  19 +-
 .../cassandra/locator/SimpleStrategyTest.java   |  20 +-
 .../cassandra/repair/DifferencerTest.java       |  23 +-
 .../apache/cassandra/repair/ValidatorTest.java  |  19 +-
 .../service/AntiEntropyServiceCounterTest.java  |   4 +-
 .../service/AntiEntropyServiceStandardTest.java |   4 +-
 .../service/AntiEntropyServiceTestAbstract.java |  22 +-
 .../service/EmbeddedCassandraServiceTest.java   |  23 +-
 .../service/LeaveAndBootstrapTest.java          | 139 ++++----
 .../org/apache/cassandra/service/MoveTest.java  | 102 +++---
 .../cassandra/service/QueryPagerTest.java       |  46 ++-
 .../apache/cassandra/service/RelocateTest.java  |   1 -
 .../apache/cassandra/service/RemoveTest.java    |   1 -
 .../cassandra/service/RowResolverTest.java      |  43 ++-
 .../cassandra/service/StorageProxyTest.java     |   3 +-
 .../streaming/StreamTransferTaskTest.java       |  23 +-
 .../streaming/StreamingTransferTest.java        |  54 ++-
 .../apache/cassandra/thrift/MultiSliceTest.java |  22 +-
 .../cassandra/thrift/ThriftValidationTest.java  |  40 ++-
 .../cassandra/tools/SSTableExportTest.java      |  63 +++-
 .../cassandra/tools/SSTableImportTest.java      |  42 ++-
 .../cassandra/triggers/TriggersSchemaTest.java  |  10 +-
 .../apache/cassandra/triggers/TriggersTest.java |  10 +-
 .../cassandra/utils/EncodedStreamsTest.java     |  33 +-
 92 files changed, 2380 insertions(+), 924 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/long/org/apache/cassandra/db/LongFlushMemtableTest.java
----------------------------------------------------------------------
diff --git a/test/long/org/apache/cassandra/db/LongFlushMemtableTest.java b/test/long/org/apache/cassandra/db/LongFlushMemtableTest.java
index 4bb8fdd..211f72f 100644
--- a/test/long/org/apache/cassandra/db/LongFlushMemtableTest.java
+++ b/test/long/org/apache/cassandra/db/LongFlushMemtableTest.java
@@ -24,22 +24,36 @@ package org.apache.cassandra.db;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
 import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.service.MigrationManager;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
-public class LongFlushMemtableTest extends SchemaLoader
+public class LongFlushMemtableTest
 {
+    public static final String KEYSPACE1 = "LongFlushMemtableTest";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.loadSchema();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1));
+    }
+
     @Test
     public void testFlushMemtables() throws IOException, ConfigurationException
     {
-        Keyspace keyspace = Keyspace.open("Keyspace1");
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
         for (int i = 0; i < 100; i++)
         {
             CFMetaData metadata = CFMetaData.denseCFMetaData(keyspace.getName(), "_CF" + i, UTF8Type.instance);
@@ -50,8 +64,8 @@ public class LongFlushMemtableTest extends SchemaLoader
         {
             for (int i = 0; i < 100; i++)
             {
-                Mutation rm = new Mutation("Keyspace1", ByteBufferUtil.bytes("key" + j));
-                ColumnFamily cf = ArrayBackedSortedColumns.factory.create("Keyspace1", "_CF" + i);
+                Mutation rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes("key" + j));
+                ColumnFamily cf = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "_CF" + i);
                 // don't cheat by allocating this outside of the loop; that defeats the purpose of deliberately using lots of memory
                 ByteBuffer value = ByteBuffer.allocate(100000);
                 cf.addColumn(new BufferCell(Util.cellname("c"), value));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/long/org/apache/cassandra/db/LongKeyspaceTest.java
----------------------------------------------------------------------
diff --git a/test/long/org/apache/cassandra/db/LongKeyspaceTest.java b/test/long/org/apache/cassandra/db/LongKeyspaceTest.java
index 7a5b837..fe22da8 100644
--- a/test/long/org/apache/cassandra/db/LongKeyspaceTest.java
+++ b/test/long/org/apache/cassandra/db/LongKeyspaceTest.java
@@ -18,27 +18,43 @@
 
 package org.apache.cassandra.db;
 
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.KSMetaData;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.utils.WrappedRunnable;
 import static org.apache.cassandra.Util.column;
 
 import org.apache.cassandra.Util;
 
-
-public class LongKeyspaceTest extends SchemaLoader
+public class LongKeyspaceTest
 {
+    public static final String KEYSPACE1 = "LongKeyspaceTest";
+    public static final String CF_STANDARD = "Standard1";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD));
+    }
+
     @Test
     public void testGetRowMultiColumn() throws Throwable
     {
-        final Keyspace keyspace = Keyspace.open("Keyspace1");
+        final Keyspace keyspace = Keyspace.open(KEYSPACE1);
         final ColumnFamilyStore cfStore = keyspace.getColumnFamilyStore("Standard1");
 
         for (int i = 1; i < 5000; i += 100)
         {
-            Mutation rm = new Mutation("Keyspace1", Util.dk("key" + i).getKey());
-            ColumnFamily cf = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
+            Mutation rm = new Mutation(KEYSPACE1, Util.dk("key" + i).getKey());
+            ColumnFamily cf = ArrayBackedSortedColumns.factory.create(KEYSPACE1, "Standard1");
             for (int j = 0; j < i; j++)
                 cf.addColumn(column("c" + j, "v" + j, 1L));
             rm.add(cf);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/long/org/apache/cassandra/db/commitlog/ComitLogStress.java
----------------------------------------------------------------------
diff --git a/test/long/org/apache/cassandra/db/commitlog/ComitLogStress.java b/test/long/org/apache/cassandra/db/commitlog/ComitLogStress.java
index 5b334cc..dc90804 100644
--- a/test/long/org/apache/cassandra/db/commitlog/ComitLogStress.java
+++ b/test/long/org/apache/cassandra/db/commitlog/ComitLogStress.java
@@ -52,6 +52,7 @@ public class ComitLogStress
         ScheduledExecutorService scheduled = Executors.newScheduledThreadPool(1);
 
         org.apache.cassandra.SchemaLoader.loadSchema();
+        org.apache.cassandra.SchemaLoader.schemaDefinition(""); // leave def. blank to maintain old behaviour
         final AtomicLong count = new AtomicLong();
         final long start = System.currentTimeMillis();
         System.out.println(String.format(format, "seconds", "max_mb", "allocated_mb", "free_mb", "diffrence", "count"));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/long/org/apache/cassandra/db/compaction/LongCompactionsTest.java
----------------------------------------------------------------------
diff --git a/test/long/org/apache/cassandra/db/compaction/LongCompactionsTest.java b/test/long/org/apache/cassandra/db/compaction/LongCompactionsTest.java
index 94bc09f..7bc8ef5 100644
--- a/test/long/org/apache/cassandra/db/compaction/LongCompactionsTest.java
+++ b/test/long/org/apache/cassandra/db/compaction/LongCompactionsTest.java
@@ -24,21 +24,39 @@ import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 
-import org.apache.cassandra.config.Schema;
+import org.junit.BeforeClass;
 import org.junit.Test;
 import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.KSMetaData;
+import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.Util;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.io.sstable.SSTableReader;
 import org.apache.cassandra.io.sstable.SSTableUtils;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 
 import static org.junit.Assert.assertEquals;
 
-public class LongCompactionsTest extends SchemaLoader
+public class LongCompactionsTest
 {
     public static final String KEYSPACE1 = "Keyspace1";
+    public static final String CF_STANDARD = "Standard1";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        Map<String, String> compactionOptions = new HashMap<>();
+        compactionOptions.put("tombstone_compaction_interval", "1");
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD)
+                                                .compactionStrategyOptions(compactionOptions));
+    }
 
     /**
      * Test compaction with a very wide row.
@@ -87,7 +105,7 @@ public class LongCompactionsTest extends SchemaLoader
                     // last sstable has highest timestamps
                     cols[i] = Util.column(String.valueOf(i), String.valueOf(i), k);
                 }
-                rows.put(key, SSTableUtils.createCF(Long.MIN_VALUE, Integer.MIN_VALUE, cols));
+                rows.put(key, SSTableUtils.createCF(KEYSPACE1, CF_STANDARD, Long.MIN_VALUE, Integer.MIN_VALUE, cols));
             }
             SSTableReader sstable = SSTableUtils.prepare().write(rows);
             sstables.add(sstable);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/long/org/apache/cassandra/db/compaction/LongLeveledCompactionStrategyTest.java
----------------------------------------------------------------------
diff --git a/test/long/org/apache/cassandra/db/compaction/LongLeveledCompactionStrategyTest.java b/test/long/org/apache/cassandra/db/compaction/LongLeveledCompactionStrategyTest.java
index b071001..159b641 100644
--- a/test/long/org/apache/cassandra/db/compaction/LongLeveledCompactionStrategyTest.java
+++ b/test/long/org/apache/cassandra/db/compaction/LongLeveledCompactionStrategyTest.java
@@ -18,25 +18,44 @@
 package org.apache.cassandra.db.compaction;
 
 import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Set;
+import java.util.*;
 import java.util.concurrent.*;
 
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.db.*;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.sstable.SSTableReader;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.utils.FBUtilities;
 
-public class LongLeveledCompactionStrategyTest extends SchemaLoader
+public class LongLeveledCompactionStrategyTest
 {
+    public static final String KEYSPACE1 = "LongLeveledCompactionStrategyTest";
+    public static final String CF_STANDARDLVL = "StandardLeveled";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        Map<String, String> leveledOptions = new HashMap<>();
+        leveledOptions.put("sstable_size_in_mb", "1");
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARDLVL)
+                                                .compactionStrategyClass(LeveledCompactionStrategy.class)
+                                                .compactionStrategyOptions(leveledOptions));
+    }
+
     @Test
     public void testParallelLeveledCompaction() throws Exception
     {
-        String ksname = "Keyspace1";
+        String ksname = KEYSPACE1;
         String cfname = "StandardLeveled";
         Keyspace keyspace = Keyspace.open(ksname);
         ColumnFamilyStore store = keyspace.getColumnFamilyStore(cfname);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/AbstractSerializationsTester.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/AbstractSerializationsTester.java b/test/unit/org/apache/cassandra/AbstractSerializationsTester.java
index 1c97eae..15e5d34 100644
--- a/test/unit/org/apache/cassandra/AbstractSerializationsTester.java
+++ b/test/unit/org/apache/cassandra/AbstractSerializationsTester.java
@@ -32,7 +32,7 @@ import java.io.IOException;
 import java.util.HashMap;
 import java.util.Map;
 
-public class AbstractSerializationsTester extends SchemaLoader
+public class AbstractSerializationsTester
 {
     protected static final String CUR_VER = System.getProperty("cassandra.version", "2.1");
     protected static final Map<String, Integer> VERSION_MAP = new HashMap<String, Integer> ()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/SchemaLoader.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/SchemaLoader.java b/test/unit/org/apache/cassandra/SchemaLoader.java
index a9d69fd..066f454 100644
--- a/test/unit/org/apache/cassandra/SchemaLoader.java
+++ b/test/unit/org/apache/cassandra/SchemaLoader.java
@@ -57,11 +57,6 @@ public class SchemaLoader
         // Migrations aren't happy if gossiper is not started.  Even if we don't use migrations though,
         // some tests now expect us to start gossip for them.
         startGossiper();
-
-        // if you're messing with low-level sstable stuff, it can be useful to inject the schema directly
-        // Schema.instance.load(schemaDefinition());
-        for (KSMetaData ksm : schemaDefinition())
-            MigrationManager.announceNewKeyspace(ksm);
     }
 
     public static void prepareServer()
@@ -84,32 +79,27 @@ public class SchemaLoader
 
     public static void startGossiper()
     {
-        Gossiper.instance.start((int) (System.currentTimeMillis() / 1000));
-    }
-
-    @AfterClass
-    public static void stopGossiper()
-    {
-        Gossiper.instance.stop();
+        if (!Gossiper.instance.isEnabled())
+            Gossiper.instance.start((int) (System.currentTimeMillis() / 1000));
     }
 
-    public static Collection<KSMetaData> schemaDefinition() throws ConfigurationException
+    public static void schemaDefinition(String testName) throws ConfigurationException
     {
         List<KSMetaData> schema = new ArrayList<KSMetaData>();
 
         // A whole bucket of shorthand
-        String ks1 = "Keyspace1";
-        String ks2 = "Keyspace2";
-        String ks3 = "Keyspace3";
-        String ks4 = "Keyspace4";
-        String ks5 = "Keyspace5";
-        String ks6 = "Keyspace6";
-        String ks_kcs = "KeyCacheSpace";
-        String ks_rcs = "RowCacheSpace";
-        String ks_ccs = "CounterCacheSpace";
-        String ks_nocommit = "NoCommitlogSpace";
-        String ks_prsi = "PerRowSecondaryIndex";
-        String ks_cql = "cql_keyspace";
+        String ks1 = testName + "Keyspace1";
+        String ks2 = testName + "Keyspace2";
+        String ks3 = testName + "Keyspace3";
+        String ks4 = testName + "Keyspace4";
+        String ks5 = testName + "Keyspace5";
+        String ks6 = testName + "Keyspace6";
+        String ks_kcs = testName + "KeyCacheSpace";
+        String ks_rcs = testName + "RowCacheSpace";
+        String ks_ccs = testName + "CounterCacheSpace";
+        String ks_nocommit = testName + "NoCommitlogSpace";
+        String ks_prsi = testName + "PerRowSecondaryIndex";
+        String ks_cql = testName + "cql_keyspace";
 
         Class<? extends AbstractReplicationStrategy> simple = SimpleStrategy.class;
 
@@ -306,7 +296,30 @@ public class SchemaLoader
         if (Boolean.parseBoolean(System.getProperty("cassandra.test.compression", "false")))
             useCompression(schema);
 
-        return schema;
+        // if you're messing with low-level sstable stuff, it can be useful to inject the schema directly
+        // Schema.instance.load(schemaDefinition());
+        for (KSMetaData ksm : schema)
+            MigrationManager.announceNewKeyspace(ksm, false);
+    }
+
+    public static void createKeyspace(String keyspaceName,
+                                      Class<? extends AbstractReplicationStrategy> strategy,
+                                      Map<String, String> options,
+                                      CFMetaData... cfmetas) throws ConfigurationException
+    {
+        createKeyspace(keyspaceName, true, true, strategy, options, cfmetas);
+    }
+
+    public static void createKeyspace(String keyspaceName,
+                                      boolean durable,
+                                      boolean announceLocally,
+                                      Class<? extends AbstractReplicationStrategy> strategy,
+                                      Map<String, String> options,
+                                      CFMetaData... cfmetas) throws ConfigurationException
+    {
+        KSMetaData ksm = durable ? KSMetaData.testMetadata(keyspaceName, strategy, options, cfmetas)
+                                 : KSMetaData.testMetadataNotDurable(keyspaceName, strategy, options, cfmetas);
+        MigrationManager.announceNewKeyspace(ksm, announceLocally);
     }
 
     private static ColumnDefinition integerColumn(String ksName, String cfName)
@@ -335,7 +348,7 @@ public class SchemaLoader
                                     ColumnDefinition.Kind.REGULAR);
     }
 
-    private static CFMetaData perRowIndexedCFMD(String ksName, String cfName)
+    public static CFMetaData perRowIndexedCFMD(String ksName, String cfName)
     {
         final Map<String, String> indexOptions = Collections.singletonMap(
                                                       SecondaryIndex.CUSTOM_INDEX_OPTION_NAME,
@@ -359,19 +372,19 @@ public class SchemaLoader
         }
     }
 
-    private static CFMetaData standardCFMD(String ksName, String cfName)
+    public static CFMetaData standardCFMD(String ksName, String cfName)
     {
         return CFMetaData.denseCFMetaData(ksName, cfName, BytesType.instance);
     }
-    private static CFMetaData superCFMD(String ksName, String cfName, AbstractType subcc)
+    public static CFMetaData superCFMD(String ksName, String cfName, AbstractType subcc)
     {
         return superCFMD(ksName, cfName, BytesType.instance, subcc);
     }
-    private static CFMetaData superCFMD(String ksName, String cfName, AbstractType cc, AbstractType subcc)
+    public static CFMetaData superCFMD(String ksName, String cfName, AbstractType cc, AbstractType subcc)
     {
         return CFMetaData.denseCFMetaData(ksName, cfName, cc, subcc);
     }
-    private static CFMetaData indexCFMD(String ksName, String cfName, final Boolean withIdxType) throws ConfigurationException
+    public static CFMetaData indexCFMD(String ksName, String cfName, final Boolean withIdxType) throws ConfigurationException
     {
         CFMetaData cfm = CFMetaData.sparseCFMetaData(ksName, cfName, BytesType.instance).keyValidator(AsciiType.instance);
 
@@ -380,7 +393,7 @@ public class SchemaLoader
         return cfm.addColumnDefinition(ColumnDefinition.regularDef(cfm, cName, LongType.instance, null)
                                                        .setIndex(withIdxType ? ByteBufferUtil.bytesToHex(cName) : null, keys, null));
     }
-    private static CFMetaData compositeIndexCFMD(String ksName, String cfName, final Boolean withIdxType) throws ConfigurationException
+    public static CFMetaData compositeIndexCFMD(String ksName, String cfName, final Boolean withIdxType) throws ConfigurationException
     {
         final CompositeType composite = CompositeType.getInstance(Arrays.asList(new AbstractType<?>[]{UTF8Type.instance, UTF8Type.instance})); 
         CFMetaData cfm = CFMetaData.sparseCFMetaData(ksName, cfName, composite);
@@ -438,7 +451,7 @@ public class SchemaLoader
         DatabaseDescriptor.createAllDirectories();
     }
 
-    protected void insertData(String keyspace, String columnFamily, int offset, int numberOfRows)
+    public static void insertData(String keyspace, String columnFamily, int offset, int numberOfRows)
     {
         for (int i = offset; i < offset + numberOfRows; i++)
         {
@@ -450,7 +463,7 @@ public class SchemaLoader
     }
 
     /* usually used to populate the cache */
-    protected void readData(String keyspace, String columnFamily, int offset, int numberOfRows)
+    public static void readData(String keyspace, String columnFamily, int offset, int numberOfRows)
     {
         ColumnFamilyStore store = Keyspace.open(keyspace).getColumnFamilyStore(columnFamily);
         for (int i = offset; i < offset + numberOfRows; i++)
@@ -460,7 +473,7 @@ public class SchemaLoader
         }
     }
 
-    protected static void cleanupSavedCaches()
+    public static void cleanupSavedCaches()
     {
         File cachesDir = new File(DatabaseDescriptor.getSavedCachesLocation());
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/cache/AutoSavingCacheTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cache/AutoSavingCacheTest.java b/test/unit/org/apache/cassandra/cache/AutoSavingCacheTest.java
index 28afef1..683062c 100644
--- a/test/unit/org/apache/cassandra/cache/AutoSavingCacheTest.java
+++ b/test/unit/org/apache/cassandra/cache/AutoSavingCacheTest.java
@@ -18,28 +18,45 @@
 package org.apache.cassandra.cache;
 
 import org.junit.Assert;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.RowIndexEntry;
 import org.apache.cassandra.db.Mutation;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.sstable.SSTableReader;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.service.CacheService;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
-public class AutoSavingCacheTest extends SchemaLoader
+public class AutoSavingCacheTest
 {
+    private static final String KEYSPACE1 = "AutoSavingCacheTest1";
+    private static final String CF_STANDARD1 = "Standard1";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1));
+    }
+
     @Test
     public void testSerializeAndLoadKeyCache() throws Exception
     {
-        ColumnFamilyStore cfs = Keyspace.open("Keyspace1").getColumnFamilyStore("Standard1");
+        ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF_STANDARD1);
         for (int i = 0; i < 2; i++)
         {
-            Mutation rm = new Mutation("Keyspace1", ByteBufferUtil.bytes("key1"));
-            rm.add("Standard1", Util.cellname("c1"), ByteBufferUtil.bytes(i), 0);
+            Mutation rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes("key1"));
+            rm.add(CF_STANDARD1, Util.cellname("c1"), ByteBufferUtil.bytes(i), 0);
             rm.apply();
             cfs.forceBlockingFlush();
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/cache/CacheProviderTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cache/CacheProviderTest.java b/test/unit/org/apache/cassandra/cache/CacheProviderTest.java
index 71d4f80..fe26616 100644
--- a/test/unit/org/apache/cassandra/cache/CacheProviderTest.java
+++ b/test/unit/org/apache/cassandra/cache/CacheProviderTest.java
@@ -26,18 +26,22 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.UUID;
 
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.db.ArrayBackedSortedColumns;
 import org.apache.cassandra.db.ColumnFamily;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.locator.SimpleStrategy;
 
 import com.googlecode.concurrentlinkedhashmap.Weighers;
 
 import static org.apache.cassandra.Util.column;
 import static org.junit.Assert.*;
 
-public class CacheProviderTest extends SchemaLoader
+public class CacheProviderTest
 {
     MeasureableString key1 = new MeasureableString("key1");
     MeasureableString key2 = new MeasureableString("key2");
@@ -45,8 +49,18 @@ public class CacheProviderTest extends SchemaLoader
     MeasureableString key4 = new MeasureableString("key4");
     MeasureableString key5 = new MeasureableString("key5");
     private static final long CAPACITY = 4;
-    private String keyspaceName = "Keyspace1";
-    private String cfName = "Standard1";
+    private static final String KEYSPACE1 = "CacheProviderTest1";
+    private static final String CF_STANDARD1 = "Standard1";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1));
+    }
 
     private void simpleCase(ColumnFamily cf, ICache<MeasureableString, IRowCacheEntry> cache)
     {
@@ -100,7 +114,7 @@ public class CacheProviderTest extends SchemaLoader
 
     private ColumnFamily createCF()
     {
-        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(keyspaceName, cfName);
+        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(KEYSPACE1, CF_STANDARD1);
         cf.addColumn(column("vijay", "great", 1));
         cf.addColumn(column("awesome", "vijay", 1));
         return cf;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/cli/CliTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cli/CliTest.java b/test/unit/org/apache/cassandra/cli/CliTest.java
index 5d28c70..8c5a848 100644
--- a/test/unit/org/apache/cassandra/cli/CliTest.java
+++ b/test/unit/org/apache/cassandra/cli/CliTest.java
@@ -19,10 +19,14 @@
 package org.apache.cassandra.cli;
 
 import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.service.EmbeddedCassandraService;
 import org.apache.cassandra.thrift.*;
 import org.apache.thrift.TException;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import java.io.ByteArrayOutputStream;
@@ -33,11 +37,14 @@ import java.util.regex.Pattern;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
-public class CliTest extends SchemaLoader
+public class CliTest
 {
+    private static final String KEYSPACE1 = "CliTest";
+    private static final String CF_STANDARD1 = "Standard1";
+
     // please add new statements here so they could be auto-runned by this test.
     private String[] statements = {
-        "use TestKeySpace;",
+        "use " + KEYSPACE1,
         "create column family SecondaryIndicesWithoutIdxName" +
                 " with comparator = UTF8Type" +
                 " and default_validation_class = UTF8Type" +
@@ -136,7 +143,7 @@ public class CliTest extends SchemaLoader
         "drop index on '123'.617070;",
         "drop index on '123'.'-617071';",
         "drop index on CF3.'big world';",
-        "update keyspace TestKeySpace with durable_writes = false;",
+        "update keyspace " + KEYSPACE1 + " with durable_writes = false;",
         "assume 123 comparator as utf8;",
         "assume 123 sub_comparator as integer;",
         "assume 123 validator as lexicaluuid;",
@@ -179,9 +186,9 @@ public class CliTest extends SchemaLoader
         "set myCF['key']['scName']['firstname'] = 'John';",
         "get myCF['key']['scName']",
         "assume CF3 keys as utf8;",
-        "use TestKEYSpace;",
-        "update keyspace TestKeySpace with placement_strategy='org.apache.cassandra.locator.NetworkTopologyStrategy';",
-        "update keyspace TestKeySpace with strategy_options=[{DC1:3, DC2:4, DC5:1}];",
+        "use " + KEYSPACE1 + ";",
+        "update keyspace " + KEYSPACE1 + " with placement_strategy='org.apache.cassandra.locator.NetworkTopologyStrategy';",
+        "update keyspace " + KEYSPACE1 + " with strategy_options={DC1:3, DC2:4, DC5:1};",
         "describe cluster;",
         "help describe cluster;",
         "show cluster name",
@@ -212,9 +219,20 @@ public class CliTest extends SchemaLoader
         "HELP",
         "?",
         "show schema",
-        "show schema TestKeySpace"
+        "show schema " + KEYSPACE1
     };
 
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.startGossiper();
+        SchemaLoader.createKeyspace(KEYSPACE1, true, false,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1));
+    }
+
     @Test
     public void testCli() throws IOException, TException, TimedOutException, NotFoundException, SchemaDisagreementException, NoSuchFieldException, InvalidRequestException, UnavailableException, InstantiationException, IllegalAccessException
     {
@@ -236,13 +254,13 @@ public class CliTest extends SchemaLoader
         try
         {
             // dropping in case it exists e.g. could be left from previous run
-            CliMain.processStatement("drop keyspace TestKeySpace;");
+            CliMain.processStatement(String.format("drop keyspace %s;", KEYSPACE1));
         }
         catch (Exception e)
         {
             // TODO check before drop so we don't have this fragile ignored exception block
         }
-        CliMain.processStatement("create keyspace TestKeySpace;");
+        CliMain.processStatement(String.format("create keyspace %s;", KEYSPACE1));
 
         for (String statement : statements)
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/config/CFMetaDataTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/config/CFMetaDataTest.java b/test/unit/org/apache/cassandra/config/CFMetaDataTest.java
index 71f21a2..2b98da9 100644
--- a/test/unit/org/apache/cassandra/config/CFMetaDataTest.java
+++ b/test/unit/org/apache/cassandra/config/CFMetaDataTest.java
@@ -29,22 +29,25 @@ import org.apache.cassandra.cql3.UntypedResultSet;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.marshal.AsciiType;
 import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.compress.*;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.thrift.CfDef;
 import org.apache.cassandra.thrift.ColumnDef;
 import org.apache.cassandra.thrift.IndexType;
 import org.apache.cassandra.utils.ByteBufferUtil;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThat;
 
-public class CFMetaDataTest extends SchemaLoader
+public class CFMetaDataTest
 {
-    private static String KEYSPACE = "Keyspace1";
-    private static String COLUMN_FAMILY = "Standard1";
+    private static final String KEYSPACE1 = "CFMetaDataTest1";
+    private static final String CF_STANDARD1 = "Standard1";
 
     private static List<ColumnDef> columnDefs = new ArrayList<ColumnDef>();
 
@@ -59,21 +62,31 @@ public class CFMetaDataTest extends SchemaLoader
                                     .setIndex_type(IndexType.KEYS));
     }
 
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1));
+    }
+
     @Test
     public void testThriftConversion() throws Exception
     {
         CfDef cfDef = new CfDef().setDefault_validation_class(AsciiType.class.getCanonicalName())
                                  .setComment("Test comment")
                                  .setColumn_metadata(columnDefs)
-                                 .setKeyspace(KEYSPACE)
-                                 .setName(COLUMN_FAMILY);
+                                 .setKeyspace(KEYSPACE1)
+                                 .setName(CF_STANDARD1);
 
         // convert Thrift to CFMetaData
         CFMetaData cfMetaData = CFMetaData.fromThrift(cfDef);
 
         CfDef thriftCfDef = new CfDef();
-        thriftCfDef.keyspace = KEYSPACE;
-        thriftCfDef.name = COLUMN_FAMILY;
+        thriftCfDef.keyspace = KEYSPACE1;
+        thriftCfDef.name = CF_STANDARD1;
         thriftCfDef.default_validation_class = cfDef.default_validation_class;
         thriftCfDef.comment = cfDef.comment;
         thriftCfDef.column_metadata = new ArrayList<ColumnDef>();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/config/DefsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/config/DefsTest.java b/test/unit/org/apache/cassandra/config/DefsTest.java
index 1722407..206a336 100644
--- a/test/unit/org/apache/cassandra/config/DefsTest.java
+++ b/test/unit/org/apache/cassandra/config/DefsTest.java
@@ -40,17 +40,47 @@ import org.apache.cassandra.utils.ByteBufferUtil;
 import static org.apache.cassandra.Util.cellname;
 
 import org.junit.Assert;
+import org.junit.BeforeClass;
 import org.junit.Ignore;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 
 @RunWith(OrderedJUnit4ClassRunner.class)
-public class DefsTest extends SchemaLoader
+public class DefsTest
 {
+    private static final String KEYSPACE1 = "Keyspace1";
+    private static final String KEYSPACE3 = "Keyspace3";
+    private static final String KEYSPACE6 = "Keyspace6";
+    private static final String EMPTYKEYSPACE = "DefsTestEmptyKeyspace";
+    private static final String CF_STANDARD1 = "Standard1";
+    private static final String CF_STANDARD2 = "Standard2";
+    private static final String CF_INDEXED = "Indexed1";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.startGossiper();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD2));
+        SchemaLoader.createKeyspace(KEYSPACE3, true, false,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(5),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1),
+                                    SchemaLoader.indexCFMD(KEYSPACE3, CF_INDEXED, true));
+        SchemaLoader.createKeyspace(KEYSPACE6,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.indexCFMD(KEYSPACE6, CF_INDEXED, true));
+    }
+
     @Test
     public void testCFMetaDataApply() throws ConfigurationException
     {
-        CFMetaData cfm = new CFMetaData("Keyspace1",
+        CFMetaData cfm = new CFMetaData(KEYSPACE1,
                                         "TestApplyCFM_CF",
                                         ColumnFamilyType.Standard,
                                         new SimpleDenseCellNameType(BytesType.instance));
@@ -139,7 +169,7 @@ public class DefsTest extends SchemaLoader
     @Test
     public void addNewCfWithNullComment() throws ConfigurationException
     {
-        final String ks = "Keyspace1";
+        final String ks = KEYSPACE1;
         final String cf = "BrandNewCfWithNull";
         KSMetaData original = Schema.instance.getKSMetaData(ks);
 
@@ -155,7 +185,7 @@ public class DefsTest extends SchemaLoader
     @Test
     public void addNewCF() throws ConfigurationException
     {
-        final String ks = "Keyspace1";
+        final String ks = KEYSPACE1;
         final String cf = "BrandNewCf";
         KSMetaData original = Schema.instance.getKSMetaData(ks);
 
@@ -188,7 +218,7 @@ public class DefsTest extends SchemaLoader
     {
         DecoratedKey dk = Util.dk("dropCf");
         // sanity
-        final KSMetaData ks = Schema.instance.getKSMetaData("Keyspace1");
+        final KSMetaData ks = Schema.instance.getKSMetaData(KEYSPACE1);
         Assert.assertNotNull(ks);
         final CFMetaData cfm = ks.cfMetaData().get("Standard1");
         Assert.assertNotNull(cfm);
@@ -262,7 +292,7 @@ public class DefsTest extends SchemaLoader
     {
         DecoratedKey dk = Util.dk("dropKs");
         // sanity
-        final KSMetaData ks = Schema.instance.getKSMetaData("Keyspace1");
+        final KSMetaData ks = Schema.instance.getKSMetaData(KEYSPACE1);
         Assert.assertNotNull(ks);
         final CFMetaData cfm = ks.cfMetaData().get("Standard2");
         Assert.assertNotNull(cfm);
@@ -313,7 +343,7 @@ public class DefsTest extends SchemaLoader
     {
         DecoratedKey dk = Util.dk("dropKs");
         // sanity
-        final KSMetaData ks = Schema.instance.getKSMetaData("Keyspace3");
+        final KSMetaData ks = Schema.instance.getKSMetaData(KEYSPACE3);
         Assert.assertNotNull(ks);
         final CFMetaData cfm = ks.cfMetaData().get("Standard1");
         Assert.assertNotNull(cfm);
@@ -332,14 +362,14 @@ public class DefsTest extends SchemaLoader
     @Test
     public void createEmptyKsAddNewCf() throws ConfigurationException
     {
-        Assert.assertNull(Schema.instance.getKSMetaData("EmptyKeyspace"));
+        Assert.assertNull(Schema.instance.getKSMetaData(EMPTYKEYSPACE));
 
-        KSMetaData newKs = KSMetaData.testMetadata("EmptyKeyspace", SimpleStrategy.class, KSMetaData.optsWithRF(5));
+        KSMetaData newKs = KSMetaData.testMetadata(EMPTYKEYSPACE, SimpleStrategy.class, KSMetaData.optsWithRF(5));
 
         MigrationManager.announceNewKeyspace(newKs);
-        Assert.assertNotNull(Schema.instance.getKSMetaData("EmptyKeyspace"));
+        Assert.assertNotNull(Schema.instance.getKSMetaData(EMPTYKEYSPACE));
 
-        CFMetaData newCf = addTestCF("EmptyKeyspace", "AddedLater", "A new CF to add to an empty KS");
+        CFMetaData newCf = addTestCF(EMPTYKEYSPACE, "AddedLater", "A new CF to add to an empty KS");
 
         //should not exist until apply
         Assert.assertFalse(Schema.instance.getKSMetaData(newKs.name).cfMetaData().containsKey(newCf.cfName));
@@ -498,11 +528,11 @@ public class DefsTest extends SchemaLoader
     public void testDropIndex() throws ConfigurationException
     {
         // persist keyspace definition in the system keyspace
-        Schema.instance.getKSMetaData("Keyspace6").toSchema(System.currentTimeMillis()).apply();
-        ColumnFamilyStore cfs = Keyspace.open("Keyspace6").getColumnFamilyStore("Indexed1");
+        Schema.instance.getKSMetaData(KEYSPACE6).toSchema(System.currentTimeMillis()).apply();
+        ColumnFamilyStore cfs = Keyspace.open(KEYSPACE6).getColumnFamilyStore("Indexed1");
 
         // insert some data.  save the sstable descriptor so we can make sure it's marked for delete after the drop
-        Mutation rm = new Mutation("Keyspace6", ByteBufferUtil.bytes("k1"));
+        Mutation rm = new Mutation(KEYSPACE6, ByteBufferUtil.bytes("k1"));
         rm.add("Indexed1", cellname("notbirthdate"), ByteBufferUtil.bytes(1L), 0);
         rm.add("Indexed1", cellname("birthdate"), ByteBufferUtil.bytes(1L), 0);
         rm.apply();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/cql3/DeleteTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/DeleteTest.java b/test/unit/org/apache/cassandra/cql3/DeleteTest.java
index 3395dcc..bb34095 100644
--- a/test/unit/org/apache/cassandra/cql3/DeleteTest.java
+++ b/test/unit/org/apache/cassandra/cql3/DeleteTest.java
@@ -16,7 +16,7 @@ import org.junit.Test;
 
 import java.io.IOException;
 
-public class DeleteTest extends SchemaLoader
+public class DeleteTest
 {
 
     private static EmbeddedCassandraService cassandra;
@@ -35,7 +35,7 @@ public class DeleteTest extends SchemaLoader
     @BeforeClass()
     public static void setup() throws ConfigurationException, IOException
     {
-
+        SchemaLoader.loadSchema();
         Schema.instance.clear(); // Schema are now written on disk and will be reloaded
         cassandra = new EmbeddedCassandraService();
         cassandra.start();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/db/ArrayBackedSortedColumnsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/ArrayBackedSortedColumnsTest.java b/test/unit/org/apache/cassandra/db/ArrayBackedSortedColumnsTest.java
index 968fb93..46fe812 100644
--- a/test/unit/org/apache/cassandra/db/ArrayBackedSortedColumnsTest.java
+++ b/test/unit/org/apache/cassandra/db/ArrayBackedSortedColumnsTest.java
@@ -21,21 +21,39 @@
 package org.apache.cassandra.db;
 
 import java.util.*;
+
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import static org.junit.Assert.*;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.db.composites.*;
 import org.apache.cassandra.db.filter.ColumnSlice;
 import org.apache.cassandra.db.marshal.Int32Type;
 import org.apache.cassandra.utils.SearchIterator;
 
-public class ArrayBackedSortedColumnsTest extends SchemaLoader
+public class ArrayBackedSortedColumnsTest
 {
+    private static final String KEYSPACE1 = "ArrayBackedSortedColumnsTest";
+    private static final String CF_STANDARD1 = "Standard1";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1));
+    }
+
     @Test
     public void testAdd()
     {
@@ -45,7 +63,7 @@ public class ArrayBackedSortedColumnsTest extends SchemaLoader
 
     private CFMetaData metadata()
     {
-        return Schema.instance.getCFMetaData("Keyspace1", "Standard1");
+        return Schema.instance.getCFMetaData(KEYSPACE1, CF_STANDARD1);
     }
 
     private void testAddInternal(boolean reversed)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/db/BatchlogManagerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/BatchlogManagerTest.java b/test/unit/org/apache/cassandra/db/BatchlogManagerTest.java
index 57f6844..6f36fe9 100644
--- a/test/unit/org/apache/cassandra/db/BatchlogManagerTest.java
+++ b/test/unit/org/apache/cassandra/db/BatchlogManagerTest.java
@@ -23,17 +23,21 @@ import java.util.List;
 import java.util.concurrent.ExecutionException;
 
 import com.google.common.collect.Lists;
+import org.junit.BeforeClass;
 import org.junit.Before;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
 import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.cql3.QueryProcessor;
 import org.apache.cassandra.cql3.UntypedResultSet;
 import org.apache.cassandra.db.composites.CellNameType;
 import org.apache.cassandra.db.commitlog.ReplayPosition;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.locator.TokenMetadata;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.UUIDGen;
@@ -43,8 +47,25 @@ import static org.junit.Assert.assertTrue;
 
 import static org.apache.cassandra.utils.ByteBufferUtil.bytes;
 
-public class BatchlogManagerTest extends SchemaLoader
+public class BatchlogManagerTest
 {
+    private static final String KEYSPACE1 = "BatchlogManagerTest1";
+    private static final String CF_STANDARD1 = "Standard1";
+    private static final String CF_STANDARD2 = "Standard2";
+    private static final String CF_STANDARD3 = "Standard3";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                SimpleStrategy.class,
+                KSMetaData.optsWithRF(1),
+                SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1),
+                SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD2),
+                SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD3));
+    }
+
     @Before
     public void setUp() throws Exception
     {
@@ -62,10 +83,10 @@ public class BatchlogManagerTest extends SchemaLoader
 
         // Generate 1000 mutations and put them all into the batchlog.
         // Half (500) ready to be replayed, half not.
-        CellNameType comparator = Keyspace.open("Keyspace1").getColumnFamilyStore("Standard1").metadata.comparator;
+        CellNameType comparator = Keyspace.open(KEYSPACE1).getColumnFamilyStore("Standard1").metadata.comparator;
         for (int i = 0; i < 1000; i++)
         {
-            Mutation mutation = new Mutation("Keyspace1", bytes(i));
+            Mutation mutation = new Mutation(KEYSPACE1, bytes(i));
             mutation.add("Standard1", comparator.makeCellName(bytes(i)), bytes(i), System.currentTimeMillis());
 
             long timestamp = i < 500
@@ -94,7 +115,7 @@ public class BatchlogManagerTest extends SchemaLoader
 
         for (int i = 0; i < 1000; i++)
         {
-            UntypedResultSet result = QueryProcessor.executeInternal(String.format("SELECT * FROM \"Keyspace1\".\"Standard1\" WHERE key = intAsBlob(%d)", i));
+            UntypedResultSet result = QueryProcessor.executeInternal(String.format("SELECT * FROM \"%s\".\"%s\" WHERE key = intAsBlob(%d)", KEYSPACE1, CF_STANDARD1, i));
             if (i < 500)
             {
                 assertEquals(bytes(i), result.one().getBytes("key"));
@@ -108,23 +129,23 @@ public class BatchlogManagerTest extends SchemaLoader
         }
 
         // Ensure that no stray mutations got somehow applied.
-        UntypedResultSet result = QueryProcessor.executeInternal(String.format("SELECT count(*) FROM \"Keyspace1\".\"Standard1\""));
+        UntypedResultSet result = QueryProcessor.executeInternal(String.format("SELECT count(*) FROM \"%s\".\"%s\"", KEYSPACE1, CF_STANDARD1));
         assertEquals(500, result.one().getLong("count"));
     }
 
     @Test
     public void testTruncatedReplay() throws InterruptedException, ExecutionException
     {
-        CellNameType comparator2 = Keyspace.open("Keyspace1").getColumnFamilyStore("Standard2").metadata.comparator;
-        CellNameType comparator3 = Keyspace.open("Keyspace1").getColumnFamilyStore("Standard3").metadata.comparator;
+        CellNameType comparator2 = Keyspace.open(KEYSPACE1).getColumnFamilyStore("Standard2").metadata.comparator;
+        CellNameType comparator3 = Keyspace.open(KEYSPACE1).getColumnFamilyStore("Standard3").metadata.comparator;
         // Generate 2000 mutations (1000 batchlog entries) and put them all into the batchlog.
         // Each batchlog entry with a mutation for Standard2 and Standard3.
         // In the middle of the process, 'truncate' Standard2.
         for (int i = 0; i < 1000; i++)
         {
-            Mutation mutation1 = new Mutation("Keyspace1", bytes(i));
+            Mutation mutation1 = new Mutation(KEYSPACE1, bytes(i));
             mutation1.add("Standard2", comparator2.makeCellName(bytes(i)), bytes(i), 0);
-            Mutation mutation2 = new Mutation("Keyspace1", bytes(i));
+            Mutation mutation2 = new Mutation(KEYSPACE1, bytes(i));
             mutation2.add("Standard3", comparator3.makeCellName(bytes(i)), bytes(i), 0);
             List<Mutation> mutations = Lists.newArrayList(mutation1, mutation2);
 
@@ -132,7 +153,7 @@ public class BatchlogManagerTest extends SchemaLoader
             long timestamp = System.currentTimeMillis() - DatabaseDescriptor.getWriteRpcTimeout() * 2;
 
             if (i == 500)
-                SystemKeyspace.saveTruncationRecord(Keyspace.open("Keyspace1").getColumnFamilyStore("Standard2"),
+                SystemKeyspace.saveTruncationRecord(Keyspace.open(KEYSPACE1).getColumnFamilyStore("Standard2"),
                                                     timestamp,
                                                     ReplayPosition.NONE);
 
@@ -158,7 +179,7 @@ public class BatchlogManagerTest extends SchemaLoader
         // We should see half of Standard2-targeted mutations written after the replay and all of Standard3 mutations applied.
         for (int i = 0; i < 1000; i++)
         {
-            UntypedResultSet result = QueryProcessor.executeInternal(String.format("SELECT * FROM \"Keyspace1\".\"Standard2\" WHERE key = intAsBlob(%d)", i));
+            UntypedResultSet result = QueryProcessor.executeInternal(String.format("SELECT * FROM \"%s\".\"%s\" WHERE key = intAsBlob(%d)", KEYSPACE1, CF_STANDARD2,i));
             if (i >= 500)
             {
                 assertEquals(bytes(i), result.one().getBytes("key"));
@@ -173,7 +194,7 @@ public class BatchlogManagerTest extends SchemaLoader
 
         for (int i = 0; i < 1000; i++)
         {
-            UntypedResultSet result = QueryProcessor.executeInternal(String.format("SELECT * FROM \"Keyspace1\".\"Standard3\" WHERE key = intAsBlob(%d)", i));
+            UntypedResultSet result = QueryProcessor.executeInternal(String.format("SELECT * FROM \"%s\".\"%s\" WHERE key = intAsBlob(%d)", KEYSPACE1, CF_STANDARD3, i));
             assertEquals(bytes(i), result.one().getBytes("key"));
             assertEquals(bytes(i), result.one().getBytes("column1"));
             assertEquals(bytes(i), result.one().getBytes("value"));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/db/CleanupTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/CleanupTest.java b/test/unit/org/apache/cassandra/db/CleanupTest.java
index 79a8470..213c5b8 100644
--- a/test/unit/org/apache/cassandra/db/CleanupTest.java
+++ b/test/unit/org/apache/cassandra/db/CleanupTest.java
@@ -31,22 +31,26 @@ import java.util.concurrent.TimeUnit;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.db.filter.IDiskAtomFilter;
 import org.apache.cassandra.db.columniterator.IdentityQueryFilter;
 import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.db.index.SecondaryIndex;
 import org.apache.cassandra.dht.BytesToken;
 import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.sstable.SSTableReader;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.locator.TokenMetadata;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.ByteBufferUtil;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
-public class CleanupTest extends SchemaLoader
+public class CleanupTest
 {
     public static final int LOOPS = 200;
-    public static final String KEYSPACE1 = "Keyspace1";
+    public static final String KEYSPACE1 = "CleanupTest1";
     public static final String CF1 = "Indexed1";
     public static final String CF2 = "Standard1";
     public static final ByteBuffer COLUMN = ByteBufferUtil.bytes("birthdate");
@@ -57,6 +61,17 @@ public class CleanupTest extends SchemaLoader
         VALUE.flip();
     }
 
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF2),
+                                    SchemaLoader.indexCFMD(KEYSPACE1, CF1, true));
+    }
+
     @Test
     public void testCleanup() throws ExecutionException, InterruptedException
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/d2a3827a/test/unit/org/apache/cassandra/db/CollationControllerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/CollationControllerTest.java b/test/unit/org/apache/cassandra/db/CollationControllerTest.java
index 22c60b8..923cb9e 100644
--- a/test/unit/org/apache/cassandra/db/CollationControllerTest.java
+++ b/test/unit/org/apache/cassandra/db/CollationControllerTest.java
@@ -18,24 +18,43 @@
 */
 package org.apache.cassandra.db;
 
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
+import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.db.composites.CellName;
 import org.apache.cassandra.db.filter.QueryFilter;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 
 import static org.junit.Assert.assertEquals;
 
-public class CollationControllerTest extends SchemaLoader
+public class CollationControllerTest
 {
+    private static final String KEYSPACE1 = "CollationControllerTest";
+    private static final String CF = "Standard1";
+    private static final String CFGCGRACE = "StandardGCGS0";
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                                    SimpleStrategy.class,
+                                    KSMetaData.optsWithRF(1),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF),
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CFGCGRACE).gcGraceSeconds(0));
+    }
+
     @Test
     public void getTopLevelColumnsSkipsSSTablesModifiedBeforeRowDelete() 
     {
-        Keyspace keyspace = Keyspace.open("Keyspace1");
-        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("Standard1");
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF);
         Mutation rm;
         DecoratedKey dk = Util.dk("key1");
         
@@ -83,8 +102,8 @@ public class CollationControllerTest extends SchemaLoader
     @Test
     public void ensureTombstonesAppliedAfterGCGS()
     {
-        Keyspace keyspace = Keyspace.open("Keyspace1");
-        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("StandardGCGS0");
+        Keyspace keyspace = Keyspace.open(KEYSPACE1);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CFGCGRACE);
         cfs.disableAutoCompaction();
 
         Mutation rm;