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 2013/12/17 17:03:40 UTC

[04/13] Push composites support in the storage engine

http://git-wip-us.apache.org/repos/asf/cassandra/blob/362cc053/src/java/org/apache/cassandra/tools/SSTableImport.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/SSTableImport.java b/src/java/org/apache/cassandra/tools/SSTableImport.java
index 41986b2..0a6ef6b 100644
--- a/src/java/org/apache/cassandra/tools/SSTableImport.java
+++ b/src/java/org/apache/cassandra/tools/SSTableImport.java
@@ -37,12 +37,12 @@ import org.apache.commons.cli.PosixParser;
 
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.composites.*;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.Schema;
 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.dht.IPartitioner;
 import org.apache.cassandra.io.sstable.SSTableWriter;
 import org.apache.cassandra.serializers.MarshalException;
@@ -107,12 +107,12 @@ public class SSTableImport
         {
             if (json instanceof List)
             {
-                AbstractType<?> comparator = oldSCFormat ? SuperColumns.getComparatorFor(meta, isSubColumn) : meta.comparator;
+                CellNameType comparator = oldSCFormat ? new SimpleDenseCellNameType(SuperColumns.getComparatorFor(meta, isSubColumn)) : meta.comparator;
                 List fields = (List<?>) json;
 
                 assert fields.size() >= 3 : "Column definition should have at least 3";
 
-                name  = stringAsType((String) fields.get(0), comparator);
+                name  = stringAsType((String) fields.get(0), comparator.asAbstractType());
                 timestamp = (Long) fields.get(2);
                 kind = "";
 
@@ -157,11 +157,11 @@ public class SSTableImport
                 }
                 else if (isRangeTombstone())
                 {
-                    value = comparator.fromString((String)fields.get(1));
+                    value = stringAsType((String) fields.get(1), comparator.asAbstractType());
                 }
                 else
                 {
-                    value = stringAsType((String) fields.get(1), meta.getValueValidator(meta.getColumnDefinitionFromCellName(name)));
+                    value = stringAsType((String) fields.get(1), meta.getValueValidatorForFullCellName(name));
                 }
             }
         }
@@ -239,7 +239,21 @@ public class SSTableImport
         for (Object c : row)
         {
             JsonColumn col = new JsonColumn<List>((List) c, cfm, oldSCFormat, (superName != null));
-            ByteBuffer cname = superName == null ? col.getName() : CompositeType.build(superName, col.getName());
+            if (col.isRangeTombstone())
+            {
+                Composite start = superName == null
+                                ? cfm.comparator.fromByteBuffer(col.getName())
+                                : cfm.comparator.make(superName, col.getName());
+                Composite end = superName == null
+                              ? cfm.comparator.fromByteBuffer(col.getValue())
+                              : cfm.comparator.make(superName, col.getValue());
+                cfamily.addAtom(new RangeTombstone(start, end, col.timestamp, col.localExpirationTime));
+                continue;
+            }
+
+            CellName cname = superName == null
+                           ? cfm.comparator.cellFromByteBuffer(col.getName())
+                           : cfm.comparator.makeCellName(superName, col.getName());
 
             if (col.isExpiring())
             {
@@ -255,13 +269,15 @@ public class SSTableImport
             }
             else if (col.isRangeTombstone())
             {
-                ByteBuffer end = superName == null ? col.getValue() : CompositeType.build(superName, col.getValue());
+                CellName end = superName == null
+                             ? cfm.comparator.cellFromByteBuffer(col.getValue())
+                             : cfm.comparator.makeCellName(superName, col.getValue());
                 cfamily.addAtom(new RangeTombstone(cname, end, col.timestamp, col.localExpirationTime));
             }
             // cql3 row marker, see CASSANDRA-5852
-            else if (!cname.hasRemaining())
+            else if (cname.isEmpty())
             {
-                cfamily.addColumn(ByteBuffer.wrap(new byte[3]), col.getValue(), col.timestamp);
+                cfamily.addColumn(cfm.comparator.rowMarker(Composites.EMPTY), col.getValue(), col.timestamp);
             }
             else
             {
@@ -298,14 +314,14 @@ public class SSTableImport
         CFMetaData metaData = cfamily.metadata();
         assert metaData != null;
 
-        AbstractType<?> comparator = metaData.comparator;
+        CellNameType comparator = metaData.comparator;
 
         // Super columns
         for (Map.Entry<?, ?> entry : row.entrySet())
         {
             Map<?, ?> data = (Map<?, ?>) entry.getValue();
 
-            ByteBuffer superName = stringAsType((String) entry.getKey(), ((CompositeType)comparator).types.get(0));
+            ByteBuffer superName = stringAsType((String) entry.getKey(), comparator.subtype(0));
 
             addColumnsToCF((List<?>) data.get("subColumns"), superName, cfamily);
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/362cc053/src/java/org/apache/cassandra/tracing/Tracing.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tracing/Tracing.java b/src/java/org/apache/cassandra/tracing/Tracing.java
index 3c3092e..b1f618d 100644
--- a/src/java/org/apache/cassandra/tracing/Tracing.java
+++ b/src/java/org/apache/cassandra/tracing/Tracing.java
@@ -33,8 +33,8 @@ import org.slf4j.LoggerFactory;
 import org.apache.cassandra.concurrent.Stage;
 import org.apache.cassandra.concurrent.StageManager;
 import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.cql3.ColumnNameBuilder;
 import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.composites.CellName;
 import org.apache.cassandra.db.marshal.TimeUUIDType;
 import org.apache.cassandra.exceptions.OverloadedException;
 import org.apache.cassandra.exceptions.UnavailableException;
@@ -71,27 +71,27 @@ public class Tracing
 
     public static final Tracing instance = new Tracing();
 
-    public static void addColumn(ColumnFamily cf, ByteBuffer name, InetAddress address)
+    public static void addColumn(ColumnFamily cf, CellName name, InetAddress address)
     {
         addColumn(cf, name, ByteBufferUtil.bytes(address));
     }
 
-    public static void addColumn(ColumnFamily cf, ByteBuffer name, int value)
+    public static void addColumn(ColumnFamily cf, CellName name, int value)
     {
         addColumn(cf, name, ByteBufferUtil.bytes(value));
     }
 
-    public static void addColumn(ColumnFamily cf, ByteBuffer name, long value)
+    public static void addColumn(ColumnFamily cf, CellName name, long value)
     {
         addColumn(cf, name, ByteBufferUtil.bytes(value));
     }
 
-    public static void addColumn(ColumnFamily cf, ByteBuffer name, String value)
+    public static void addColumn(ColumnFamily cf, CellName name, String value)
     {
         addColumn(cf, name, ByteBufferUtil.bytes(value));
     }
 
-    private static void addColumn(ColumnFamily cf, ByteBuffer name, ByteBuffer value)
+    private static void addColumn(ColumnFamily cf, CellName name, ByteBuffer value)
     {
         cf.addColumn(new ExpiringColumn(name, value, System.currentTimeMillis(), TTL));
     }
@@ -100,17 +100,14 @@ public class Tracing
     {
         for (Map.Entry<String, String> entry : rawPayload.entrySet())
         {
-            cf.addColumn(new ExpiringColumn(buildName(cf.metadata(), bytes("parameters"), bytes(entry.getKey())),
+            cf.addColumn(new ExpiringColumn(buildName(cf.metadata(), "parameters", entry.getKey()),
                                             bytes(entry.getValue()), System.currentTimeMillis(), TTL));
         }
     }
 
-    public static ByteBuffer buildName(CFMetaData meta, ByteBuffer... args)
+    public static CellName buildName(CFMetaData meta, Object... args)
     {
-        ColumnNameBuilder builder = meta.getColumnNameBuilder();
-        for (ByteBuffer arg : args)
-            builder.add(arg);
-        return builder.build();
+        return meta.comparator.makeCellName(args);
     }
 
     public UUID getSessionId()
@@ -169,7 +166,7 @@ public class Tracing
                 {
                     CFMetaData cfMeta = CFMetaData.TraceSessionsCf;
                     ColumnFamily cf = ArrayBackedSortedColumns.factory.create(cfMeta);
-                    addColumn(cf, buildName(cfMeta, bytes("duration")), elapsed);
+                    addColumn(cf, buildName(cfMeta, "duration"), elapsed);
                     mutateWithCatch(new RowMutation(TRACE_KS, sessionIdBytes, cf));
                 }
             });
@@ -207,10 +204,10 @@ public class Tracing
             {
                 CFMetaData cfMeta = CFMetaData.TraceSessionsCf;
                 ColumnFamily cf = TreeMapBackedSortedColumns.factory.create(cfMeta);
-                addColumn(cf, buildName(cfMeta, bytes("coordinator")), FBUtilities.getBroadcastAddress());
+                addColumn(cf, buildName(cfMeta, "coordinator"), FBUtilities.getBroadcastAddress());
                 addParameterColumns(cf, parameters);
-                addColumn(cf, buildName(cfMeta, bytes("request")), request);
-                addColumn(cf, buildName(cfMeta, bytes("started_at")), started_at);
+                addColumn(cf, buildName(cfMeta, "request"), request);
+                addColumn(cf, buildName(cfMeta, "started_at"), started_at);
                 mutateWithCatch(new RowMutation(TRACE_KS, sessionIdBytes, cf));
             }
         });

http://git-wip-us.apache.org/repos/asf/cassandra/blob/362cc053/src/java/org/apache/cassandra/triggers/TriggerExecutor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/triggers/TriggerExecutor.java b/src/java/org/apache/cassandra/triggers/TriggerExecutor.java
index 4b3c24a..c294128 100644
--- a/src/java/org/apache/cassandra/triggers/TriggerExecutor.java
+++ b/src/java/org/apache/cassandra/triggers/TriggerExecutor.java
@@ -29,6 +29,7 @@ import com.google.common.collect.Maps;
 
 import org.apache.cassandra.config.TriggerDefinition;
 import org.apache.cassandra.cql.QueryProcessor;
+import org.apache.cassandra.db.composites.CellName;
 import org.apache.cassandra.db.ColumnFamily;
 import org.apache.cassandra.db.CounterMutation;
 import org.apache.cassandra.db.IMutation;
@@ -94,7 +95,7 @@ public class TriggerExecutor
         {
             QueryProcessor.validateKey(mutation.key());
             for (ColumnFamily tcf : mutation.getColumnFamilies())
-                for (ByteBuffer tName : tcf.getColumnNames())
+                for (CellName tName : tcf.getColumnNames())
                     QueryProcessor.validateColumn(tcf.metadata(), tName, tcf.getColumn(tName).value());
         }
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/362cc053/src/java/org/apache/cassandra/utils/FBUtilities.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/FBUtilities.java b/src/java/org/apache/cassandra/utils/FBUtilities.java
index 7eb735e..ccdc4f3 100644
--- a/src/java/org/apache/cassandra/utils/FBUtilities.java
+++ b/src/java/org/apache/cassandra/utils/FBUtilities.java
@@ -504,9 +504,11 @@ public class FBUtilities
         }
     }
 
-    public static <T extends Comparable> SortedSet<T> singleton(T column)
+    public static <T> SortedSet<T> singleton(T column, Comparator<? super T> comparator)
     {
-        return new TreeSet<T>(Arrays.asList(column));
+        SortedSet<T> s = new TreeSet<T>(comparator);
+        s.add(column);
+        return s;
     }
 
     public static String toString(Map<?,?> map)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/362cc053/src/java/org/apache/cassandra/utils/ObjectSizes.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/ObjectSizes.java b/src/java/org/apache/cassandra/utils/ObjectSizes.java
index acee919..28ddffd 100644
--- a/src/java/org/apache/cassandra/utils/ObjectSizes.java
+++ b/src/java/org/apache/cassandra/utils/ObjectSizes.java
@@ -98,6 +98,19 @@ public class ObjectSizes
     }
 
     /**
+     * Memory a ByteBuffer array consumes.
+     */
+    public static long getArraySize(ByteBuffer[] array)
+    {
+        long allElementsSize = 0;
+        for (int i = 0; i < array.length; i++)
+            if (array[i] != null)
+                allElementsSize += getSize(array[i]);
+
+        return allElementsSize + getArraySize(array.length, getReferenceSize());
+    }
+
+    /**
      * Memory a byte buffer consumes
      * @param buffer ByteBuffer to calculate in memory size
      * @return Total in-memory size of the byte buffer

http://git-wip-us.apache.org/repos/asf/cassandra/blob/362cc053/test/data/serialization/2.0/db.RowMutation.bin
----------------------------------------------------------------------
diff --git a/test/data/serialization/2.0/db.RowMutation.bin b/test/data/serialization/2.0/db.RowMutation.bin
deleted file mode 100644
index 73d93e8..0000000
Binary files a/test/data/serialization/2.0/db.RowMutation.bin and /dev/null differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/362cc053/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 b1ef690..0dd872c 100644
--- a/test/long/org/apache/cassandra/db/LongKeyspaceTest.java
+++ b/test/long/org/apache/cassandra/db/LongKeyspaceTest.java
@@ -56,10 +56,7 @@ public class LongKeyspaceTest extends SchemaLoader
                 {
                     for (int j = 0; j < i; j++)
                     {
-                        cf = cfStore.getColumnFamily(QueryFilter.getNamesFilter(Util.dk("key" + i),
-                                                                                "Standard1",
-                                                                                ByteBufferUtil.bytes("c" + j),
-                                                                                System.currentTimeMillis()));
+                        cf = cfStore.getColumnFamily(Util.namesQueryFilter(cfStore, Util.dk("key" + i), "c" + j));
                         KeyspaceTest.assertColumns(cf, "c" + j);
                     }
                 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/362cc053/test/long/org/apache/cassandra/db/MeteredFlusherTest.java
----------------------------------------------------------------------
diff --git a/test/long/org/apache/cassandra/db/MeteredFlusherTest.java b/test/long/org/apache/cassandra/db/MeteredFlusherTest.java
index 5e25744..99ae65a 100644
--- a/test/long/org/apache/cassandra/db/MeteredFlusherTest.java
+++ b/test/long/org/apache/cassandra/db/MeteredFlusherTest.java
@@ -27,6 +27,7 @@ import java.nio.ByteBuffer;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.Util;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.db.marshal.UTF8Type;
@@ -41,11 +42,10 @@ public class MeteredFlusherTest extends SchemaLoader
         Keyspace keyspace = Keyspace.open("Keyspace1");
         for (int i = 0; i < 100; i++)
         {
-            CFMetaData metadata = new CFMetaData(keyspace.getName(), "_CF" + i, ColumnFamilyType.Standard, UTF8Type.instance, null);
+            CFMetaData metadata = CFMetaData.denseCFMetaData(keyspace.getName(), "_CF" + i, UTF8Type.instance);
             MigrationManager.announceNewColumnFamily(metadata);
         }
 
-        ByteBuffer name = ByteBufferUtil.bytes("c");
         for (int j = 0; j < 200; j++)
         {
             for (int i = 0; i < 100; i++)
@@ -54,7 +54,7 @@ public class MeteredFlusherTest extends SchemaLoader
                 ColumnFamily cf = TreeMapBackedSortedColumns.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 Column(name, value));
+                cf.addColumn(new Column(Util.cellname("c"), value));
                 rm.add(cf);
                 rm.applyUnsafe();
             }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/362cc053/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 3f65714..b212a22 100644
--- a/test/long/org/apache/cassandra/db/commitlog/ComitLogStress.java
+++ b/test/long/org/apache/cassandra/db/commitlog/ComitLogStress.java
@@ -8,6 +8,7 @@ import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicLong;
 
+import org.apache.cassandra.Util;
 import org.apache.cassandra.concurrent.JMXEnabledThreadPoolExecutor;
 import org.apache.cassandra.concurrent.NamedThreadFactory;
 import org.apache.cassandra.db.RowMutation;
@@ -64,9 +65,9 @@ public class ComitLogStress
             String ks = "Keyspace1";
             ByteBuffer key = ByteBufferUtil.bytes(keyString);
             RowMutation mutation = new RowMutation(ks, key);
-            mutation.add("Standard1", ByteBufferUtil.bytes("name"), ByteBufferUtil.bytes("value"),
+            mutation.add("Standard1", Util.cellname("name"), ByteBufferUtil.bytes("value"),
                     System.currentTimeMillis());
             CommitLog.instance.add(mutation);
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/362cc053/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 21c6457..68a2489 100644
--- a/test/long/org/apache/cassandra/db/compaction/LongCompactionsTest.java
+++ b/test/long/org/apache/cassandra/db/compaction/LongCompactionsTest.java
@@ -129,7 +129,7 @@ public class LongCompactionsTest extends SchemaLoader
                 DecoratedKey key = Util.dk(String.valueOf(i % 2));
                 RowMutation rm = new RowMutation(KEYSPACE1, key.key);
                 long timestamp = j * ROWS_PER_SSTABLE + i;
-                rm.add("Standard1", ByteBufferUtil.bytes(String.valueOf(i / 2)),
+                rm.add("Standard1", Util.cellname(String.valueOf(i / 2)),
                        ByteBufferUtil.EMPTY_BYTE_BUFFER,
                        timestamp);
                 maxTimestampExpected = Math.max(timestamp, maxTimestampExpected);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/362cc053/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 0eb44d0..0de0395 100644
--- a/test/long/org/apache/cassandra/db/compaction/LongLeveledCompactionStrategyTest.java
+++ b/test/long/org/apache/cassandra/db/compaction/LongLeveledCompactionStrategyTest.java
@@ -61,7 +61,7 @@ public class LongLeveledCompactionStrategyTest extends SchemaLoader
             RowMutation rm = new RowMutation(ksname, key.key);
             for (int c = 0; c < columns; c++)
             {
-                rm.add(cfname, ByteBufferUtil.bytes("column" + c), value, 0);
+                rm.add(cfname, Util.cellname("column" + c), value, 0);
             }
             rm.apply();
             store.forceBlockingFlush();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/362cc053/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 22b2424..9bba196 100644
--- a/test/unit/org/apache/cassandra/AbstractSerializationsTester.java
+++ b/test/unit/org/apache/cassandra/AbstractSerializationsTester.java
@@ -21,6 +21,7 @@ package org.apache.cassandra;
 
 import com.google.common.io.ByteArrayDataOutput;
 import com.google.common.io.ByteStreams;
+import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.net.MessagingService;
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/362cc053/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 1c13942..90dc629 100644
--- a/test/unit/org/apache/cassandra/SchemaLoader.java
+++ b/test/unit/org/apache/cassandra/SchemaLoader.java
@@ -145,12 +145,7 @@ public class SchemaLoader
                                            standardCFMD(ks1, "StandardGCGS0").gcGraceSeconds(0),
                                            standardCFMD(ks1, "StandardLong1"),
                                            standardCFMD(ks1, "StandardLong2"),
-                                           new CFMetaData(ks1,
-                                                          "ValuesWithQuotes",
-                                                          st,
-                                                          BytesType.instance,
-                                                          null)
-                                                   .defaultValidator(UTF8Type.instance),
+                                           CFMetaData.denseCFMetaData(ks1, "ValuesWithQuotes", BytesType.instance).defaultValidator(UTF8Type.instance),
                                            superCFMD(ks1, "Super1", LongType.instance),
                                            superCFMD(ks1, "Super2", LongType.instance),
                                            superCFMD(ks1, "Super3", LongType.instance),
@@ -159,44 +154,18 @@ public class SchemaLoader
                                            superCFMD(ks1, "Super6", LexicalUUIDType.instance, UTF8Type.instance),
                                            indexCFMD(ks1, "Indexed1", true),
                                            indexCFMD(ks1, "Indexed2", false),
-                                           new CFMetaData(ks1,
-                                                          "StandardInteger1",
-                                                          st,
-                                                          IntegerType.instance,
-                                                          null),
-                                           new CFMetaData(ks1,
-                                                          "Counter1",
-                                                          st,
-                                                          bytes,
-                                                          null)
-                                                   .defaultValidator(CounterColumnType.instance),
-                                           new CFMetaData(ks1,
-                                                          "SuperCounter1",
-                                                          su,
-                                                          bytes,
-                                                          bytes)
-                                                   .defaultValidator(CounterColumnType.instance),
+                                           CFMetaData.denseCFMetaData(ks1, "StandardInteger1", 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),
-                                           jdbcCFMD(ks1, "JdbcInteger", IntegerType.instance).addColumnDefinition(integerColumn("ks1", "JdbcInteger")),
-                                           jdbcCFMD(ks1, "JdbcUtf8", UTF8Type.instance).addColumnDefinition(utf8Column("ks1", "JdbcUtf8")),
+                                           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),
-                                           new CFMetaData(ks1,
-                                                          "StandardComposite",
-                                                          st,
-                                                          composite,
-                                                          null),
-                                           new CFMetaData(ks1,
-                                                          "StandardComposite2",
-                                                          st,
-                                                          compositeMaxMin,
-                                                          null),
-                                           new CFMetaData(ks1,
-                                                          "StandardDynamicComposite",
-                                                          st,
-                                                          dynamicComposite,
-                                                          null),
+                                           CFMetaData.denseCFMetaData(ks1, "StandardComposite", composite),
+                                           CFMetaData.denseCFMetaData(ks1, "StandardComposite2", compositeMaxMin),
+                                           CFMetaData.denseCFMetaData(ks1, "StandardDynamicComposite", dynamicComposite),
                                            standardCFMD(ks1, "StandardLeveled")
                                                                                .compactionStrategyClass(LeveledCompactionStrategy.class)
                                                                                .compactionStrategyOptions(leveledOptions),
@@ -238,11 +207,7 @@ public class SchemaLoader
                                            standardCFMD(ks4, "Standard3"),
                                            superCFMD(ks4, "Super3", bytes),
                                            superCFMD(ks4, "Super4", TimeUUIDType.instance),
-                                           new CFMetaData(ks4,
-                                                          "Super5",
-                                                          su,
-                                                          TimeUUIDType.instance,
-                                                          bytes)));
+                                           CFMetaData.denseCFMetaData(ks4, "Super5", TimeUUIDType.instance, bytes)));
 
         // Keyspace 5
         schema.add(KSMetaData.testMetadata(ks5,
@@ -335,7 +300,7 @@ public class SchemaLoader
                                                       SecondaryIndex.CUSTOM_INDEX_OPTION_NAME,
                                                       PerRowSecondaryIndexTest.TestIndex.class.getName());
 
-        CFMetaData cfm =  standardCFMD(ksName, cfName).keyValidator(AsciiType.instance);
+        CFMetaData cfm =  CFMetaData.sparseCFMetaData(ksName, cfName, BytesType.instance).keyValidator(AsciiType.instance);
 
         ByteBuffer cName = ByteBufferUtil.bytes("indexed");
         return cfm.addOrReplaceColumnDefinition(ColumnDefinition.regularDef(cfm, cName, AsciiType.instance, null)
@@ -355,7 +320,7 @@ public class SchemaLoader
 
     private static CFMetaData standardCFMD(String ksName, String cfName)
     {
-        return new CFMetaData(ksName, cfName, ColumnFamilyType.Standard, BytesType.instance, null);
+        return CFMetaData.denseCFMetaData(ksName, cfName, BytesType.instance);
     }
     private static CFMetaData superCFMD(String ksName, String cfName, AbstractType subcc)
     {
@@ -363,11 +328,11 @@ public class SchemaLoader
     }
     private static CFMetaData superCFMD(String ksName, String cfName, AbstractType cc, AbstractType subcc)
     {
-        return new CFMetaData(ksName, cfName, ColumnFamilyType.Super, cc, subcc);
+        return CFMetaData.denseCFMetaData(ksName, cfName, cc, subcc);
     }
     private static CFMetaData indexCFMD(String ksName, String cfName, final Boolean withIdxType) throws ConfigurationException
     {
-        CFMetaData cfm = standardCFMD(ksName, cfName).keyValidator(AsciiType.instance);
+        CFMetaData cfm = CFMetaData.sparseCFMetaData(ksName, cfName, BytesType.instance).keyValidator(AsciiType.instance);
 
         ByteBuffer cName = ByteBufferUtil.bytes("birthdate");
         IndexType keys = withIdxType ? IndexType.KEYS : null;
@@ -377,7 +342,7 @@ public class SchemaLoader
     private static CFMetaData compositeIndexCFMD(String ksName, String cfName, final Boolean withIdxType, boolean withOldCfIds) throws ConfigurationException
     {
         final CompositeType composite = CompositeType.getInstance(Arrays.asList(new AbstractType<?>[]{UTF8Type.instance, UTF8Type.instance})); 
-        CFMetaData cfm = new CFMetaData(ksName, cfName, ColumnFamilyType.Standard, composite, null);
+        CFMetaData cfm = CFMetaData.sparseCFMetaData(ksName, cfName, composite);
 
         ByteBuffer cName = ByteBufferUtil.bytes("col1");
         IndexType idxType = withIdxType ? IndexType.COMPOSITES : null;
@@ -387,7 +352,12 @@ public class SchemaLoader
     
     private static CFMetaData jdbcCFMD(String ksName, String cfName, AbstractType comp)
     {
-        return new CFMetaData(ksName, cfName, ColumnFamilyType.Standard, comp, null).defaultValidator(comp);
+        return CFMetaData.denseCFMetaData(ksName, cfName, comp).defaultValidator(comp);
+    }
+
+    private static CFMetaData jdbcSparseCFMD(String ksName, String cfName, AbstractType comp)
+    {
+        return CFMetaData.sparseCFMetaData(ksName, cfName, comp).defaultValidator(comp);
     }
 
     public static void cleanupAndLeaveDirs()
@@ -433,7 +403,7 @@ public class SchemaLoader
         {
             ByteBuffer key = ByteBufferUtil.bytes("key" + i);
             RowMutation rowMutation = new RowMutation(keyspace, key);
-            rowMutation.add(columnFamily, ByteBufferUtil.bytes("col" + i), ByteBufferUtil.bytes("val" + i), System.currentTimeMillis());
+            rowMutation.add(columnFamily, Util.cellname("col" + i), ByteBufferUtil.bytes("val" + i), System.currentTimeMillis());
             rowMutation.applyUnsafe();
         }
     }
@@ -445,7 +415,7 @@ public class SchemaLoader
         for (int i = offset; i < offset + numberOfRows; i++)
         {
             DecoratedKey key = Util.dk("key" + i);
-            store.getColumnFamily(QueryFilter.getNamesFilter(key, columnFamily, ByteBufferUtil.bytes("col" + i), System.currentTimeMillis()));
+            store.getColumnFamily(Util.namesQueryFilter(store, key, "col" + i));
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/362cc053/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 a71dc48..87d19d2 100644
--- a/test/unit/org/apache/cassandra/Util.java
+++ b/test/unit/org/apache/cassandra/Util.java
@@ -25,21 +25,26 @@ import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
+import java.util.SortedSet;
+import java.util.TreeSet;
 import java.util.UUID;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Future;
 
 import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.composites.*;
 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.marshal.CompositeType;
 import org.apache.cassandra.dht.*;
 import org.apache.cassandra.gms.ApplicationState;
@@ -77,19 +82,45 @@ public class Util
         return RowPosition.forKey(ByteBufferUtil.bytes(key), partitioner);
     }
 
+    public static CellName cellname(ByteBuffer... bbs)
+    {
+        if (bbs.length == 1)
+            return CellNames.simpleDense(bbs[0]);
+        else
+            return CellNames.compositeDense(bbs);
+    }
+
+    public static CellName cellname(String... strs)
+    {
+        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 Column column(String name, String value, long timestamp)
     {
-        return new Column(ByteBufferUtil.bytes(name), ByteBufferUtil.bytes(value), timestamp);
+        return new Column(cellname(name), ByteBufferUtil.bytes(value), timestamp);
     }
 
     public static Column expiringColumn(String name, String value, long timestamp, int ttl)
     {
-        return new ExpiringColumn(ByteBufferUtil.bytes(name), ByteBufferUtil.bytes(value), timestamp, ttl);
+        return new ExpiringColumn(cellname(name), ByteBufferUtil.bytes(value), timestamp, ttl);
     }
 
     public static Column counterColumn(String name, long value, long timestamp)
     {
-        return new CounterUpdateColumn(ByteBufferUtil.bytes(name), value, timestamp);
+        return new CounterUpdateColumn(cellname(name), value, timestamp);
     }
 
     public static Token token(String key)
@@ -114,9 +145,9 @@ public class Util
 
     public static void addMutation(RowMutation rm, String columnFamilyName, String superColumnName, long columnName, String value, long timestamp)
     {
-        ByteBuffer cname = superColumnName == null
-                         ? getBytes(columnName)
-                         : CompositeType.build(ByteBufferUtil.bytes(superColumnName), getBytes(columnName));
+        CellName cname = superColumnName == null
+                       ? CellNames.simpleDense(getBytes(columnName))
+                       : CellNames.compositeDense(ByteBufferUtil.bytes(superColumnName), getBytes(columnName));
         rm.add(columnFamilyName, cname, ByteBufferUtil.bytes(value), timestamp);
     }
 
@@ -281,4 +312,63 @@ public class Util
 
         assert thrown : exception.getName() + " not received";
     }
+
+    public static ByteBuffer serializeForSSTable(ColumnFamily cf)
+    {
+        try
+        {
+            ByteArrayOutputStream baos = new ByteArrayOutputStream();
+            DataOutputStream out = new DataOutputStream(baos);
+            DeletionTime.serializer.serialize(cf.deletionInfo().getTopLevelDeletion(), out);
+            out.writeInt(cf.getColumnCount());
+            new ColumnIndex.Builder(cf, ByteBufferUtil.EMPTY_BYTE_BUFFER, out).build(cf);
+            return ByteBuffer.wrap(baos.toByteArray());
+        }
+        catch (IOException e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+
+    public static QueryFilter namesQueryFilter(ColumnFamilyStore cfs, DecoratedKey key)
+    {
+        SortedSet<CellName> s = new TreeSet<CellName>(cfs.getComparator());
+        return QueryFilter.getNamesFilter(key, cfs.name, s, System.currentTimeMillis());
+    }
+
+    public static QueryFilter namesQueryFilter(ColumnFamilyStore cfs, DecoratedKey key, String... names)
+    {
+        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());
+    }
+
+    public static QueryFilter namesQueryFilter(ColumnFamilyStore cfs, DecoratedKey key, CellName... names)
+    {
+        SortedSet<CellName> s = new TreeSet<CellName>(cfs.getComparator());
+        for (CellName n : names)
+            s.add(n);
+        return QueryFilter.getNamesFilter(key, cfs.name, s, System.currentTimeMillis());
+    }
+
+    public static NamesQueryFilter namesFilter(ColumnFamilyStore cfs, String... names)
+    {
+        SortedSet<CellName> s = new TreeSet<CellName>(cfs.getComparator());
+        for (String str : names)
+            s.add(cellname(str));
+        return new NamesQueryFilter(s);
+    }
+
+    public static String string(ByteBuffer bb)
+    {
+        try
+        {
+            return ByteBufferUtil.string(bb);
+        }
+        catch (Exception e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/362cc053/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 b8c061f..fd8b637 100644
--- a/test/unit/org/apache/cassandra/config/CFMetaDataTest.java
+++ b/test/unit/org/apache/cassandra/config/CFMetaDataTest.java
@@ -22,6 +22,7 @@ import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
 import java.util.HashMap;
+import java.util.HashSet;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.cql3.QueryProcessor;
@@ -92,7 +93,7 @@ public class CFMetaDataTest extends SchemaLoader
         assertEquals(thriftCfDef.name, converted.name);
         assertEquals(thriftCfDef.default_validation_class, converted.default_validation_class);
         assertEquals(thriftCfDef.comment, converted.comment);
-        assertEquals(thriftCfDef.column_metadata, converted.column_metadata);
+        assertEquals(new HashSet<>(thriftCfDef.column_metadata), new HashSet<>(converted.column_metadata));
     }
 
     @Test
@@ -103,6 +104,9 @@ public class CFMetaDataTest extends SchemaLoader
             for (ColumnFamilyStore cfs : Keyspace.open(keyspaceName).getColumnFamilyStores())
             {
                 CFMetaData cfm = cfs.metadata;
+                if (!cfm.isThriftCompatible())
+                    continue;
+
                 checkInverses(cfm);
 
                 // Testing with compression to catch #3558
@@ -113,31 +117,13 @@ public class CFMetaDataTest extends SchemaLoader
         }
     }
 
-    private static CFMetaData withoutThriftIncompatible(CFMetaData cfm)
-    {
-        CFMetaData result = cfm.clone();
-
-        // This is a nasty hack to work around the fact that in thrift we exposes:
-        //   - neither definition with a non-nulll componentIndex
-        //   - nor non REGULAR definitions.
-        Iterator<ColumnDefinition> iter = result.allColumns().iterator();
-        while (iter.hasNext())
-        {
-            ColumnDefinition def = iter.next();
-            // Remove what we know is not thrift compatible
-            if (!def.isThriftCompatible())
-                iter.remove();
-        }
-        return result;
-    }
-
     private void checkInverses(CFMetaData cfm) throws Exception
     {
         DecoratedKey k = StorageService.getPartitioner().decorateKey(ByteBufferUtil.bytes(cfm.ksName));
 
         // Test thrift conversion
-        CFMetaData before = withoutThriftIncompatible(cfm);
-        CFMetaData after = withoutThriftIncompatible(CFMetaData.fromThrift(before.toThrift()));
+        CFMetaData before = cfm;
+        CFMetaData after = CFMetaData.fromThrift(before.toThrift());
         assert before.equals(after) : String.format("\n%s\n!=\n%s", before, after);
 
         // Test schema conversion
@@ -145,7 +131,7 @@ public class CFMetaDataTest extends SchemaLoader
         ColumnFamily serializedCf = rm.getColumnFamily(Schema.instance.getId(Keyspace.SYSTEM_KS, SystemKeyspace.SCHEMA_COLUMNFAMILIES_CF));
         ColumnFamily serializedCD = rm.getColumnFamily(Schema.instance.getId(Keyspace.SYSTEM_KS, SystemKeyspace.SCHEMA_COLUMNS_CF));
         UntypedResultSet.Row result = QueryProcessor.resultify("SELECT * FROM system.schema_columnfamilies", new Row(k, serializedCf)).one();
-        CFMetaData newCfm = CFMetaData.addColumnDefinitionsFromSchema(CFMetaData.fromSchemaNoColumnsNoTriggers(result), new Row(k, serializedCD));
+        CFMetaData newCfm = CFMetaData.fromSchemaNoTriggers(result, ColumnDefinition.resultify(new Row(k, serializedCD)));
         assert cfm.equals(newCfm) : String.format("\n%s\n!=\n%s", cfm, newCfm);
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/362cc053/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 d9f5a4f..8cc543d 100644
--- a/test/unit/org/apache/cassandra/config/ColumnDefinitionTest.java
+++ b/test/unit/org/apache/cassandra/config/ColumnDefinitionTest.java
@@ -33,7 +33,7 @@ public class ColumnDefinitionTest
     @Test
     public void testSerializeDeserialize() throws Exception
     {
-        CFMetaData cfm = new CFMetaData("ks", "cf", ColumnFamilyType.Standard, UTF8Type.instance);
+        CFMetaData cfm = CFMetaData.denseCFMetaData("ks", "cf", UTF8Type.instance);
 
         ColumnDefinition cd0 = ColumnDefinition.regularDef(cfm, ByteBufferUtil.bytes("TestColumnDefinitionName0"), BytesType.instance, null)
                                                .setIndex("random index name 0", IndexType.KEYS, null);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/362cc053/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 754c029..fd2cb82 100644
--- a/test/unit/org/apache/cassandra/config/DefsTest.java
+++ b/test/unit/org/apache/cassandra/config/DefsTest.java
@@ -28,6 +28,7 @@ import org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
 import org.apache.cassandra.db.*;
+import org.apache.cassandra.db.composites.*;
 import org.apache.cassandra.db.filter.QueryFilter;
 import org.apache.cassandra.db.marshal.BytesType;
 import org.apache.cassandra.db.marshal.UTF8Type;
@@ -40,6 +41,7 @@ import org.apache.cassandra.locator.OldNetworkTopologyStrategy;
 import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.service.MigrationManager;
 import org.apache.cassandra.utils.ByteBufferUtil;
+import static org.apache.cassandra.Util.cellname;
 
 import org.junit.Test;
 import org.junit.runner.RunWith;
@@ -53,7 +55,7 @@ public class DefsTest extends SchemaLoader
         CFMetaData cfm = new CFMetaData("Keyspace1",
                                         "TestApplyCFM_CF",
                                         ColumnFamilyType.Standard,
-                                        BytesType.instance);
+                                        new SimpleDenseCellNameType(BytesType.instance));
 
         for (int i = 0; i < 5; i++)
         {
@@ -65,7 +67,6 @@ public class DefsTest extends SchemaLoader
            .readRepairChance(0.5)
            .replicateOnWrite(false)
            .gcGraceSeconds(100000)
-           .defaultValidator(null)
            .minCompactionThreshold(500)
            .maxCompactionThreshold(500);
 
@@ -169,17 +170,18 @@ public class DefsTest extends SchemaLoader
         assert Schema.instance.getKSMetaData(ks).cfMetaData().get(newCf.cfName).equals(newCf);
 
         // now read and write to it.
+        CellName col0 = cellname("col0");
         DecoratedKey dk = Util.dk("key0");
         RowMutation rm = new RowMutation(ks, dk.key);
-        rm.add(cf, ByteBufferUtil.bytes("col0"), ByteBufferUtil.bytes("value0"), 1L);
+        rm.add(cf, col0, ByteBufferUtil.bytes("value0"), 1L);
         rm.apply();
         ColumnFamilyStore store = Keyspace.open(ks).getColumnFamilyStore(cf);
         assert store != null;
         store.forceBlockingFlush();
 
-        ColumnFamily cfam = store.getColumnFamily(QueryFilter.getNamesFilter(dk, cf, ByteBufferUtil.bytes("col0"), System.currentTimeMillis()));
-        assert cfam.getColumn(ByteBufferUtil.bytes("col0")) != null;
-        Column col = cfam.getColumn(ByteBufferUtil.bytes("col0"));
+        ColumnFamily cfam = store.getColumnFamily(Util.namesQueryFilter(store, dk, col0));
+        assert cfam.getColumn(col0) != null;
+        Column col = cfam.getColumn(col0);
         assert ByteBufferUtil.bytes("value0").equals(col.value());
     }
 
@@ -196,7 +198,7 @@ public class DefsTest extends SchemaLoader
         // write some data, force a flush, then verify that files exist on disk.
         RowMutation rm = new RowMutation(ks.name, dk.key);
         for (int i = 0; i < 100; i++)
-            rm.add(cfm.cfName, ByteBufferUtil.bytes(("col" + i)), ByteBufferUtil.bytes("anyvalue"), 1L);
+            rm.add(cfm.cfName, cellname("col" + i), ByteBufferUtil.bytes("anyvalue"), 1L);
         rm.apply();
         ColumnFamilyStore store = Keyspace.open(cfm.ksName).getColumnFamilyStore(cfm.cfName);
         assert store != null;
@@ -212,7 +214,7 @@ public class DefsTest extends SchemaLoader
         boolean success = true;
         try
         {
-            rm.add("Standard1", ByteBufferUtil.bytes("col0"), ByteBufferUtil.bytes("value0"), 1L);
+            rm.add("Standard1", cellname("col0"), ByteBufferUtil.bytes("value0"), 1L);
             rm.apply();
         }
         catch (Throwable th)
@@ -243,16 +245,17 @@ public class DefsTest extends SchemaLoader
         assert Schema.instance.getKSMetaData(newCf.ksName).equals(newKs);
 
         // test reads and writes.
+        CellName col0 = cellname("col0");
         RowMutation rm = new RowMutation(newCf.ksName, dk.key);
-        rm.add(newCf.cfName, ByteBufferUtil.bytes("col0"), ByteBufferUtil.bytes("value0"), 1L);
+        rm.add(newCf.cfName, col0, ByteBufferUtil.bytes("value0"), 1L);
         rm.apply();
         ColumnFamilyStore store = Keyspace.open(newCf.ksName).getColumnFamilyStore(newCf.cfName);
         assert store != null;
         store.forceBlockingFlush();
 
-        ColumnFamily cfam = store.getColumnFamily(QueryFilter.getNamesFilter(dk, newCf.cfName, ByteBufferUtil.bytes("col0"), System.currentTimeMillis()));
-        assert cfam.getColumn(ByteBufferUtil.bytes("col0")) != null;
-        Column col = cfam.getColumn(ByteBufferUtil.bytes("col0"));
+        ColumnFamily cfam = store.getColumnFamily(Util.namesQueryFilter(store, dk, col0));
+        assert cfam.getColumn(col0) != null;
+        Column col = cfam.getColumn(col0);
         assert ByteBufferUtil.bytes("value0").equals(col.value());
     }
 
@@ -269,7 +272,7 @@ public class DefsTest extends SchemaLoader
         // write some data, force a flush, then verify that files exist on disk.
         RowMutation rm = new RowMutation(ks.name, dk.key);
         for (int i = 0; i < 100; i++)
-            rm.add(cfm.cfName, ByteBufferUtil.bytes(("col" + i)), ByteBufferUtil.bytes("anyvalue"), 1L);
+            rm.add(cfm.cfName, cellname("col" + i), ByteBufferUtil.bytes("anyvalue"), 1L);
         rm.apply();
         ColumnFamilyStore store = Keyspace.open(cfm.ksName).getColumnFamilyStore(cfm.cfName);
         assert store != null;
@@ -285,7 +288,7 @@ public class DefsTest extends SchemaLoader
         boolean success = true;
         try
         {
-            rm.add("Standard1", ByteBufferUtil.bytes("col0"), ByteBufferUtil.bytes("value0"), 1L);
+            rm.add("Standard1", cellname("col0"), ByteBufferUtil.bytes("value0"), 1L);
             rm.apply();
         }
         catch (Throwable th)
@@ -320,7 +323,7 @@ public class DefsTest extends SchemaLoader
         // write some data
         RowMutation rm = new RowMutation(ks.name, dk.key);
         for (int i = 0; i < 100; i++)
-            rm.add(cfm.cfName, ByteBufferUtil.bytes(("col" + i)), ByteBufferUtil.bytes("anyvalue"), 1L);
+            rm.add(cfm.cfName, cellname("col" + i), ByteBufferUtil.bytes("anyvalue"), 1L);
         rm.apply();
 
         MigrationManager.announceKeyspaceDrop(ks.name);
@@ -350,17 +353,18 @@ public class DefsTest extends SchemaLoader
         assert Schema.instance.getKSMetaData(newKs.name).cfMetaData().get(newCf.cfName).equals(newCf);
 
         // now read and write to it.
+        CellName col0 = cellname("col0");
         DecoratedKey dk = Util.dk("key0");
         RowMutation rm = new RowMutation(newKs.name, dk.key);
-        rm.add(newCf.cfName, ByteBufferUtil.bytes("col0"), ByteBufferUtil.bytes("value0"), 1L);
+        rm.add(newCf.cfName, col0, ByteBufferUtil.bytes("value0"), 1L);
         rm.apply();
         ColumnFamilyStore store = Keyspace.open(newKs.name).getColumnFamilyStore(newCf.cfName);
         assert store != null;
         store.forceBlockingFlush();
 
-        ColumnFamily cfam = store.getColumnFamily(QueryFilter.getNamesFilter(dk, newCf.cfName, ByteBufferUtil.bytes("col0"), System.currentTimeMillis()));
-        assert cfam.getColumn(ByteBufferUtil.bytes("col0")) != null;
-        Column col = cfam.getColumn(ByteBufferUtil.bytes("col0"));
+        ColumnFamily cfam = store.getColumnFamily(Util.namesQueryFilter(store, dk, col0));
+        assert cfam.getColumn(col0) != null;
+        Column col = cfam.getColumn(col0);
         assert ByteBufferUtil.bytes("value0").equals(col.value());
     }
 
@@ -410,7 +414,6 @@ public class DefsTest extends SchemaLoader
 
         // updating certain fields should fail.
         CFMetaData newCfm = cf.clone();
-        newCfm.columnMetadata(new HashMap<ByteBuffer, ColumnDefinition>());
         newCfm.defaultValidator(BytesType.instance);
         newCfm.minCompactionThreshold(5);
         newCfm.maxCompactionThreshold(31);
@@ -483,7 +486,7 @@ public class DefsTest extends SchemaLoader
         catch (ConfigurationException expected) {}
 
         // Change comparator
-        newCfm = new CFMetaData(cf.ksName, cf.cfName, cf.cfType, TimeUUIDType.instance);
+        newCfm = new CFMetaData(cf.ksName, cf.cfName, cf.cfType, new SimpleDenseCellNameType(TimeUUIDType.instance));
         CFMetaData.copyOpts(newCfm, cf);
         try
         {
@@ -498,13 +501,13 @@ public class DefsTest extends SchemaLoader
     {
         // persist keyspace definition in the system keyspace
         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
         RowMutation rm = new RowMutation("Keyspace6", ByteBufferUtil.bytes("k1"));
-        rm.add("Indexed1", ByteBufferUtil.bytes("notbirthdate"), ByteBufferUtil.bytes(1L), 0);
-        rm.add("Indexed1", ByteBufferUtil.bytes("birthdate"), ByteBufferUtil.bytes(1L), 0);
+        rm.add("Indexed1", cellname("notbirthdate"), ByteBufferUtil.bytes(1L), 0);
+        rm.add("Indexed1", cellname("birthdate"), ByteBufferUtil.bytes(1L), 0);
         rm.apply();
-        ColumnFamilyStore cfs = Keyspace.open("Keyspace6").getColumnFamilyStore("Indexed1");
         cfs.forceBlockingFlush();
         ColumnFamilyStore indexedCfs = cfs.indexManager.getIndexForColumn(ByteBufferUtil.bytes("birthdate")).getIndexCfs();
         Descriptor desc = indexedCfs.getSSTables().iterator().next().descriptor;
@@ -524,7 +527,7 @@ public class DefsTest extends SchemaLoader
 
     private CFMetaData addTestCF(String ks, String cf, String comment)
     {
-        CFMetaData newCFMD = new CFMetaData(ks, cf, ColumnFamilyType.Standard, UTF8Type.instance);
+        CFMetaData newCFMD = new CFMetaData(ks, cf, ColumnFamilyType.Standard, new SimpleDenseCellNameType(UTF8Type.instance));
         newCFMD.comment(comment)
                .readRepairChance(0.0);
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/362cc053/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 c720d28..9074cf1 100644
--- a/test/unit/org/apache/cassandra/db/ArrayBackedSortedColumnsTest.java
+++ b/test/unit/org/apache/cassandra/db/ArrayBackedSortedColumnsTest.java
@@ -33,7 +33,9 @@ import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.Schema;
 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.HeapAllocator;
 
 public class ArrayBackedSortedColumnsTest extends SchemaLoader
@@ -52,16 +54,17 @@ public class ArrayBackedSortedColumnsTest extends SchemaLoader
 
     private void testAddInternal(boolean reversed)
     {
+        CellNameType type = new SimpleDenseCellNameType(Int32Type.instance);
         ColumnFamily map = ArrayBackedSortedColumns.factory.create(metadata(), reversed);
         int[] values = new int[]{ 1, 2, 2, 3 };
 
         for (int i = 0; i < values.length; ++i)
-            map.addColumn(new Column(ByteBufferUtil.bytes(values[reversed ? values.length - 1 - i : i])), HeapAllocator.instance);
+            map.addColumn(new Column(type.makeCellName(values[reversed ? values.length - 1 - i : i])), HeapAllocator.instance);
 
         Iterator<Column> iter = map.iterator();
-        assertEquals("1st column", 1, iter.next().name().getInt(0));
-        assertEquals("2nd column", 2, iter.next().name().getInt(0));
-        assertEquals("3rd column", 3, iter.next().name().getInt(0));
+        assertEquals("1st column", 1, iter.next().name().toByteBuffer().getInt(0));
+        assertEquals("2nd column", 2, iter.next().name().toByteBuffer().getInt(0));
+        assertEquals("3rd column", 3, iter.next().name().toByteBuffer().getInt(0));
     }
 
     @Test
@@ -73,6 +76,7 @@ public class ArrayBackedSortedColumnsTest extends SchemaLoader
 
     private void testAddAllInternal(boolean reversed)
     {
+        CellNameType type = new SimpleDenseCellNameType(Int32Type.instance);
         ColumnFamily map = ArrayBackedSortedColumns.factory.create(metadata(), reversed);
         ColumnFamily map2 = ArrayBackedSortedColumns.factory.create(metadata(), reversed);
 
@@ -80,20 +84,20 @@ public class ArrayBackedSortedColumnsTest extends SchemaLoader
         int[] values2 = new int[]{ 2, 4, 5, 6 };
 
         for (int i = 0; i < values1.length; ++i)
-            map.addColumn(new Column(ByteBufferUtil.bytes(values1[reversed ? values1.length - 1 - i : i])), HeapAllocator.instance);
+            map.addColumn(new Column(type.makeCellName(values1[reversed ? values1.length - 1 - i : i])), HeapAllocator.instance);
 
         for (int i = 0; i < values2.length; ++i)
-            map2.addColumn(new Column(ByteBufferUtil.bytes(values2[reversed ? values2.length - 1 - i : i])), HeapAllocator.instance);
+            map2.addColumn(new Column(type.makeCellName(values2[reversed ? values2.length - 1 - i : i])), HeapAllocator.instance);
 
         map2.addAll(map, HeapAllocator.instance, Functions.<Column>identity());
 
         Iterator<Column> iter = map2.iterator();
-        assertEquals("1st column", 1, iter.next().name().getInt(0));
-        assertEquals("2nd column", 2, iter.next().name().getInt(0));
-        assertEquals("3rd column", 3, iter.next().name().getInt(0));
-        assertEquals("4st column", 4, iter.next().name().getInt(0));
-        assertEquals("5st column", 5, iter.next().name().getInt(0));
-        assertEquals("6st column", 6, iter.next().name().getInt(0));
+        assertEquals("1st column", 1, iter.next().name().toByteBuffer().getInt(0));
+        assertEquals("2nd column", 2, iter.next().name().toByteBuffer().getInt(0));
+        assertEquals("3rd column", 3, iter.next().name().toByteBuffer().getInt(0));
+        assertEquals("4st column", 4, iter.next().name().toByteBuffer().getInt(0));
+        assertEquals("5st column", 5, iter.next().name().toByteBuffer().getInt(0));
+        assertEquals("6st column", 6, iter.next().name().toByteBuffer().getInt(0));
     }
 
     @Test
@@ -105,17 +109,18 @@ public class ArrayBackedSortedColumnsTest extends SchemaLoader
 
     private void testGetCollectionInternal(boolean reversed)
     {
+        CellNameType type = new SimpleDenseCellNameType(Int32Type.instance);
         ColumnFamily map = ArrayBackedSortedColumns.factory.create(metadata(), reversed);
         int[] values = new int[]{ 1, 2, 3, 5, 9 };
 
         List<Column> sorted = new ArrayList<Column>();
         for (int v : values)
-            sorted.add(new Column(ByteBufferUtil.bytes(v)));
+            sorted.add(new Column(type.makeCellName(v)));
         List<Column> reverseSorted = new ArrayList<Column>(sorted);
         Collections.reverse(reverseSorted);
 
         for (int i = 0; i < values.length; ++i)
-            map.addColumn(new Column(ByteBufferUtil.bytes(values[reversed ? values.length - 1 - i : i])), HeapAllocator.instance);
+            map.addColumn(new Column(type.makeCellName(values[reversed ? values.length - 1 - i : i])), HeapAllocator.instance);
 
         assertSame(sorted, map.getSortedColumns());
         assertSame(reverseSorted, map.getReverseSortedColumns());
@@ -130,15 +135,16 @@ public class ArrayBackedSortedColumnsTest extends SchemaLoader
 
     private void testIteratorInternal(boolean reversed)
     {
+        CellNameType type = new SimpleDenseCellNameType(Int32Type.instance);
         ColumnFamily map = ArrayBackedSortedColumns.factory.create(metadata(), reversed);
 
         int[] values = new int[]{ 1, 2, 3, 5, 9 };
 
         for (int i = 0; i < values.length; ++i)
-            map.addColumn(new Column(ByteBufferUtil.bytes(values[reversed ? values.length - 1 - i : i])), HeapAllocator.instance);
+            map.addColumn(new Column(type.makeCellName(values[reversed ? values.length - 1 - i : i])), HeapAllocator.instance);
 
-        assertSame(new int[]{ 3, 2, 1 }, map.reverseIterator(new ColumnSlice[]{ new ColumnSlice(ByteBufferUtil.bytes(3), ByteBufferUtil.EMPTY_BYTE_BUFFER) }));
-        assertSame(new int[]{ 3, 2, 1 }, map.reverseIterator(new ColumnSlice[]{ new ColumnSlice(ByteBufferUtil.bytes(4), ByteBufferUtil.EMPTY_BYTE_BUFFER) }));
+        assertSame(new int[]{ 3, 2, 1 }, map.reverseIterator(new ColumnSlice[]{ new ColumnSlice(type.make(3), Composites.EMPTY) }));
+        assertSame(new int[]{ 3, 2, 1 }, map.reverseIterator(new ColumnSlice[]{ new ColumnSlice(type.make(4), Composites.EMPTY) }));
 
         assertSame(map.iterator(), map.iterator(ColumnSlice.ALL_COLUMNS_ARRAY));
     }
@@ -161,7 +167,7 @@ public class ArrayBackedSortedColumnsTest extends SchemaLoader
         for (int name : names)
         {
             assert iter.hasNext() : "Expected " + name + " but no more result";
-            int value = ByteBufferUtil.toInt(iter.next().name());
+            int value = ByteBufferUtil.toInt(iter.next().name().toByteBuffer());
             assert name == value : "Expected " + name + " but got " + value;
         }
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/362cc053/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 8199275..29a538e 100644
--- a/test/unit/org/apache/cassandra/db/CleanupTest.java
+++ b/test/unit/org/apache/cassandra/db/CleanupTest.java
@@ -149,7 +149,7 @@ public class CleanupTest extends SchemaLoader
             // create a row and update the birthdate value, test that the index query fetches the new version
             RowMutation rm;
             rm = new RowMutation(KEYSPACE1, ByteBufferUtil.bytes(key));
-            rm.add(cfs.name, COLUMN, VALUE, System.currentTimeMillis());
+            rm.add(cfs.name, Util.cellname(COLUMN), VALUE, System.currentTimeMillis());
             rm.applyUnsafe();
         }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/362cc053/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 8763a41..5b7197b 100644
--- a/test/unit/org/apache/cassandra/db/CollationControllerTest.java
+++ b/test/unit/org/apache/cassandra/db/CollationControllerTest.java
@@ -26,8 +26,10 @@ import java.util.concurrent.ExecutionException;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
+import org.apache.cassandra.db.composites.CellName;
 import org.apache.cassandra.db.filter.QueryFilter;
 import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
 import org.junit.Test;
 
 import org.apache.cassandra.io.sstable.SSTableReader;
@@ -45,7 +47,7 @@ public class CollationControllerTest extends SchemaLoader
         
         // add data
         rm = new RowMutation(keyspace.getName(), dk.key);
-        rm.add(cfs.name, ByteBufferUtil.bytes("Column1"), ByteBufferUtil.bytes("asdf"), 0);
+        rm.add(cfs.name, Util.cellname("Column1"), ByteBufferUtil.bytes("asdf"), 0);
         rm.apply();
         cfs.forceBlockingFlush();
         
@@ -57,20 +59,20 @@ public class CollationControllerTest extends SchemaLoader
         // add another mutation because sstable maxtimestamp isn't set
         // correctly during flush if the most recent mutation is a row delete
         rm = new RowMutation(keyspace.getName(), Util.dk("key2").key);
-        rm.add(cfs.name, ByteBufferUtil.bytes("Column1"), ByteBufferUtil.bytes("zxcv"), 20);
+        rm.add(cfs.name, Util.cellname("Column1"), ByteBufferUtil.bytes("zxcv"), 20);
         rm.apply();
         
         cfs.forceBlockingFlush();
 
         // add yet one more mutation
         rm = new RowMutation(keyspace.getName(), dk.key);
-        rm.add(cfs.name, ByteBufferUtil.bytes("Column1"), ByteBufferUtil.bytes("foobar"), 30);
+        rm.add(cfs.name, Util.cellname("Column1"), ByteBufferUtil.bytes("foobar"), 30);
         rm.apply();
         cfs.forceBlockingFlush();
 
         // A NamesQueryFilter goes down one code path (through collectTimeOrderedData())
         // It should only iterate the last flushed sstable, since it probably contains the most recent value for Column1
-        QueryFilter filter = QueryFilter.getNamesFilter(dk, cfs.name, ByteBufferUtil.bytes("Column1"), System.currentTimeMillis());
+        QueryFilter filter = Util.namesQueryFilter(cfs, dk, "Column1");
         CollationController controller = new CollationController(cfs, filter, Integer.MIN_VALUE);
         controller.getTopLevelColumns();
         assertEquals(1, controller.getSstablesIterated());
@@ -94,7 +96,7 @@ public class CollationControllerTest extends SchemaLoader
 
         RowMutation rm;
         DecoratedKey dk = Util.dk("key1");
-        ByteBuffer cellName = ByteBufferUtil.bytes("Column1");
+        CellName cellName = Util.cellname("Column1");
 
         // add data
         rm = new RowMutation(keyspace.getName(), dk.key);
@@ -113,7 +115,7 @@ public class CollationControllerTest extends SchemaLoader
         long queryAt = System.currentTimeMillis() + 1000;
         int gcBefore = cfs.gcBefore(queryAt);
 
-        filter = QueryFilter.getNamesFilter(dk, cfs.name, cellName, queryAt);
+        filter = QueryFilter.getNamesFilter(dk, cfs.name, FBUtilities.singleton(cellName, cfs.getComparator()), queryAt);
         CollationController controller = new CollationController(cfs, filter, gcBefore);
         assert ColumnFamilyStore.removeDeleted(controller.getTopLevelColumns(), gcBefore) == null;