You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by sl...@apache.org on 2015/06/30 12:47:29 UTC

[05/51] [partial] cassandra git commit: Storage engine refactor, a.k.a CASSANDRA-8099

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a991b648/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 46f4a9a..6aca407 100644
--- a/test/unit/org/apache/cassandra/SchemaLoader.java
+++ b/test/unit/org/apache/cassandra/SchemaLoader.java
@@ -19,7 +19,6 @@ package org.apache.cassandra;
 
 import java.io.File;
 import java.io.IOException;
-import java.nio.ByteBuffer;
 import java.util.*;
 
 import org.junit.After;
@@ -29,6 +28,7 @@ import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.cache.CachingOptions;
 import org.apache.cassandra.config.*;
+import org.apache.cassandra.cql3.CQLTester;
 import org.apache.cassandra.cql3.ColumnIdentifier;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.commitlog.CommitLog;
@@ -45,6 +45,7 @@ import org.apache.cassandra.locator.AbstractReplicationStrategy;
 import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.service.MigrationManager;
 import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
 
 public class SchemaLoader
 {
@@ -71,26 +72,7 @@ public class SchemaLoader
 
     public static void prepareServer()
     {
-        // Cleanup first
-        try
-        {
-            cleanupAndLeaveDirs();
-        }
-        catch (IOException e)
-        {
-            logger.error("Failed to cleanup and recreate directories and files.");
-            throw new RuntimeException(e);
-        }
-
-        Thread.setDefaultUncaughtExceptionHandler(new Thread.UncaughtExceptionHandler()
-        {
-            public void uncaughtException(Thread t, Throwable e)
-            {
-                logger.error("Fatal exception in thread " + t, e);
-            }
-        });
-
-        Keyspace.setInitialized();
+       CQLTester.prepareServer(false);
     }
 
     public static void startGossiper()
@@ -143,172 +125,169 @@ public class SchemaLoader
 
         // Keyspace 1
         schema.add(KSMetaData.testMetadata(ks1,
-                                           simple,
-                                           opts_rf1,
-
-                                           // Column Families
-                                           standardCFMD(ks1, "Standard1").compactionStrategyOptions(compactionOptions),
-                                           standardCFMD(ks1, "Standard2"),
-                                           standardCFMD(ks1, "Standard3"),
-                                           standardCFMD(ks1, "Standard4"),
-                                           standardCFMD(ks1, "StandardGCGS0").gcGraceSeconds(0),
-                                           standardCFMD(ks1, "StandardLong1"),
-                                           standardCFMD(ks1, "StandardLong2"),
-                                           CFMetaData.denseCFMetaData(ks1, "ValuesWithQuotes", BytesType.instance).defaultValidator(UTF8Type.instance),
-                                           superCFMD(ks1, "Super1", LongType.instance),
-                                           superCFMD(ks1, "Super2", LongType.instance),
-                                           superCFMD(ks1, "Super3", LongType.instance),
-                                           superCFMD(ks1, "Super4", UTF8Type.instance),
-                                           superCFMD(ks1, "Super5", bytes),
-                                           superCFMD(ks1, "Super6", LexicalUUIDType.instance, UTF8Type.instance),
-                                           indexCFMD(ks1, "Indexed1", true),
-                                           indexCFMD(ks1, "Indexed2", false),
-                                           CFMetaData.denseCFMetaData(ks1, "StandardInteger1", IntegerType.instance),
-                                           CFMetaData.denseCFMetaData(ks1, "StandardLong3", IntegerType.instance),
-                                           CFMetaData.denseCFMetaData(ks1, "Counter1", bytes).defaultValidator(CounterColumnType.instance),
-                                           CFMetaData.denseCFMetaData(ks1, "SuperCounter1", bytes, bytes).defaultValidator(CounterColumnType.instance),
-                                           superCFMD(ks1, "SuperDirectGC", BytesType.instance).gcGraceSeconds(0),
-                                           jdbcSparseCFMD(ks1, "JdbcInteger", IntegerType.instance).addColumnDefinition(integerColumn(ks1, "JdbcInteger")),
-                                           jdbcSparseCFMD(ks1, "JdbcUtf8", UTF8Type.instance).addColumnDefinition(utf8Column(ks1, "JdbcUtf8")),
-                                           jdbcCFMD(ks1, "JdbcLong", LongType.instance),
-                                           jdbcCFMD(ks1, "JdbcBytes", bytes),
-                                           jdbcCFMD(ks1, "JdbcAscii", AsciiType.instance),
-                                           CFMetaData.denseCFMetaData(ks1, "StandardComposite", composite),
-                                           CFMetaData.denseCFMetaData(ks1, "StandardComposite2", compositeMaxMin),
-                                           CFMetaData.denseCFMetaData(ks1, "StandardDynamicComposite", dynamicComposite),
-                                           standardCFMD(ks1, "StandardLeveled")
-                                                                               .compactionStrategyClass(LeveledCompactionStrategy.class)
-                                                                               .compactionStrategyOptions(leveledOptions),
-                                           standardCFMD(ks1, "legacyleveled")
-                                                                               .compactionStrategyClass(LeveledCompactionStrategy.class)
-                                                                               .compactionStrategyOptions(leveledOptions),
-                                           standardCFMD(ks1, "StandardLowIndexInterval").minIndexInterval(8)
-                                                                                        .maxIndexInterval(256)
-                                                                                        .caching(CachingOptions.NONE),
-                                           standardCFMD(ks1, "UUIDKeys").keyValidator(UUIDType.instance),
-                                           CFMetaData.denseCFMetaData(ks1, "MixedTypes", LongType.instance).keyValidator(UUIDType.instance).defaultValidator(BooleanType.instance),
-                                           CFMetaData.denseCFMetaData(ks1, "MixedTypesComposite", composite).keyValidator(composite).defaultValidator(BooleanType.instance),
-                                           standardCFMD(ks1, "AsciiKeys").keyValidator(AsciiType.instance)
+                simple,
+                opts_rf1,
+
+                // Column Families
+                standardCFMD(ks1, "Standard1").compactionStrategyOptions(compactionOptions),
+                standardCFMD(ks1, "Standard2"),
+                standardCFMD(ks1, "Standard3"),
+                standardCFMD(ks1, "Standard4"),
+                standardCFMD(ks1, "StandardGCGS0").gcGraceSeconds(0),
+                standardCFMD(ks1, "StandardLong1"),
+                standardCFMD(ks1, "StandardLong2"),
+                //CFMetaData.Builder.create(ks1, "ValuesWithQuotes").build(),
+                superCFMD(ks1, "Super1", LongType.instance),
+                superCFMD(ks1, "Super2", LongType.instance),
+                superCFMD(ks1, "Super3", LongType.instance),
+                superCFMD(ks1, "Super4", UTF8Type.instance),
+                superCFMD(ks1, "Super5", bytes),
+                superCFMD(ks1, "Super6", LexicalUUIDType.instance, UTF8Type.instance),
+                keysIndexCFMD(ks1, "Indexed1", true),
+                keysIndexCFMD(ks1, "Indexed2", false),
+                //CFMetaData.Builder.create(ks1, "StandardInteger1").withColumnNameComparator(IntegerType.instance).build(),
+                //CFMetaData.Builder.create(ks1, "StandardLong3").withColumnNameComparator(IntegerType.instance).build(),
+                //CFMetaData.Builder.create(ks1, "Counter1", false, false, true).build(),
+                //CFMetaData.Builder.create(ks1, "SuperCounter1", false, false, true, true).build(),
+                superCFMD(ks1, "SuperDirectGC", BytesType.instance).gcGraceSeconds(0),
+                jdbcCFMD(ks1, "JdbcInteger", IntegerType.instance).addColumnDefinition(integerColumn(ks1, "JdbcInteger")),
+                jdbcCFMD(ks1, "JdbcUtf8", UTF8Type.instance).addColumnDefinition(utf8Column(ks1, "JdbcUtf8")),
+                jdbcCFMD(ks1, "JdbcLong", LongType.instance),
+                jdbcCFMD(ks1, "JdbcBytes", bytes),
+                jdbcCFMD(ks1, "JdbcAscii", AsciiType.instance),
+                //CFMetaData.Builder.create(ks1, "StandardComposite", false, true, false).withColumnNameComparator(composite).build(),
+                //CFMetaData.Builder.create(ks1, "StandardComposite2", false, true, false).withColumnNameComparator(compositeMaxMin).build(),
+                //CFMetaData.Builder.create(ks1, "StandardDynamicComposite", false, true, false).withColumnNameComparator(dynamicComposite).build(),
+                standardCFMD(ks1, "StandardLeveled")
+                        .compactionStrategyClass(LeveledCompactionStrategy.class)
+                        .compactionStrategyOptions(leveledOptions),
+                standardCFMD(ks1, "legacyleveled")
+                        .compactionStrategyClass(LeveledCompactionStrategy.class)
+                        .compactionStrategyOptions(leveledOptions),
+                standardCFMD(ks1, "StandardLowIndexInterval").minIndexInterval(8)
+                        .maxIndexInterval(256)
+                        .caching(CachingOptions.NONE)
+                //CFMetaData.Builder.create(ks1, "UUIDKeys").addPartitionKey("key",UUIDType.instance).build(),
+                //CFMetaData.Builder.create(ks1, "MixedTypes").withColumnNameComparator(LongType.instance).addPartitionKey("key", UUIDType.instance).build(),
+                //CFMetaData.Builder.create(ks1, "MixedTypesComposite", false, true, false).withColumnNameComparator(composite).addPartitionKey("key", composite).build(),
+                //CFMetaData.Builder.create(ks1, "AsciiKeys").addPartitionKey("key", AsciiType.instance).build()
         ));
 
         // Keyspace 2
         schema.add(KSMetaData.testMetadata(ks2,
-                                           simple,
-                                           opts_rf1,
-
-                                           // Column Families
-                                           standardCFMD(ks2, "Standard1"),
-                                           standardCFMD(ks2, "Standard3"),
-                                           superCFMD(ks2, "Super3", bytes),
-                                           superCFMD(ks2, "Super4", TimeUUIDType.instance),
-                                           indexCFMD(ks2, "Indexed1", true),
-                                           compositeIndexCFMD(ks2, "Indexed2", true),
-                                           compositeIndexCFMD(ks2, "Indexed3", true).gcGraceSeconds(0)));
+                simple,
+                opts_rf1,
+
+                // Column Families
+                standardCFMD(ks2, "Standard1"),
+                standardCFMD(ks2, "Standard3"),
+                superCFMD(ks2, "Super3", bytes),
+                superCFMD(ks2, "Super4", TimeUUIDType.instance),
+                keysIndexCFMD(ks2, "Indexed1", true),
+                compositeIndexCFMD(ks2, "Indexed2", true),
+                compositeIndexCFMD(ks2, "Indexed3", true).gcGraceSeconds(0)));
 
         // Keyspace 3
         schema.add(KSMetaData.testMetadata(ks3,
-                                           simple,
-                                           opts_rf5,
+                simple,
+                opts_rf5,
 
-                                           // Column Families
-                                           standardCFMD(ks3, "Standard1"),
-                                           indexCFMD(ks3, "Indexed1", true)));
+                // Column Families
+                standardCFMD(ks3, "Standard1"),
+                keysIndexCFMD(ks3, "Indexed1", true)));
 
         // Keyspace 4
         schema.add(KSMetaData.testMetadata(ks4,
-                                           simple,
-                                           opts_rf3,
+                simple,
+                opts_rf3,
 
-                                           // Column Families
-                                           standardCFMD(ks4, "Standard1"),
-                                           standardCFMD(ks4, "Standard3"),
-                                           superCFMD(ks4, "Super3", bytes),
-                                           superCFMD(ks4, "Super4", TimeUUIDType.instance),
-                                           CFMetaData.denseCFMetaData(ks4, "Super5", TimeUUIDType.instance, bytes)));
+                // Column Families
+                standardCFMD(ks4, "Standard1"),
+                standardCFMD(ks4, "Standard3"),
+                superCFMD(ks4, "Super3", bytes),
+                superCFMD(ks4, "Super4", TimeUUIDType.instance),
+                superCFMD(ks4, "Super5", TimeUUIDType.instance, BytesType.instance)));
 
         // Keyspace 5
         schema.add(KSMetaData.testMetadata(ks5,
-                                           simple,
-                                           opts_rf2,
-                                           standardCFMD(ks5, "Standard1"),
-                                           standardCFMD(ks5, "Counter1")
-                                                   .defaultValidator(CounterColumnType.instance)));
+                simple,
+                opts_rf2,
+                standardCFMD(ks5, "Standard1")));
 
         // Keyspace 6
         schema.add(KSMetaData.testMetadata(ks6,
-                                           simple,
-                                           opts_rf1,
-                                           indexCFMD(ks6, "Indexed1", true)));
+                simple,
+                opts_rf1,
+                keysIndexCFMD(ks6, "Indexed1", true)));
 
         // KeyCacheSpace
         schema.add(KSMetaData.testMetadata(ks_kcs,
-                                           simple,
-                                           opts_rf1,
-                                           standardCFMD(ks_kcs, "Standard1"),
-                                           standardCFMD(ks_kcs, "Standard2"),
-                                           standardCFMD(ks_kcs, "Standard3")));
+                simple,
+                opts_rf1,
+                standardCFMD(ks_kcs, "Standard1"),
+                standardCFMD(ks_kcs, "Standard2"),
+                standardCFMD(ks_kcs, "Standard3")));
 
         // RowCacheSpace
         schema.add(KSMetaData.testMetadata(ks_rcs,
-                                           simple,
-                                           opts_rf1,
-                                           standardCFMD(ks_rcs, "CFWithoutCache").caching(CachingOptions.NONE),
-                                           standardCFMD(ks_rcs, "CachedCF").caching(CachingOptions.ALL),
-                                           standardCFMD(ks_rcs, "CachedIntCF").
-                                                   defaultValidator(IntegerType.instance).
-                                                   caching(new CachingOptions(new CachingOptions.KeyCache(CachingOptions.KeyCache.Type.ALL),
-                                                                                  new CachingOptions.RowCache(CachingOptions.RowCache.Type.HEAD, 100)))));
+                simple,
+                opts_rf1,
+                standardCFMD(ks_rcs, "CFWithoutCache").caching(CachingOptions.NONE),
+                standardCFMD(ks_rcs, "CachedCF").caching(CachingOptions.ALL),
+                standardCFMD(ks_rcs, "CachedIntCF").
+                        caching(new CachingOptions(new CachingOptions.KeyCache(CachingOptions.KeyCache.Type.ALL),
+                                new CachingOptions.RowCache(CachingOptions.RowCache.Type.HEAD, 100)))));
 
         // CounterCacheSpace
-        schema.add(KSMetaData.testMetadata(ks_ccs,
-                                           simple,
-                                           opts_rf1,
-                                           standardCFMD(ks_ccs, "Counter1").defaultValidator(CounterColumnType.instance),
-                                           standardCFMD(ks_ccs, "Counter2").defaultValidator(CounterColumnType.instance)));
+        /*schema.add(KSMetaData.testMetadata(ks_ccs,
+                simple,
+                opts_rf1,
+                CFMetaData.Builder.create(ks_ccs, "Counter1", false, false, true).build(),
+                CFMetaData.Builder.create(ks_ccs, "Counter1", false, false, true).build()));*/
 
         schema.add(KSMetaData.testMetadataNotDurable(ks_nocommit,
-                                                     simple,
-                                                     opts_rf1,
-                                                     standardCFMD(ks_nocommit, "Standard1")));
+                simple,
+                opts_rf1,
+                standardCFMD(ks_nocommit, "Standard1")));
 
         // PerRowSecondaryIndexTest
         schema.add(KSMetaData.testMetadata(ks_prsi,
-                                           simple,
-                                           opts_rf1,
-                                           perRowIndexedCFMD(ks_prsi, "Indexed1")));
+                simple,
+                opts_rf1,
+                perRowIndexedCFMD(ks_prsi, "Indexed1")));
 
         // CQLKeyspace
         schema.add(KSMetaData.testMetadata(ks_cql,
-                                           simple,
-                                           opts_rf1,
-
-                                           // Column Families
-                                           CFMetaData.compile("CREATE TABLE table1 ("
-                                                              + "k int PRIMARY KEY,"
-                                                              + "v1 text,"
-                                                              + "v2 int"
-                                                              + ")", ks_cql),
-
-                                           CFMetaData.compile("CREATE TABLE table2 ("
-                                                              + "k text,"
-                                                              + "c text,"
-                                                              + "v text,"
-                                                              + "PRIMARY KEY (k, c))", ks_cql),
-                                           CFMetaData.compile("CREATE TABLE foo ("
-                                                   + "bar text, "
-                                                   + "baz text, "
-                                                   + "qux text, "
-                                                   + "PRIMARY KEY(bar, baz) ) "
-                                                   + "WITH COMPACT STORAGE", ks_cql),
-                                           CFMetaData.compile("CREATE TABLE foofoo ("
-                                                   + "bar text, "
-                                                   + "baz text, "
-                                                   + "qux text, "
-                                                   + "quz text, "
-                                                   + "foo text, "
-                                                   + "PRIMARY KEY((bar, baz), qux, quz) ) "
-                                                   + "WITH COMPACT STORAGE", ks_cql)
-                                           ));
+                simple,
+                opts_rf1,
+
+                // Column Families
+                CFMetaData.compile("CREATE TABLE table1 ("
+                        + "k int PRIMARY KEY,"
+                        + "v1 text,"
+                        + "v2 int"
+                        + ")", ks_cql),
+
+                CFMetaData.compile("CREATE TABLE table2 ("
+                        + "k text,"
+                        + "c text,"
+                        + "v text,"
+                        + "PRIMARY KEY (k, c))", ks_cql),
+                CFMetaData.compile("CREATE TABLE foo ("
+                        + "bar text, "
+                        + "baz text, "
+                        + "qux text, "
+                        + "PRIMARY KEY(bar, baz) ) "
+                        + "WITH COMPACT STORAGE", ks_cql),
+                CFMetaData.compile("CREATE TABLE foofoo ("
+                        + "bar text, "
+                        + "baz text, "
+                        + "qux text, "
+                        + "quz text, "
+                        + "foo text, "
+                        + "PRIMARY KEY((bar, baz), qux, quz) ) "
+                        + "WITH COMPACT STORAGE", ks_cql)
+        ));
 
 
         if (Boolean.parseBoolean(System.getProperty("cassandra.test.compression", "false")))
@@ -344,7 +323,7 @@ public class SchemaLoader
     {
         return new ColumnDefinition(ksName,
                                     cfName,
-                                    new ColumnIdentifier(IntegerType.instance.fromString("42"), IntegerType.instance),
+                                    ColumnIdentifier.getInterned(IntegerType.instance.fromString("42"), IntegerType.instance),
                                     UTF8Type.instance,
                                     null,
                                     null,
@@ -357,7 +336,7 @@ public class SchemaLoader
     {
         return new ColumnDefinition(ksName,
                                     cfName,
-                                    new ColumnIdentifier("fortytwo", true),
+                                    ColumnIdentifier.getInterned("fortytwo", true),
                                     UTF8Type.instance,
                                     null,
                                     null,
@@ -372,10 +351,11 @@ public class SchemaLoader
                                                       SecondaryIndex.CUSTOM_INDEX_OPTION_NAME,
                                                       PerRowSecondaryIndexTest.TestIndex.class.getName());
 
-        CFMetaData cfm =  CFMetaData.sparseCFMetaData(ksName, cfName, AsciiType.instance).keyValidator(AsciiType.instance);
+        CFMetaData cfm =  CFMetaData.Builder.create(ksName, cfName)
+                .addPartitionKey("key", AsciiType.instance)
+                .build();
 
-        ByteBuffer cName = ByteBufferUtil.bytes("indexed");
-        return cfm.addOrReplaceColumnDefinition(ColumnDefinition.regularDef(cfm, cName, AsciiType.instance, null)
+        return cfm.addOrReplaceColumnDefinition(ColumnDefinition.regularDef(ksName, cfName, "indexed", AsciiType.instance, null)
                                                                 .setIndex("indexe1", IndexType.CUSTOM, indexOptions));
     }
 
@@ -390,57 +370,128 @@ public class SchemaLoader
         }
     }
 
-    public static CFMetaData standardCFMD(String ksName, String cfName)
+    public static CFMetaData counterCFMD(String ksName, String cfName)
     {
-        return CFMetaData.denseCFMetaData(ksName, cfName, BytesType.instance).compressionParameters(getCompressionParameters());
+        return CFMetaData.Builder.create(ksName, cfName, false, true, true)
+                .addPartitionKey("key", AsciiType.instance)
+                .addClusteringColumn("name", AsciiType.instance)
+                .addRegularColumn("val", CounterColumnType.instance)
+                .addRegularColumn("val2", CounterColumnType.instance)
+                .build()
+                .compressionParameters(getCompressionParameters());
     }
 
-    public static CFMetaData standardCFMD(String ksName, String cfName, AbstractType<?> comparator)
+    public static CFMetaData standardCFMD(String ksName, String cfName)
     {
-        return CFMetaData.denseCFMetaData(ksName, cfName, comparator).compressionParameters(getCompressionParameters());
+        return standardCFMD(ksName, cfName, 1, AsciiType.instance);
     }
 
-    public static CFMetaData superCFMD(String ksName, String cfName, AbstractType subcc)
+    public static CFMetaData standardCFMD(String ksName, String cfName, int columnCount, AbstractType<?> keyType)
     {
-        return superCFMD(ksName, cfName, BytesType.instance, subcc).compressionParameters(getCompressionParameters());
+        return standardCFMD(ksName, cfName, columnCount, keyType, AsciiType.instance);
     }
 
-    public static CFMetaData superCFMD(String ksName, String cfName, AbstractType cc, AbstractType subcc)
+    public static CFMetaData standardCFMD(String ksName, String cfName, int columnCount, AbstractType<?> keyType, AbstractType<?> valType)
     {
-        return CFMetaData.denseCFMetaData(ksName, cfName, cc, subcc).compressionParameters(getCompressionParameters());
+        return standardCFMD(ksName, cfName, columnCount, keyType, valType, AsciiType.instance);
     }
 
-    public static CFMetaData indexCFMD(String ksName, String cfName, final Boolean withIdxType) throws ConfigurationException
+    public static CFMetaData standardCFMD(String ksName, String cfName, int columnCount, AbstractType<?> keyType, AbstractType<?> valType, AbstractType<?> clusteringType)
     {
-        CFMetaData cfm = CFMetaData.sparseCFMetaData(ksName, cfName, BytesType.instance).keyValidator(AsciiType.instance);
+        CFMetaData.Builder builder = CFMetaData.Builder.create(ksName, cfName)
+                .addPartitionKey("key", keyType)
+                .addClusteringColumn("name", clusteringType)
+                .addRegularColumn("val", valType);
 
-        ByteBuffer cName = ByteBufferUtil.bytes("birthdate");
-        IndexType keys = withIdxType ? IndexType.KEYS : null;
-        return cfm.addColumnDefinition(ColumnDefinition.regularDef(cfm, cName, LongType.instance, null)
-                                                       .setIndex(withIdxType ? ByteBufferUtil.bytesToHex(cName) : null, keys, null))
-                                      .compressionParameters(getCompressionParameters());
+        for (int i = 0; i < columnCount; i++)
+            builder.addRegularColumn("val" + i, AsciiType.instance);
+
+        return builder.build()
+               .compressionParameters(getCompressionParameters());
     }
 
-    public static CFMetaData compositeIndexCFMD(String ksName, String cfName, final Boolean withIdxType) throws ConfigurationException
+    public static CFMetaData denseCFMD(String ksName, String cfName)
+    {
+        return denseCFMD(ksName, cfName, AsciiType.instance);
+    }
+    public static CFMetaData denseCFMD(String ksName, String cfName, AbstractType cc)
+    {
+        return denseCFMD(ksName, cfName, cc, null);
+    }
+    public static CFMetaData denseCFMD(String ksName, String cfName, AbstractType cc, AbstractType subcc)
     {
-        final CompositeType composite = CompositeType.getInstance(Arrays.asList(new AbstractType<?>[]{UTF8Type.instance, UTF8Type.instance})); 
-        CFMetaData cfm = CFMetaData.sparseCFMetaData(ksName, cfName, composite);
+        AbstractType comp = cc;
+        if (subcc != null)
+            comp = CompositeType.getInstance(Arrays.asList(new AbstractType<?>[]{cc, subcc}));
+
+        return CFMetaData.Builder.createDense(ksName, cfName, subcc != null, false)
+            .addPartitionKey("key", AsciiType.instance)
+            .addClusteringColumn("cols", comp)
+            .addRegularColumn("val", AsciiType.instance)
+            .build()
+            .compressionParameters(getCompressionParameters());
+    }
 
-        ByteBuffer cName = ByteBufferUtil.bytes("col1");
-        IndexType idxType = withIdxType ? IndexType.COMPOSITES : null;
-        return cfm.addColumnDefinition(ColumnDefinition.regularDef(cfm, cName, UTF8Type.instance, 1)
-                                                       .setIndex(withIdxType ? "col1_idx" : null, idxType, Collections.<String, String>emptyMap()))
-                                       .compressionParameters(getCompressionParameters());
+    // TODO: Fix superCFMD failing on legacy table creation. Seems to be applying composite comparator to partition key
+    public static CFMetaData superCFMD(String ksName, String cfName, AbstractType subcc)
+    {
+        return superCFMD(ksName, cfName, BytesType.instance, subcc);
     }
-    
-    private static CFMetaData jdbcCFMD(String ksName, String cfName, AbstractType comp)
+    public static CFMetaData superCFMD(String ksName, String cfName, AbstractType cc, AbstractType subcc)
     {
-        return CFMetaData.denseCFMetaData(ksName, cfName, comp).defaultValidator(comp).compressionParameters(getCompressionParameters());
+        return superCFMD(ksName, cfName, "cols", cc, subcc);
     }
+    public static CFMetaData superCFMD(String ksName, String cfName, String ccName, AbstractType cc, AbstractType subcc)
+    {
+        //This is busted
+//        return CFMetaData.Builder.createSuper(ksName, cfName, false)
+//            .addPartitionKey("0", BytesType.instance)
+//            .addClusteringColumn("1", cc)
+//            .addClusteringColumn("2", subcc)
+//            .addRegularColumn("3", AsciiType.instance)
+//            .build();
+        return standardCFMD(ksName, cfName);
 
-    public static CFMetaData jdbcSparseCFMD(String ksName, String cfName, AbstractType comp)
+    }
+    public static CFMetaData compositeIndexCFMD(String ksName, String cfName, boolean withIndex) throws ConfigurationException
+    {
+        // the withIndex flag exists to allow tests index creation
+        // on existing columns
+        CFMetaData cfm = CFMetaData.Builder.create(ksName, cfName)
+                .addPartitionKey("key", AsciiType.instance)
+                .addClusteringColumn("c1", AsciiType.instance)
+                .addRegularColumn("birthdate", LongType.instance)
+                .addRegularColumn("notbirthdate", LongType.instance)
+                .build();
+
+        if (withIndex)
+            cfm.getColumnDefinition(new ColumnIdentifier("birthdate", true))
+               .setIndex("birthdate_key_index", IndexType.COMPOSITES, Collections.EMPTY_MAP);
+
+        return cfm.compressionParameters(getCompressionParameters());
+    }
+    public static CFMetaData keysIndexCFMD(String ksName, String cfName, boolean withIndex) throws ConfigurationException
     {
-        return CFMetaData.sparseCFMetaData(ksName, cfName, comp).defaultValidator(comp).compressionParameters(getCompressionParameters());
+        CFMetaData cfm = CFMetaData.Builder.createDense(ksName, cfName, false, false)
+                                           .addPartitionKey("key", AsciiType.instance)
+                                           .addClusteringColumn("c1", AsciiType.instance)
+                                           .addStaticColumn("birthdate", LongType.instance)
+                                           .addStaticColumn("notbirthdate", LongType.instance)
+                                           .addRegularColumn("value", LongType.instance)
+                                           .build();
+
+        if (withIndex)
+            cfm.getColumnDefinition(new ColumnIdentifier("birthdate", true))
+               .setIndex("birthdate_composite_index", IndexType.KEYS, Collections.EMPTY_MAP);
+
+        return cfm.compressionParameters(getCompressionParameters());
+    }
+    
+    public static CFMetaData jdbcCFMD(String ksName, String cfName, AbstractType comp)
+    {
+        return CFMetaData.Builder.create(ksName, cfName).addPartitionKey("key", BytesType.instance)
+                                                        .build()
+                                                        .compressionParameters(getCompressionParameters());
     }
 
     public static CompressionParameters getCompressionParameters()
@@ -503,23 +554,13 @@ public class SchemaLoader
 
     public static void insertData(String keyspace, String columnFamily, int offset, int numberOfRows)
     {
-        for (int i = offset; i < offset + numberOfRows; i++)
-        {
-            ByteBuffer key = ByteBufferUtil.bytes("key" + i);
-            Mutation mutation = new Mutation(keyspace, key);
-            mutation.add(columnFamily, Util.cellname("col" + i), ByteBufferUtil.bytes("val" + i), System.currentTimeMillis());
-            mutation.applyUnsafe();
-        }
-    }
+        CFMetaData cfm = Schema.instance.getCFMetaData(keyspace, columnFamily);
 
-    /* usually used to populate the cache */
-    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++)
         {
-            DecoratedKey key = Util.dk("key" + i);
-            store.getColumnFamily(Util.namesQueryFilter(store, key, "col" + i));
+            RowUpdateBuilder builder = new RowUpdateBuilder(cfm, FBUtilities.timestampMicros(), ByteBufferUtil.bytes("key"+i));
+            builder.clustering(ByteBufferUtil.bytes("col"+ i)).add("val", ByteBufferUtil.bytes("val" + i));
+            builder.build().apply();
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a991b648/test/unit/org/apache/cassandra/UpdateBuilder.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/UpdateBuilder.java b/test/unit/org/apache/cassandra/UpdateBuilder.java
new file mode 100644
index 0000000..dc6b859
--- /dev/null
+++ b/test/unit/org/apache/cassandra/UpdateBuilder.java
@@ -0,0 +1,119 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra;
+
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.partitions.*;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.service.StorageService;
+
+
+/**
+ * Convenience object to create updates to a single partition.
+ *
+ * This is not unlike RowUpdateBuilder except that it allows to create update to multiple rows more easily.
+ * It is also aimed at unit tests so favor convenience over efficiency.
+ */
+public class UpdateBuilder
+{
+    private final PartitionUpdate update;
+    private RowUpdateBuilder currentRow;
+    private long timestamp = FBUtilities.timestampMicros();
+
+    private UpdateBuilder(CFMetaData metadata, DecoratedKey partitionKey)
+    {
+        this.update = new PartitionUpdate(metadata, partitionKey, metadata.partitionColumns(), 4);
+    }
+
+    public static UpdateBuilder create(CFMetaData metadata, Object... partitionKey)
+    {
+        return new UpdateBuilder(metadata, makeKey(metadata, partitionKey));
+    }
+
+    public UpdateBuilder withTimestamp(long timestamp)
+    {
+        this.timestamp = timestamp;
+        return this;
+    }
+
+    public UpdateBuilder newRow(Object... clustering)
+    {
+        maybeBuildCurrentRow();
+        currentRow = new RowUpdateBuilder(update, timestamp, 0);
+        if (clustering.length > 0)
+            currentRow.clustering(clustering);
+        return this;
+    }
+
+    public UpdateBuilder add(String column, Object value)
+    {
+        assert currentRow != null;
+        currentRow.add(column, value);
+        return this;
+    }
+
+    public PartitionUpdate build()
+    {
+        maybeBuildCurrentRow();
+        return update;
+    }
+
+    public IMutation makeMutation()
+    {
+        Mutation m = new Mutation(build());
+        return update.metadata().isCounter
+             ? new CounterMutation(m, ConsistencyLevel.ONE)
+             : m;
+    }
+
+    public void apply()
+    {
+        Mutation m = new Mutation(build());
+        if (update.metadata().isCounter)
+            new CounterMutation(m, ConsistencyLevel.ONE).apply();
+        else
+            m.apply();
+    }
+
+    public void applyUnsafe()
+    {
+        assert !update.metadata().isCounter : "Counters have currently no applyUnsafe() option";
+        new Mutation(build()).applyUnsafe();
+    }
+
+    private void maybeBuildCurrentRow()
+    {
+        if (currentRow != null)
+        {
+            currentRow.build();
+            currentRow = null;
+        }
+    }
+
+    private static DecoratedKey makeKey(CFMetaData metadata, Object[] partitionKey)
+    {
+        if (partitionKey.length == 1 && partitionKey[0] instanceof DecoratedKey)
+            return (DecoratedKey)partitionKey[0];
+
+        ByteBuffer key = CFMetaData.serializePartitionKey(metadata.getKeyValidatorAsClusteringComparator().make(partitionKey));
+        return StorageService.getPartitioner().decorateKey(key);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a991b648/test/unit/org/apache/cassandra/Util.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/Util.java b/test/unit/org/apache/cassandra/Util.java
index 2d59abb..423b3c0 100644
--- a/test/unit/org/apache/cassandra/Util.java
+++ b/test/unit/org/apache/cassandra/Util.java
@@ -3,8 +3,7 @@ package org.apache.cassandra;
  *
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
+ * distributed with this work for additional information * regarding copyright ownership.  The ASF licenses this file
  * to you under the Apache License, Version 2.0 (the
  * "License"); you may not use this file except in compliance
  * with the License.  You may obtain a copy of the License at
@@ -20,35 +19,37 @@ package org.apache.cassandra;
  *
  */
 
-import java.io.*;
+import java.io.EOFException;
 import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.nio.ByteBuffer;
-import java.nio.channels.FileChannel;
 import java.util.*;
 import java.util.concurrent.Callable;
 import java.util.concurrent.Future;
+import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.function.Supplier;
 
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.ImmutableSet;
+import com.google.common.base.Function;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterators;
+import org.apache.commons.lang3.StringUtils;
 
-import org.apache.cassandra.cache.CachingOptions;
 import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.KSMetaData;
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.cql3.Operator;
 import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.composites.*;
+import org.apache.cassandra.db.Slice.Bound;
+import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.db.compaction.AbstractCompactionTask;
 import org.apache.cassandra.db.compaction.CompactionManager;
-import org.apache.cassandra.db.columniterator.IdentityQueryFilter;
-import org.apache.cassandra.db.filter.IDiskAtomFilter;
-import org.apache.cassandra.db.filter.QueryFilter;
-import org.apache.cassandra.db.filter.SliceQueryFilter;
-import org.apache.cassandra.db.filter.NamesQueryFilter;
+import org.apache.cassandra.db.filter.*;
 import org.apache.cassandra.db.marshal.AbstractType;
-import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.db.partitions.*;
 import org.apache.cassandra.dht.*;
 import org.apache.cassandra.dht.RandomPartitioner.BigIntegerToken;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.gms.ApplicationState;
 import org.apache.cassandra.gms.Gossiper;
 import org.apache.cassandra.gms.VersionedValue;
@@ -56,19 +57,14 @@ import org.apache.cassandra.io.sstable.Component;
 import org.apache.cassandra.io.sstable.Descriptor;
 import org.apache.cassandra.io.sstable.IndexSummary;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
-import org.apache.cassandra.io.sstable.format.big.BigTableReader;
-import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
-import org.apache.cassandra.io.sstable.metadata.MetadataType;
-import org.apache.cassandra.io.sstable.metadata.StatsMetadata;
-import org.apache.cassandra.io.util.*;
-import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.AlwaysPresentFilter;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.CounterId;
-import org.apache.hadoop.fs.FileUtil;
+import org.apache.cassandra.utils.FBUtilities;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 
 public class Util
@@ -90,60 +86,26 @@ public class Util
         return StorageService.getPartitioner().decorateKey(key);
     }
 
-    public static RowPosition rp(String key)
+    public static PartitionPosition rp(String key)
     {
         return rp(key, StorageService.getPartitioner());
     }
 
-    public static RowPosition rp(String key, IPartitioner partitioner)
+    public static PartitionPosition rp(String key, IPartitioner partitioner)
     {
-        return RowPosition.ForKey.get(ByteBufferUtil.bytes(key), partitioner);
+        return PartitionPosition.ForKey.get(ByteBufferUtil.bytes(key), partitioner);
     }
 
-    public static CellName cellname(ByteBuffer... bbs)
+    public static Cell getRegularCell(CFMetaData metadata, Row row, String name)
     {
-        if (bbs.length == 1)
-            return CellNames.simpleDense(bbs[0]);
-        else
-            return CellNames.compositeDense(bbs);
+        ColumnDefinition column = metadata.getColumnDefinition(ByteBufferUtil.bytes(name));
+        assert column != null;
+        return row.getCell(column);
     }
 
-    public static CellName cellname(String... strs)
+    public static ClusteringPrefix clustering(ClusteringComparator comparator, Object... o)
     {
-        ByteBuffer[] bbs = new ByteBuffer[strs.length];
-        for (int i = 0; i < strs.length; i++)
-            bbs[i] = ByteBufferUtil.bytes(strs[i]);
-        return cellname(bbs);
-    }
-
-    public static CellName cellname(int i)
-    {
-        return CellNames.simpleDense(ByteBufferUtil.bytes(i));
-    }
-
-    public static CellName cellname(long l)
-    {
-        return CellNames.simpleDense(ByteBufferUtil.bytes(l));
-    }
-
-    public static Cell column(String name, String value, long timestamp)
-    {
-        return new BufferCell(cellname(name), ByteBufferUtil.bytes(value), timestamp);
-    }
-
-    public static Cell column(String name, long value, long timestamp)
-    {
-        return new BufferCell(cellname(name), ByteBufferUtil.bytes(value), timestamp);
-    }
-
-    public static Cell column(String clusterKey, String name, long value, long timestamp)
-    {
-        return new BufferCell(cellname(clusterKey, name), ByteBufferUtil.bytes(value), timestamp);
-    }
-
-    public static Cell expiringColumn(String name, String value, long timestamp, int ttl)
-    {
-        return new BufferExpiringCell(cellname(name), ByteBufferUtil.bytes(value), timestamp, ttl);
+        return comparator.make(o).clustering();
     }
 
     public static Token token(String key)
@@ -151,27 +113,28 @@ public class Util
         return StorageService.getPartitioner().getToken(ByteBufferUtil.bytes(key));
     }
 
-    public static Range<RowPosition> range(String left, String right)
+    public static Range<PartitionPosition> range(String left, String right)
     {
-        return new Range<RowPosition>(rp(left), rp(right));
+        return new Range<>(rp(left), rp(right));
     }
 
-    public static Range<RowPosition> range(IPartitioner p, String left, String right)
+    public static Range<PartitionPosition> range(IPartitioner p, String left, String right)
     {
-        return new Range<RowPosition>(rp(left, p), rp(right, p));
+        return new Range<>(rp(left, p), rp(right, p));
     }
 
-    public static Bounds<RowPosition> bounds(String left, String right)
+    //Test helper to make an iterator iterable once
+    public static <T> Iterable<T> once(final Iterator<T> source)
     {
-        return new Bounds<RowPosition>(rp(left), rp(right));
-    }
-
-    public static void addMutation(Mutation rm, String columnFamilyName, String superColumnName, long columnName, String value, long timestamp)
-    {
-        CellName cname = superColumnName == null
-                       ? CellNames.simpleDense(getBytes(columnName))
-                       : CellNames.compositeDense(ByteBufferUtil.bytes(superColumnName), getBytes(columnName));
-        rm.add(columnFamilyName, cname, ByteBufferUtil.bytes(value), timestamp);
+        return new Iterable<T>()
+        {
+            private AtomicBoolean exhausted = new AtomicBoolean();
+            public Iterator<T> iterator()
+            {
+                Preconditions.checkState(!exhausted.getAndSet(true));
+                return source;
+            }
+        };
     }
 
     public static ByteBuffer getBytes(long v)
@@ -192,39 +155,6 @@ public class Util
         return bb;
     }
 
-    public static ByteBuffer getBytes(short v)
-    {
-        byte[] bytes = new byte[2];
-        ByteBuffer bb = ByteBuffer.wrap(bytes);
-        bb.putShort(v);
-        bb.rewind();
-        return bb;
-    }
-
-    public static ByteBuffer getBytes(byte v)
-    {
-        byte[] bytes = new byte[1];
-        ByteBuffer bb = ByteBuffer.wrap(bytes);
-        bb.put(v);
-        bb.rewind();
-        return bb;
-    }
-
-    public static List<Row> getRangeSlice(ColumnFamilyStore cfs)
-    {
-        return getRangeSlice(cfs, null);
-    }
-
-    public static List<Row> getRangeSlice(ColumnFamilyStore cfs, ByteBuffer superColumn)
-    {
-        IDiskAtomFilter filter = superColumn == null
-                               ? new IdentityQueryFilter()
-                               : new SliceQueryFilter(SuperColumns.startOf(superColumn), SuperColumns.endOf(superColumn), false, Integer.MAX_VALUE);
-
-        Token min = StorageService.getPartitioner().getMinimumToken();
-        return cfs.getRangeSlice(Bounds.makeRowBounds(min, min), null, filter, 10000);
-    }
-
     /**
      * Writes out a bunch of mutations for a single column family.
      *
@@ -245,23 +175,11 @@ public class Util
         return store;
     }
 
-    public static ColumnFamily getColumnFamily(Keyspace keyspace, DecoratedKey key, String cfName)
-    {
-        ColumnFamilyStore cfStore = keyspace.getColumnFamilyStore(cfName);
-        assert cfStore != null : "Table " + cfName + " has not been defined";
-        return cfStore.getColumnFamily(QueryFilter.getIdentityFilter(key, cfName, System.currentTimeMillis()));
-    }
-
     public static boolean equalsCounterId(CounterId n, ByteBuffer context, int offset)
     {
         return CounterId.wrap(context, context.position() + offset).equals(n);
     }
 
-    public static ColumnFamily cloneAndRemoveDeleted(ColumnFamily cf, int gcBefore)
-    {
-        return ColumnFamilyStore.removeDeleted(cf.cloneMe(), gcBefore);
-    }
-
     /**
      * Creates initial set of nodes and tokens. Nodes are added to StorageService as 'normal'
      */
@@ -306,7 +224,7 @@ public class Util
 
     public static void compact(ColumnFamilyStore cfs, Collection<SSTableReader> sstables)
     {
-        int gcBefore = cfs.gcBefore(System.currentTimeMillis());
+        int gcBefore = cfs.gcBefore(FBUtilities.nowInSeconds());
         AbstractCompactionTask task = cfs.getCompactionStrategyManager().getUserDefinedTask(sstables, gcBefore);
         task.execute(null);
     }
@@ -333,55 +251,235 @@ public class Util
         assert thrown : exception.getName() + " not received";
     }
 
-    public static QueryFilter namesQueryFilter(ColumnFamilyStore cfs, DecoratedKey key)
+    public static AbstractReadCommandBuilder.SinglePartitionBuilder cmd(ColumnFamilyStore cfs, Object... partitionKey)
     {
-        SortedSet<CellName> s = new TreeSet<CellName>(cfs.getComparator());
-        return QueryFilter.getNamesFilter(key, cfs.name, s, System.currentTimeMillis());
+        return new AbstractReadCommandBuilder.SinglePartitionBuilder(cfs, makeKey(cfs.metadata, partitionKey));
     }
 
-    public static QueryFilter namesQueryFilter(ColumnFamilyStore cfs, DecoratedKey key, String... names)
+    public static AbstractReadCommandBuilder.PartitionRangeBuilder cmd(ColumnFamilyStore cfs)
     {
-        SortedSet<CellName> s = new TreeSet<CellName>(cfs.getComparator());
-        for (String str : names)
-            s.add(cellname(str));
-        return QueryFilter.getNamesFilter(key, cfs.name, s, System.currentTimeMillis());
+        return new AbstractReadCommandBuilder.PartitionRangeBuilder(cfs);
     }
 
-    public static QueryFilter namesQueryFilter(ColumnFamilyStore cfs, DecoratedKey key, CellName... names)
+    static DecoratedKey makeKey(CFMetaData metadata, Object... partitionKey)
     {
-        SortedSet<CellName> s = new TreeSet<CellName>(cfs.getComparator());
-        for (CellName n : names)
-            s.add(n);
-        return QueryFilter.getNamesFilter(key, cfs.name, s, System.currentTimeMillis());
+        if (partitionKey.length == 1 && partitionKey[0] instanceof DecoratedKey)
+            return (DecoratedKey)partitionKey[0];
+
+        ByteBuffer key = CFMetaData.serializePartitionKey(metadata.getKeyValidatorAsClusteringComparator().make(partitionKey));
+        return StorageService.getPartitioner().decorateKey(key);
     }
 
-    public static NamesQueryFilter namesFilter(ColumnFamilyStore cfs, String... names)
+    public static void assertEmptyUnfiltered(ReadCommand command)
     {
-        SortedSet<CellName> s = new TreeSet<CellName>(cfs.getComparator());
-        for (String str : names)
-            s.add(cellname(str));
-        return new NamesQueryFilter(s);
+        try (ReadOrderGroup orderGroup = command.startOrderGroup(); UnfilteredPartitionIterator iterator = command.executeLocally(orderGroup))
+        {
+            if (iterator.hasNext())
+            {
+                try (UnfilteredRowIterator partition = iterator.next())
+                {
+                    throw new AssertionError("Expected no results for query " + command.toCQLString() + " but got key " + command.metadata().getKeyValidator().getString(partition.partitionKey().getKey()));
+                }
+            }
+        }
     }
 
-    public static String string(ByteBuffer bb)
+    public static void assertEmpty(ReadCommand command)
     {
-        try
+        try (ReadOrderGroup orderGroup = command.startOrderGroup(); PartitionIterator iterator = command.executeInternal(orderGroup))
         {
-            return ByteBufferUtil.string(bb);
+            if (iterator.hasNext())
+            {
+                try (RowIterator partition = iterator.next())
+                {
+                    throw new AssertionError("Expected no results for query " + command.toCQLString() + " but got key " + command.metadata().getKeyValidator().getString(partition.partitionKey().getKey()));
+                }
+            }
         }
-        catch (Exception e)
+    }
+
+    public static List<ArrayBackedPartition> getAllUnfiltered(ReadCommand command)
+    {
+        List<ArrayBackedPartition> results = new ArrayList<>();
+        try (ReadOrderGroup orderGroup = command.startOrderGroup(); UnfilteredPartitionIterator iterator = command.executeLocally(orderGroup))
         {
-            throw new RuntimeException(e);
+            while (iterator.hasNext())
+            {
+                try (UnfilteredRowIterator partition = iterator.next())
+                {
+                    results.add(ArrayBackedPartition.create(partition));
+                }
+            }
         }
+        return results;
     }
 
-    public static RangeTombstone tombstone(String start, String finish, long timestamp, int localtime)
+    public static List<FilteredPartition> getAll(ReadCommand command)
     {
-        Composite startName = CellNames.simpleDense(ByteBufferUtil.bytes(start));
-        Composite endName = CellNames.simpleDense(ByteBufferUtil.bytes(finish));
-        return new RangeTombstone(startName, endName, timestamp , localtime);
+        List<FilteredPartition> results = new ArrayList<>();
+        try (ReadOrderGroup orderGroup = command.startOrderGroup(); PartitionIterator iterator = command.executeInternal(orderGroup))
+        {
+            while (iterator.hasNext())
+            {
+                try (RowIterator partition = iterator.next())
+                {
+                    results.add(FilteredPartition.create(partition));
+                }
+            }
+        }
+        return results;
     }
 
+    public static Row getOnlyRowUnfiltered(ReadCommand cmd)
+    {
+        try (ReadOrderGroup orderGroup = cmd.startOrderGroup(); UnfilteredPartitionIterator iterator = cmd.executeLocally(orderGroup))
+        {
+            assert iterator.hasNext() : "Expecting one row in one partition but got nothing";
+            try (UnfilteredRowIterator partition = iterator.next())
+            {
+                assert !iterator.hasNext() : "Expecting a single partition but got more";
+
+                assert partition.hasNext() : "Expecting one row in one partition but got an empty partition";
+                Row row = ((Row)partition.next()).takeAlias();
+                assert !partition.hasNext() : "Expecting a single row but got more";
+                return row;
+            }
+        }
+    }
+
+    public static Row getOnlyRow(ReadCommand cmd)
+    {
+        try (ReadOrderGroup orderGroup = cmd.startOrderGroup(); PartitionIterator iterator = cmd.executeInternal(orderGroup))
+        {
+            assert iterator.hasNext() : "Expecting one row in one partition but got nothing";
+            try (RowIterator partition = iterator.next())
+            {
+                assert !iterator.hasNext() : "Expecting a single partition but got more";
+                assert partition.hasNext() : "Expecting one row in one partition but got an empty partition";
+                Row row = partition.next().takeAlias();
+                assert !partition.hasNext() : "Expecting a single row but got more";
+                return row;
+            }
+        }
+    }
+
+    public static ArrayBackedPartition getOnlyPartitionUnfiltered(ReadCommand cmd)
+    {
+        try (ReadOrderGroup orderGroup = cmd.startOrderGroup(); UnfilteredPartitionIterator iterator = cmd.executeLocally(orderGroup))
+        {
+            assert iterator.hasNext() : "Expecting a single partition but got nothing";
+            try (UnfilteredRowIterator partition = iterator.next())
+            {
+                assert !iterator.hasNext() : "Expecting a single partition but got more";
+                return ArrayBackedPartition.create(partition);
+            }
+        }
+    }
+
+    public static FilteredPartition getOnlyPartition(ReadCommand cmd)
+    {
+        try (ReadOrderGroup orderGroup = cmd.startOrderGroup(); PartitionIterator iterator = cmd.executeInternal(orderGroup))
+        {
+            assert iterator.hasNext() : "Expecting a single partition but got nothing";
+            try (RowIterator partition = iterator.next())
+            {
+                assert !iterator.hasNext() : "Expecting a single partition but got more";
+                return FilteredPartition.create(partition);
+            }
+        }
+    }
+
+    public static UnfilteredRowIterator apply(Mutation mutation)
+    {
+        mutation.apply();
+        assert mutation.getPartitionUpdates().size() == 1;
+        return mutation.getPartitionUpdates().iterator().next().unfilteredIterator();
+    }
+
+    public static Cell cell(ColumnFamilyStore cfs, Row row, String columnName)
+    {
+        ColumnDefinition def = cfs.metadata.getColumnDefinition(ByteBufferUtil.bytes(columnName));
+        assert def != null;
+        return row.getCell(def);
+    }
+
+    public static Row row(Partition partition, Object... clustering)
+    {
+        return partition.getRow(partition.metadata().comparator.make(clustering));
+    }
+
+    public static void assertCellValue(Object value, ColumnFamilyStore cfs, Row row, String columnName)
+    {
+        Cell cell = cell(cfs, row, columnName);
+        assert cell != null : "Row " + row.toString(cfs.metadata) + " has no cell for " + columnName;
+        assertEquals(value, cell.column().type.compose(cell.value()));
+    }
+
+    public static void consume(UnfilteredRowIterator iter)
+    {
+        try (UnfilteredRowIterator iterator = iter)
+        {
+            while (iter.hasNext())
+                iter.next();
+        }
+    }
+
+    public static int size(PartitionIterator iter)
+    {
+        int size = 0;
+        while (iter.hasNext())
+        {
+            ++size;
+            iter.next().close();
+        }
+        return size;
+    }
+
+    public static CBuilder getCBuilderForCFM(CFMetaData cfm)
+    {
+        List<ColumnDefinition> clusteringColumns = cfm.clusteringColumns();
+        List<AbstractType<?>> types = new ArrayList<>(clusteringColumns.size());
+        for (ColumnDefinition def : clusteringColumns)
+            types.add(def.type);
+        return CBuilder.create(new ClusteringComparator(types));
+    }
+
+    // moved & refactored from KeyspaceTest in < 3.0
+    public static void assertColumns(Row row, String... expectedColumnNames)
+    {
+        Iterator<Cell> cells = row == null ? Iterators.<Cell>emptyIterator() : row.iterator();
+        String[] actual = Iterators.toArray(Iterators.transform(cells, new Function<Cell, String>()
+        {
+            public String apply(Cell cell)
+            {
+                return cell.column().name.toString();
+            }
+        }), String.class);
+
+        assert Arrays.equals(actual, expectedColumnNames)
+        : String.format("Columns [%s])] is not expected [%s]",
+                        ((row == null) ? "" : row.columns().toString()),
+                        StringUtils.join(expectedColumnNames, ","));
+    }
+
+    public static void assertColumn(CFMetaData cfm, Row row, String name, String value, long timestamp)
+    {
+        Cell cell = row.getCell(cfm.getColumnDefinition(new ColumnIdentifier(name, true)));
+        assertColumn(cell, value, timestamp);
+    }
+
+    public static void assertColumn(Cell cell, String value, long timestamp)
+    {
+        assertNotNull(cell);
+        assertEquals(0, ByteBufferUtil.compareUnsigned(cell.value(), ByteBufferUtil.bytes(value)));
+        assertEquals(timestamp, cell.livenessInfo().timestamp());
+    }
+
+    public static void assertClustering(CFMetaData cfm, Row row, Object... clusteringValue)
+    {
+        assertEquals(row.clustering().size(), clusteringValue.length);
+        assertEquals(0, cfm.comparator.compare(row.clustering(), cfm.comparator.make(clusteringValue)));
+    }
 
     public static void spinAssertEquals(Object expected, Supplier<Object> s, int timeoutInSeconds)
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a991b648/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 c1869b9..e7a1706 100644
--- a/test/unit/org/apache/cassandra/cache/AutoSavingCacheTest.java
+++ b/test/unit/org/apache/cassandra/cache/AutoSavingCacheTest.java
@@ -17,6 +17,11 @@
  */
 package org.apache.cassandra.cache;
 
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.marshal.AsciiType;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.junit.Assert;
 import org.junit.BeforeClass;
@@ -25,10 +30,6 @@ 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.locator.SimpleStrategy;
 import org.apache.cassandra.service.CacheService;
@@ -44,9 +45,12 @@ public class AutoSavingCacheTest
     {
         SchemaLoader.prepareServer();
         SchemaLoader.createKeyspace(KEYSPACE1,
-                                    SimpleStrategy.class,
-                                    KSMetaData.optsWithRF(1),
-                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1));
+                SimpleStrategy.class,
+                KSMetaData.optsWithRF(1),
+                CFMetaData.Builder.create(KEYSPACE1, CF_STANDARD1)
+                    .addPartitionKey("pKey", AsciiType.instance)
+                    .addRegularColumn("col1", AsciiType.instance)
+                    .build());
     }
 
     @Test
@@ -55,9 +59,10 @@ public class AutoSavingCacheTest
         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(CF_STANDARD1, Util.cellname("c1"), ByteBufferUtil.bytes(i), 0);
-            rm.applyUnsafe();
+            ColumnDefinition colDef = new ColumnDefinition(cfs.metadata, ByteBufferUtil.bytes("col1"), AsciiType.instance, 0, ColumnDefinition.Kind.REGULAR);
+            RowUpdateBuilder rowBuilder = new RowUpdateBuilder(cfs.metadata, System.currentTimeMillis(), "key1");
+            rowBuilder.add(colDef, "val1");
+            rowBuilder.build().apply();
             cfs.forceBlockingFlush();
         }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a991b648/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 fe26616..d92d427 100644
--- a/test/unit/org/apache/cassandra/cache/CacheProviderTest.java
+++ b/test/unit/org/apache/cassandra/cache/CacheProviderTest.java
@@ -22,24 +22,27 @@ package org.apache.cassandra.cache;
 
 
 import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.UUID;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.*;
 
+import org.apache.cassandra.Util;
 import org.junit.BeforeClass;
 import org.junit.Test;
+import static org.junit.Assert.*;
 
-import org.apache.cassandra.SchemaLoader;
+import com.googlecode.concurrentlinkedhashmap.Weighers;
+
+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.*;
+import org.apache.cassandra.db.rows.*;
+import org.apache.cassandra.db.marshal.AsciiType;
+import org.apache.cassandra.db.partitions.*;
 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.*;
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.utils.FBUtilities;
 
 public class CacheProviderTest
 {
@@ -52,59 +55,86 @@ public class CacheProviderTest
     private static final String KEYSPACE1 = "CacheProviderTest1";
     private static final String CF_STANDARD1 = "Standard1";
 
+    private static CFMetaData cfm;
+
     @BeforeClass
     public static void defineSchema() throws ConfigurationException
     {
         SchemaLoader.prepareServer();
+
+        cfm = CFMetaData.Builder.create(KEYSPACE1, CF_STANDARD1)
+                                        .addPartitionKey("pKey", AsciiType.instance)
+                                        .addRegularColumn("col1", AsciiType.instance)
+                                        .build();
         SchemaLoader.createKeyspace(KEYSPACE1,
                                     SimpleStrategy.class,
                                     KSMetaData.optsWithRF(1),
-                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1));
+                                    cfm);
+    }
+
+    private ArrayBackedCachedPartition createPartition()
+    {
+        PartitionUpdate update = new RowUpdateBuilder(cfm, System.currentTimeMillis(), "key1")
+                                 .add("col1", "val1")
+                                 .buildUpdate();
+
+        return ArrayBackedCachedPartition.create(update.unfilteredIterator(), FBUtilities.nowInSeconds());
     }
 
-    private void simpleCase(ColumnFamily cf, ICache<MeasureableString, IRowCacheEntry> cache)
+    private void simpleCase(ArrayBackedCachedPartition partition, ICache<MeasureableString, IRowCacheEntry> cache)
     {
-        cache.put(key1, cf);
+        cache.put(key1, partition);
         assertNotNull(cache.get(key1));
 
-        assertDigests(cache.get(key1), cf);
-        cache.put(key2, cf);
-        cache.put(key3, cf);
-        cache.put(key4, cf);
-        cache.put(key5, cf);
+        assertDigests(cache.get(key1), partition);
+        cache.put(key2, partition);
+        cache.put(key3, partition);
+        cache.put(key4, partition);
+        cache.put(key5, partition);
 
         assertEquals(CAPACITY, cache.size());
     }
 
-    private void assertDigests(IRowCacheEntry one, ColumnFamily two)
+    private void assertDigests(IRowCacheEntry one, ArrayBackedCachedPartition two)
     {
-        // CF does not implement .equals
-        assertTrue(one instanceof ColumnFamily);
-        assertEquals(ColumnFamily.digest((ColumnFamily)one), ColumnFamily.digest(two));
+        assertTrue(one instanceof ArrayBackedCachedPartition);
+        try
+        {
+            MessageDigest d1 = MessageDigest.getInstance("MD5");
+            MessageDigest d2 = MessageDigest.getInstance("MD5");
+            UnfilteredRowIterators.digest(((ArrayBackedCachedPartition) one).unfilteredIterator(), d1);
+            UnfilteredRowIterators.digest(((ArrayBackedCachedPartition) two).unfilteredIterator(), d2);
+            assertTrue(MessageDigest.isEqual(d1.digest(), d2.digest()));
+        }
+        catch (NoSuchAlgorithmException e)
+        {
+            throw new RuntimeException(e);
+        }
     }
 
-    // TODO this isn't terribly useful
-    private void concurrentCase(final ColumnFamily cf, final ICache<MeasureableString, IRowCacheEntry> cache) throws InterruptedException
+    private void concurrentCase(final ArrayBackedCachedPartition partition, final ICache<MeasureableString, IRowCacheEntry> cache) throws InterruptedException
     {
-        Runnable runable = new Runnable()
+        final long startTime = System.currentTimeMillis() + 500;
+        Runnable runnable = new Runnable()
         {
             public void run()
             {
-                for (int j = 0; j < 10; j++)
+                while (System.currentTimeMillis() < startTime) {}
+                for (int j = 0; j < 1000; j++)
                 {
-                    cache.put(key1, cf);
-                    cache.put(key2, cf);
-                    cache.put(key3, cf);
-                    cache.put(key4, cf);
-                    cache.put(key5, cf);
+                    cache.put(key1, partition);
+                    cache.put(key2, partition);
+                    cache.put(key3, partition);
+                    cache.put(key4, partition);
+                    cache.put(key5, partition);
                 }
             }
         };
 
-        List<Thread> threads = new ArrayList<Thread>(100);
+        List<Thread> threads = new ArrayList<>(100);
         for (int i = 0; i < 100; i++)
         {
-            Thread thread = new Thread(runable);
+            Thread thread = new Thread(runnable);
             threads.add(thread);
             thread.start();
         }
@@ -112,28 +142,21 @@ public class CacheProviderTest
             thread.join();
     }
 
-    private ColumnFamily createCF()
-    {
-        ColumnFamily cf = ArrayBackedSortedColumns.factory.create(KEYSPACE1, CF_STANDARD1);
-        cf.addColumn(column("vijay", "great", 1));
-        cf.addColumn(column("awesome", "vijay", 1));
-        return cf;
-    }
-
     @Test
     public void testSerializingCache() throws InterruptedException
     {
         ICache<MeasureableString, IRowCacheEntry> cache = SerializingCache.create(CAPACITY, Weighers.<RefCountedMemory>singleton(), new SerializingCacheProvider.RowCacheSerializer());
-        ColumnFamily cf = createCF();
-        simpleCase(cf, cache);
-        concurrentCase(cf, cache);
+        ArrayBackedCachedPartition partition = createPartition();
+        simpleCase(partition, cache);
+        concurrentCase(partition, cache);
     }
-    
+
     @Test
     public void testKeys()
     {
         UUID cfId = UUID.randomUUID();
 
+
         byte[] b1 = {1, 2, 3, 4};
         RowCacheKey key1 = new RowCacheKey(cfId, ByteBuffer.wrap(b1));
         byte[] b2 = {1, 2, 3, 4};

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a991b648/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
new file mode 100644
index 0000000..4a69a7c
--- /dev/null
+++ b/test/unit/org/apache/cassandra/config/CFMetaDataTest.java
@@ -0,0 +1,155 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.cassandra.config;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.HashMap;
+import java.util.HashSet;
+
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.marshal.AsciiType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.db.rows.UnfilteredRowIterators;
+import org.apache.cassandra.db.partitions.PartitionUpdate;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.io.compress.*;
+import org.apache.cassandra.locator.SimpleStrategy;
+import org.apache.cassandra.schema.LegacySchemaTables;
+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.thrift.ThriftConversion;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+public class CFMetaDataTest
+{
+    private static final String KEYSPACE1 = "CFMetaDataTest1";
+    private static final String CF_STANDARD1 = "Standard1";
+
+    private static List<ColumnDef> columnDefs = new ArrayList<ColumnDef>();
+
+    static
+    {
+        columnDefs.add(new ColumnDef(ByteBufferUtil.bytes("col1"), AsciiType.class.getCanonicalName())
+                                    .setIndex_name("col1Index")
+                                    .setIndex_type(IndexType.KEYS));
+
+        columnDefs.add(new ColumnDef(ByteBufferUtil.bytes("col2"), UTF8Type.class.getCanonicalName())
+                                    .setIndex_name("col2Index")
+                                    .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(KEYSPACE1)
+                                 .setName(CF_STANDARD1);
+
+        // convert Thrift to CFMetaData
+        CFMetaData cfMetaData = ThriftConversion.fromThrift(cfDef);
+
+        CfDef thriftCfDef = new CfDef();
+        thriftCfDef.keyspace = KEYSPACE1;
+        thriftCfDef.name = CF_STANDARD1;
+        thriftCfDef.default_validation_class = cfDef.default_validation_class;
+        thriftCfDef.comment = cfDef.comment;
+        thriftCfDef.column_metadata = new ArrayList<>();
+        for (ColumnDef columnDef : columnDefs)
+        {
+            ColumnDef c = new ColumnDef();
+            c.name = ByteBufferUtil.clone(columnDef.name);
+            c.validation_class = columnDef.getValidation_class();
+            c.index_name = columnDef.getIndex_name();
+            c.index_type = IndexType.KEYS;
+            thriftCfDef.column_metadata.add(c);
+        }
+
+        CfDef converted = ThriftConversion.toThrift(cfMetaData);
+
+        assertEquals(thriftCfDef.keyspace, converted.keyspace);
+        assertEquals(thriftCfDef.name, converted.name);
+        assertEquals(thriftCfDef.default_validation_class, converted.default_validation_class);
+        assertEquals(thriftCfDef.comment, converted.comment);
+        assertEquals(new HashSet<>(thriftCfDef.column_metadata), new HashSet<>(converted.column_metadata));
+    }
+
+    @Test
+    public void testConversionsInverses() throws Exception
+    {
+        for (String keyspaceName : Schema.instance.getNonSystemKeyspaces())
+        {
+            for (ColumnFamilyStore cfs : Keyspace.open(keyspaceName).getColumnFamilyStores())
+            {
+                CFMetaData cfm = cfs.metadata;
+                if (!cfm.isThriftCompatible())
+                    continue;
+
+                checkInverses(cfm);
+
+                // Testing with compression to catch #3558
+                CFMetaData withCompression = cfm.copy();
+                withCompression.compressionParameters(new CompressionParameters(SnappyCompressor.instance, 32768, new HashMap<String, String>()));
+                checkInverses(withCompression);
+            }
+        }
+    }
+
+    private void checkInverses(CFMetaData cfm) throws Exception
+    {
+        DecoratedKey k = StorageService.getPartitioner().decorateKey(ByteBufferUtil.bytes(cfm.ksName));
+        KSMetaData keyspace = Schema.instance.getKSMetaData(cfm.ksName);
+
+        // Test thrift conversion
+        CFMetaData before = cfm;
+        CFMetaData after = ThriftConversion.fromThriftForUpdate(ThriftConversion.toThrift(before), before);
+        assert before.equals(after) : String.format("%n%s%n!=%n%s", before, after);
+
+        // Test schema conversion
+        Mutation rm = LegacySchemaTables.makeCreateTableMutation(keyspace, cfm, FBUtilities.timestampMicros());
+        PartitionUpdate cfU = rm.getPartitionUpdate(Schema.instance.getId(SystemKeyspace.NAME, LegacySchemaTables.COLUMNFAMILIES));
+        PartitionUpdate cdU = rm.getPartitionUpdate(Schema.instance.getId(SystemKeyspace.NAME, LegacySchemaTables.COLUMNS));
+        CFMetaData newCfm = LegacySchemaTables.createTableFromTablePartitionAndColumnsPartition(
+                UnfilteredRowIterators.filter(cfU.unfilteredIterator(), FBUtilities.nowInSeconds()),
+                UnfilteredRowIterators.filter(cdU.unfilteredIterator(), FBUtilities.nowInSeconds())
+        );
+        assert cfm.equals(newCfm) : String.format("%n%s%n!=%n%s", cfm, newCfm);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a991b648/test/unit/org/apache/cassandra/config/ColumnDefinitionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/config/ColumnDefinitionTest.java b/test/unit/org/apache/cassandra/config/ColumnDefinitionTest.java
index 2bee0c3..0e5e192 100644
--- a/test/unit/org/apache/cassandra/config/ColumnDefinitionTest.java
+++ b/test/unit/org/apache/cassandra/config/ColumnDefinitionTest.java
@@ -23,7 +23,6 @@ package org.apache.cassandra.config;
 import org.junit.Assert;
 import org.junit.Test;
 
-import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.marshal.*;
 import org.apache.cassandra.thrift.ThriftConversion;
 import org.apache.cassandra.utils.ByteBufferUtil;
@@ -33,12 +32,16 @@ public class ColumnDefinitionTest
     @Test
     public void testSerializeDeserialize() throws Exception
     {
-        CFMetaData cfm = CFMetaData.denseCFMetaData("ks", "cf", UTF8Type.instance);
+        CFMetaData cfm = CFMetaData.Builder.create("ks", "cf", true, false, false)
+                         .addPartitionKey("pkey", AsciiType.instance)
+                         .addClusteringColumn("name", AsciiType.instance)
+                         .addRegularColumn("val", AsciiType.instance)
+                         .build();
 
-        ColumnDefinition cd0 = ColumnDefinition.regularDef(cfm, ByteBufferUtil.bytes("TestColumnDefinitionName0"), BytesType.instance, null)
+        ColumnDefinition cd0 = ColumnDefinition.staticDef(cfm, ByteBufferUtil.bytes("TestColumnDefinitionName0"), BytesType.instance, null)
                                                .setIndex("random index name 0", IndexType.KEYS, null);
 
-        ColumnDefinition cd1 = ColumnDefinition.regularDef(cfm, ByteBufferUtil.bytes("TestColumnDefinition1"), LongType.instance, null);
+        ColumnDefinition cd1 = ColumnDefinition.staticDef(cfm, ByteBufferUtil.bytes("TestColumnDefinition1"), LongType.instance, null);
 
         testSerializeDeserialize(cfm, cd0);
         testSerializeDeserialize(cfm, cd1);
@@ -46,7 +49,7 @@ public class ColumnDefinitionTest
 
     protected void testSerializeDeserialize(CFMetaData cfm, ColumnDefinition cd) throws Exception
     {
-        ColumnDefinition newCd = ThriftConversion.fromThrift(cfm.ksName, cfm.cfName, cfm.comparator.asAbstractType(), null, ThriftConversion.toThrift(cd));
+        ColumnDefinition newCd = ThriftConversion.fromThrift(cfm.ksName, cfm.cfName, cfm.comparator.subtype(0), null, ThriftConversion.toThrift(cd));
         Assert.assertNotSame(cd, newCd);
         Assert.assertEquals(cd.hashCode(), newCd.hashCode());
         Assert.assertEquals(cd, newCd);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a991b648/test/unit/org/apache/cassandra/config/LegacySchemaTablesTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/config/LegacySchemaTablesTest.java b/test/unit/org/apache/cassandra/config/LegacySchemaTablesTest.java
index 3642e7a..b7a2a37 100644
--- a/test/unit/org/apache/cassandra/config/LegacySchemaTablesTest.java
+++ b/test/unit/org/apache/cassandra/config/LegacySchemaTablesTest.java
@@ -25,6 +25,8 @@ import java.util.HashSet;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.rows.UnfilteredRowIterators;
+import org.apache.cassandra.db.partitions.PartitionUpdate;
 import org.apache.cassandra.db.marshal.AsciiType;
 import org.apache.cassandra.db.marshal.UTF8Type;
 import org.apache.cassandra.exceptions.ConfigurationException;
@@ -142,9 +144,10 @@ public class LegacySchemaTablesTest
 
         // Test schema conversion
         Mutation rm = LegacySchemaTables.makeCreateTableMutation(keyspace, cfm, FBUtilities.timestampMicros());
-        ColumnFamily serializedCf = rm.getColumnFamily(Schema.instance.getId(SystemKeyspace.NAME, LegacySchemaTables.COLUMNFAMILIES));
-        ColumnFamily serializedCD = rm.getColumnFamily(Schema.instance.getId(SystemKeyspace.NAME, LegacySchemaTables.COLUMNS));
-        CFMetaData newCfm = LegacySchemaTables.createTableFromTablePartitionAndColumnsPartition(new Row(k, serializedCf), new Row(k, serializedCD));
+        PartitionUpdate serializedCf = rm.getPartitionUpdate(Schema.instance.getId(SystemKeyspace.NAME, LegacySchemaTables.COLUMNFAMILIES));
+        PartitionUpdate serializedCD = rm.getPartitionUpdate(Schema.instance.getId(SystemKeyspace.NAME, LegacySchemaTables.COLUMNS));
+        CFMetaData newCfm = LegacySchemaTables.createTableFromTablePartitionAndColumnsPartition(UnfilteredRowIterators.filter(serializedCf.unfilteredIterator(), FBUtilities.nowInSeconds()),
+                                                                                                UnfilteredRowIterators.filter(serializedCD.unfilteredIterator(), FBUtilities.nowInSeconds()));
         assert cfm.equals(newCfm) : String.format("%n%s%n!=%n%s", cfm, newCfm);
     }
 }