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:39 UTC

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

http://git-wip-us.apache.org/repos/asf/cassandra/blob/362cc053/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java b/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
index ace32e7..e807e2e 100644
--- a/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
+++ b/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
@@ -41,6 +41,7 @@ import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.config.IndexType;
 import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.db.composites.*;
 import org.apache.cassandra.db.columniterator.IdentityQueryFilter;
 import org.apache.cassandra.db.filter.*;
 import org.apache.cassandra.db.index.SecondaryIndex;
@@ -53,6 +54,7 @@ import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.thrift.*;
 import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.WrappedRunnable;
 
 import static org.junit.Assert.*;
@@ -84,17 +86,17 @@ public class ColumnFamilyStoreTest extends SchemaLoader
 
         RowMutation rm;
         rm = new RowMutation("Keyspace1", ByteBufferUtil.bytes("key1"));
-        rm.add("Standard1", ByteBufferUtil.bytes("Column1"), ByteBufferUtil.bytes("asdf"), 0);
+        rm.add("Standard1", cellname("Column1"), ByteBufferUtil.bytes("asdf"), 0);
         rm.apply();
         cfs.forceBlockingFlush();
 
         rm = new RowMutation("Keyspace1", ByteBufferUtil.bytes("key1"));
-        rm.add("Standard1", ByteBufferUtil.bytes("Column1"), ByteBufferUtil.bytes("asdf"), 1);
+        rm.add("Standard1", cellname("Column1"), ByteBufferUtil.bytes("asdf"), 1);
         rm.apply();
         cfs.forceBlockingFlush();
 
         cfs.getRecentSSTablesPerReadHistogram(); // resets counts
-        cfs.getColumnFamily(QueryFilter.getNamesFilter(Util.dk("key1"), "Standard1", ByteBufferUtil.bytes("Column1"), System.currentTimeMillis()));
+        cfs.getColumnFamily(Util.namesQueryFilter(cfs, Util.dk("key1"), "Column1"));
         assertEquals(1, cfs.getRecentSSTablesPerReadHistogram()[0]);
     }
 
@@ -108,8 +110,8 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         List<IMutation> rms = new LinkedList<IMutation>();
         RowMutation rm;
         rm = new RowMutation("Keyspace1", ByteBufferUtil.bytes("key1"));
-        rm.add("Standard1", ByteBufferUtil.bytes("Column1"), ByteBufferUtil.bytes("asdf"), 0);
-        rm.add("Standard1", ByteBufferUtil.bytes("Column2"), ByteBufferUtil.bytes("asdf"), 0);
+        rm.add("Standard1", cellname("Column1"), ByteBufferUtil.bytes("asdf"), 0);
+        rm.add("Standard1", cellname("Column2"), ByteBufferUtil.bytes("asdf"), 0);
         rms.add(rm);
         Util.writeColumnFamily(rms);
 
@@ -135,21 +137,12 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         {
             public void runMayThrow() throws IOException
             {
-                QueryFilter sliceFilter = QueryFilter.getSliceFilter(Util.dk("key1"),
-                                                                     "Standard2",
-                                                                     ByteBufferUtil.EMPTY_BYTE_BUFFER,
-                                                                     ByteBufferUtil.EMPTY_BYTE_BUFFER,
-                                                                     false,
-                                                                     1,
-                                                                     System.currentTimeMillis());
+                QueryFilter sliceFilter = QueryFilter.getSliceFilter(Util.dk("key1"), "Standard2", Composites.EMPTY, Composites.EMPTY, false, 1, System.currentTimeMillis());
                 ColumnFamily cf = store.getColumnFamily(sliceFilter);
                 assert cf.isMarkedForDelete();
                 assert cf.getColumnCount() == 0;
 
-                QueryFilter namesFilter = QueryFilter.getNamesFilter(Util.dk("key1"),
-                                                                     "Standard2",
-                                                                     ByteBufferUtil.bytes("a"),
-                                                                     System.currentTimeMillis());
+                QueryFilter namesFilter = Util.namesQueryFilter(store, Util.dk("key1"), "a");
                 cf = store.getColumnFamily(namesFilter);
                 assert cf.isMarkedForDelete();
                 assert cf.getColumnCount() == 0;
@@ -167,7 +160,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         IPartitioner p = StorageService.getPartitioner();
         List<Row> result = cfs.getRangeSlice(Util.range(p, "key1", "key2"),
                                              null,
-                                             new NamesQueryFilter(ByteBufferUtil.bytes("asdf")),
+                                             Util.namesFilter(cfs, "asdf"),
                                              10);
         assertEquals(1, result.size());
         assert result.get(0).key.key.equals(ByteBufferUtil.bytes("key2"));
@@ -176,26 +169,29 @@ public class ColumnFamilyStoreTest extends SchemaLoader
     @Test
     public void testIndexScan()
     {
+        ColumnFamilyStore cfs = Keyspace.open("Keyspace1").getColumnFamilyStore("Indexed1");
         RowMutation rm;
+        CellName nobirthdate = cellname("notbirthdate");
+        CellName birthdate = cellname("birthdate");
 
         rm = new RowMutation("Keyspace1", 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", nobirthdate, ByteBufferUtil.bytes(1L), 0);
+        rm.add("Indexed1", birthdate, ByteBufferUtil.bytes(1L), 0);
         rm.apply();
 
         rm = new RowMutation("Keyspace1", ByteBufferUtil.bytes("k2"));
-        rm.add("Indexed1", ByteBufferUtil.bytes("notbirthdate"), ByteBufferUtil.bytes(2L), 0);
-        rm.add("Indexed1", ByteBufferUtil.bytes("birthdate"), ByteBufferUtil.bytes(2L), 0);
+        rm.add("Indexed1", nobirthdate, ByteBufferUtil.bytes(2L), 0);
+        rm.add("Indexed1", birthdate, ByteBufferUtil.bytes(2L), 0);
         rm.apply();
 
         rm = new RowMutation("Keyspace1", ByteBufferUtil.bytes("k3"));
-        rm.add("Indexed1", ByteBufferUtil.bytes("notbirthdate"), ByteBufferUtil.bytes(2L), 0);
-        rm.add("Indexed1", ByteBufferUtil.bytes("birthdate"), ByteBufferUtil.bytes(1L), 0);
+        rm.add("Indexed1", nobirthdate, ByteBufferUtil.bytes(2L), 0);
+        rm.add("Indexed1", birthdate, ByteBufferUtil.bytes(1L), 0);
         rm.apply();
 
         rm = new RowMutation("Keyspace1", ByteBufferUtil.bytes("k4aaaa"));
-        rm.add("Indexed1", ByteBufferUtil.bytes("notbirthdate"), ByteBufferUtil.bytes(2L), 0);
-        rm.add("Indexed1", ByteBufferUtil.bytes("birthdate"), ByteBufferUtil.bytes(3L), 0);
+        rm.add("Indexed1", nobirthdate, ByteBufferUtil.bytes(2L), 0);
+        rm.add("Indexed1", birthdate, ByteBufferUtil.bytes(3L), 0);
         rm.apply();
 
         // basic single-expression query
@@ -203,7 +199,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         List<IndexExpression> clause = Arrays.asList(expr);
         IDiskAtomFilter filter = new IdentityQueryFilter();
         Range<RowPosition> range = Util.range("", "");
-        List<Row> rows = Keyspace.open("Keyspace1").getColumnFamilyStore("Indexed1").search(range, clause, filter, 100);
+        List<Row> rows = cfs.search(range, clause, filter, 100);
 
         assert rows != null;
         assert rows.size() == 2 : StringUtils.join(rows, ",");
@@ -214,20 +210,20 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         key = new String(rows.get(1).key.key.array(),rows.get(1).key.key.position(),rows.get(1).key.key.remaining());
         assert "k3".equals(key) : key;
 
-        assert ByteBufferUtil.bytes(1L).equals( rows.get(0).cf.getColumn(ByteBufferUtil.bytes("birthdate")).value());
-        assert ByteBufferUtil.bytes(1L).equals( rows.get(1).cf.getColumn(ByteBufferUtil.bytes("birthdate")).value());
+        assert ByteBufferUtil.bytes(1L).equals( rows.get(0).cf.getColumn(birthdate).value());
+        assert ByteBufferUtil.bytes(1L).equals( rows.get(1).cf.getColumn(birthdate).value());
 
         // add a second expression
         IndexExpression expr2 = new IndexExpression(ByteBufferUtil.bytes("notbirthdate"), IndexExpression.Operator.GTE, ByteBufferUtil.bytes(2L));
         clause = Arrays.asList(expr, expr2);
-        rows = Keyspace.open("Keyspace1").getColumnFamilyStore("Indexed1").search(range, clause, filter, 100);
+        rows = cfs.search(range, clause, filter, 100);
 
         assert rows.size() == 1 : StringUtils.join(rows, ",");
         key = new String(rows.get(0).key.key.array(),rows.get(0).key.key.position(),rows.get(0).key.key.remaining());
         assert "k3".equals( key );
 
         // same query again, but with resultset not including the subordinate expression
-        rows = Keyspace.open("Keyspace1").getColumnFamilyStore("Indexed1").search(range, clause, new NamesQueryFilter(ByteBufferUtil.bytes("birthdate")), 100);
+        rows = cfs.search(range, clause, Util.namesFilter(cfs, "birthdate"), 100);
 
         assert rows.size() == 1 : StringUtils.join(rows, ",");
         key = new String(rows.get(0).key.key.array(),rows.get(0).key.key.position(),rows.get(0).key.key.remaining());
@@ -236,8 +232,8 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         assert rows.get(0).cf.getColumnCount() == 1 : rows.get(0).cf;
 
         // once more, this time with a slice rowset that needs to be expanded
-        SliceQueryFilter emptyFilter = new SliceQueryFilter(ByteBufferUtil.EMPTY_BYTE_BUFFER, ByteBufferUtil.EMPTY_BYTE_BUFFER, false, 0);
-        rows = Keyspace.open("Keyspace1").getColumnFamilyStore("Indexed1").search(range, clause, emptyFilter, 100);
+        SliceQueryFilter emptyFilter = new SliceQueryFilter(Composites.EMPTY, Composites.EMPTY, false, 0);
+        rows = cfs.search(range, clause, emptyFilter, 100);
 
         assert rows.size() == 1 : StringUtils.join(rows, ",");
         key = new String(rows.get(0).key.key.array(),rows.get(0).key.key.position(),rows.get(0).key.key.remaining());
@@ -249,7 +245,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         // doesn't tell the scan loop that it's done
         IndexExpression expr3 = new IndexExpression(ByteBufferUtil.bytes("notbirthdate"), IndexExpression.Operator.EQ, ByteBufferUtil.bytes(-1L));
         clause = Arrays.asList(expr, expr3);
-        rows = Keyspace.open("Keyspace1").getColumnFamilyStore("Indexed1").search(range, clause, filter, 100);
+        rows = cfs.search(range, clause, filter, 100);
 
         assert rows.isEmpty();
     }
@@ -258,11 +254,12 @@ public class ColumnFamilyStoreTest extends SchemaLoader
     public void testLargeScan()
     {
         RowMutation rm;
+        ColumnFamilyStore cfs = Keyspace.open("Keyspace1").getColumnFamilyStore("Indexed1");
         for (int i = 0; i < 100; i++)
         {
             rm = new RowMutation("Keyspace1", ByteBufferUtil.bytes("key" + i));
-            rm.add("Indexed1", ByteBufferUtil.bytes("birthdate"), ByteBufferUtil.bytes(34L), 0);
-            rm.add("Indexed1", ByteBufferUtil.bytes("notbirthdate"), ByteBufferUtil.bytes((long) (i % 2)), 0);
+            rm.add("Indexed1", cellname("birthdate"), ByteBufferUtil.bytes(34L), 0);
+            rm.add("Indexed1", cellname("notbirthdate"), ByteBufferUtil.bytes((long) (i % 2)), 0);
             rm.applyUnsafe();
         }
 
@@ -271,7 +268,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         List<IndexExpression> clause = Arrays.asList(expr, expr2);
         IDiskAtomFilter filter = new IdentityQueryFilter();
         Range<RowPosition> range = Util.range("", "");
-        List<Row> rows = Keyspace.open("Keyspace1").getColumnFamilyStore("Indexed1").search(range, clause, filter, 100);
+        List<Row> rows = cfs.search(range, clause, filter, 100);
 
         assert rows != null;
         assert rows.size() == 50 : rows.size();
@@ -289,7 +286,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         RowMutation rm;
 
         rm = new RowMutation("Keyspace3", ByteBufferUtil.bytes("k1"));
-        rm.add("Indexed1", ByteBufferUtil.bytes("birthdate"), ByteBufferUtil.bytes(1L), 0);
+        rm.add("Indexed1", cellname("birthdate"), ByteBufferUtil.bytes(1L), 0);
         rm.apply();
 
         IndexExpression expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), IndexExpression.Operator.EQ, ByteBufferUtil.bytes(1L));
@@ -303,7 +300,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
 
         // delete the column directly
         rm = new RowMutation("Keyspace3", ByteBufferUtil.bytes("k1"));
-        rm.delete("Indexed1", ByteBufferUtil.bytes("birthdate"), 1);
+        rm.delete("Indexed1", cellname("birthdate"), 1);
         rm.apply();
         rows = cfs.search(range, clause, filter, 100);
         assert rows.isEmpty();
@@ -318,7 +315,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
 
         // resurrect w/ a newer timestamp
         rm = new RowMutation("Keyspace3", ByteBufferUtil.bytes("k1"));
-        rm.add("Indexed1", ByteBufferUtil.bytes("birthdate"), ByteBufferUtil.bytes(1L), 2);
+        rm.add("Indexed1", cellname("birthdate"), ByteBufferUtil.bytes(1L), 2);
         rm.apply();
         rows = cfs.search(range, clause, filter, 100);
         assert rows.size() == 1 : StringUtils.join(rows, ",");
@@ -336,7 +333,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
 
         // similarly, column delete w/ older timestamp should do nothing
         rm = new RowMutation("Keyspace3", ByteBufferUtil.bytes("k1"));
-        rm.delete("Indexed1", ByteBufferUtil.bytes("birthdate"), 1);
+        rm.delete("Indexed1", cellname("birthdate"), 1);
         rm.apply();
         rows = cfs.search(range, clause, filter, 100);
         assert rows.size() == 1 : StringUtils.join(rows, ",");
@@ -352,14 +349,14 @@ public class ColumnFamilyStoreTest extends SchemaLoader
 
         // make sure obsolete mutations don't generate an index entry
         rm = new RowMutation("Keyspace3", ByteBufferUtil.bytes("k1"));
-        rm.add("Indexed1", ByteBufferUtil.bytes("birthdate"), ByteBufferUtil.bytes(1L), 3);
+        rm.add("Indexed1", cellname("birthdate"), ByteBufferUtil.bytes(1L), 3);
         rm.apply();
         rows = cfs.search(range, clause, filter, 100);
         assert rows.isEmpty() : StringUtils.join(rows, ",");
 
         // try insert followed by row delete in the same mutation
         rm = new RowMutation("Keyspace3", ByteBufferUtil.bytes("k1"));
-        rm.add("Indexed1", ByteBufferUtil.bytes("birthdate"), ByteBufferUtil.bytes(1L), 1);
+        rm.add("Indexed1", cellname("birthdate"), ByteBufferUtil.bytes(1L), 1);
         rm.delete("Indexed1", 2);
         rm.apply();
         rows = cfs.search(range, clause, filter, 100);
@@ -368,7 +365,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         // try row delete followed by insert in the same mutation
         rm = new RowMutation("Keyspace3", ByteBufferUtil.bytes("k1"));
         rm.delete("Indexed1", 3);
-        rm.add("Indexed1", ByteBufferUtil.bytes("birthdate"), ByteBufferUtil.bytes(1L), 4);
+        rm.add("Indexed1", cellname("birthdate"), ByteBufferUtil.bytes(1L), 4);
         rm.apply();
         rows = cfs.search(range, clause, filter, 100);
         assert rows.size() == 1 : StringUtils.join(rows, ",");
@@ -380,21 +377,23 @@ public class ColumnFamilyStoreTest extends SchemaLoader
     public void testIndexUpdate() throws IOException
     {
         Keyspace keyspace = Keyspace.open("Keyspace2");
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("Indexed1");
+        CellName birthdate = cellname("birthdate");
 
         // create a row and update the birthdate value, test that the index query fetches the new version
         RowMutation rm;
         rm = new RowMutation("Keyspace2", ByteBufferUtil.bytes("k1"));
-        rm.add("Indexed1", ByteBufferUtil.bytes("birthdate"), ByteBufferUtil.bytes(1L), 1);
+        rm.add("Indexed1", birthdate, ByteBufferUtil.bytes(1L), 1);
         rm.apply();
         rm = new RowMutation("Keyspace2", ByteBufferUtil.bytes("k1"));
-        rm.add("Indexed1", ByteBufferUtil.bytes("birthdate"), ByteBufferUtil.bytes(2L), 2);
+        rm.add("Indexed1", birthdate, ByteBufferUtil.bytes(2L), 2);
         rm.apply();
 
         IndexExpression expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), IndexExpression.Operator.EQ, ByteBufferUtil.bytes(1L));
         List<IndexExpression> clause = Arrays.asList(expr);
         IDiskAtomFilter filter = new IdentityQueryFilter();
         Range<RowPosition> range = Util.range("", "");
-        List<Row> rows = keyspace.getColumnFamilyStore("Indexed1").search(range, clause, filter, 100);
+        List<Row> rows = cfs.search(range, clause, filter, 100);
         assert rows.size() == 0;
 
         expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), IndexExpression.Operator.EQ, ByteBufferUtil.bytes(2L));
@@ -405,7 +404,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
 
         // update the birthdate value with an OLDER timestamp, and test that the index ignores this
         rm = new RowMutation("Keyspace2", ByteBufferUtil.bytes("k1"));
-        rm.add("Indexed1", ByteBufferUtil.bytes("birthdate"), ByteBufferUtil.bytes(3L), 0);
+        rm.add("Indexed1", birthdate, ByteBufferUtil.bytes(3L), 0);
         rm.apply();
 
         rows = keyspace.getColumnFamilyStore("Indexed1").search(range, clause, filter, 100);
@@ -425,7 +424,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         cfs.truncateBlocking();
 
         ByteBuffer rowKey = ByteBufferUtil.bytes("k1");
-        ByteBuffer colName = ByteBufferUtil.bytes("birthdate");
+        CellName colName = cellname("birthdate"); 
         ByteBuffer val1 = ByteBufferUtil.bytes(1L);
         ByteBuffer val2 = ByteBufferUtil.bytes(2L);
 
@@ -434,7 +433,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         rm = new RowMutation(keySpace, rowKey);
         rm.add(cfName, colName, val1, 0);
         rm.apply();
-        IndexExpression expr = new IndexExpression(colName, IndexExpression.Operator.EQ, val1);
+        IndexExpression expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), IndexExpression.Operator.EQ, val1);
         List<IndexExpression> clause = Arrays.asList(expr);
         IDiskAtomFilter filter = new IdentityQueryFilter();
         Range<RowPosition> range = Util.range("", "");
@@ -456,7 +455,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         rows = keyspace.getColumnFamilyStore(cfName).search(range, clause, filter, 100);
         assertEquals(0, rows.size());
         // now check for the updated value
-        expr = new IndexExpression(colName, IndexExpression.Operator.EQ, val2);
+        expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), IndexExpression.Operator.EQ, val2);
         clause = Arrays.asList(expr);
         filter = new IdentityQueryFilter();
         range = Util.range("", "");
@@ -469,7 +468,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         rm.add(cfName, colName, ByteBufferUtil.bytes(1L), 3);
         keyspace.apply(rm, true, false);
 
-        expr = new IndexExpression(colName, IndexExpression.Operator.EQ, ByteBufferUtil.bytes(1L));
+        expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), IndexExpression.Operator.EQ, ByteBufferUtil.bytes(1L));
         clause = Arrays.asList(expr);
         filter = new IdentityQueryFilter();
         range = Util.range("", "");
@@ -489,12 +488,10 @@ public class ColumnFamilyStoreTest extends SchemaLoader
 
         ByteBuffer rowKey = ByteBufferUtil.bytes("k1");
         ByteBuffer clusterKey = ByteBufferUtil.bytes("ck1");
-        ByteBuffer colName = ByteBufferUtil.bytes("col1");
-        CompositeType baseComparator = (CompositeType)cfs.getComparator();
-        CompositeType.Builder builder = baseComparator.builder();
-        builder.add(clusterKey);
-        builder.add(colName);
-        ByteBuffer compositeName = builder.build();
+        ByteBuffer colName = ByteBufferUtil.bytes("col1"); 
+
+        CellNameType baseComparator = cfs.getComparator();
+        CellName compositeName = baseComparator.makeCellName(clusterKey, colName);
 
         ByteBuffer val1 = ByteBufferUtil.bytes("v1");
         ByteBuffer val2 = ByteBufferUtil.bytes("v2");
@@ -565,11 +562,9 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         ByteBuffer rowKey = ByteBufferUtil.bytes("k1");
         ByteBuffer clusterKey = ByteBufferUtil.bytes("ck1");
         ByteBuffer colName = ByteBufferUtil.bytes("col1");
-        CompositeType baseComparator = (CompositeType)cfs.getComparator();
-        CompositeType.Builder builder = baseComparator.builder();
-        builder.add(clusterKey);
-        builder.add(colName);
-        ByteBuffer compositeName = builder.build();
+
+        CellNameType baseComparator = cfs.getComparator();
+        CellName compositeName = baseComparator.makeCellName(clusterKey, colName);
 
         ByteBuffer val1 = ByteBufferUtil.bytes("v2");
 
@@ -603,26 +598,30 @@ public class ColumnFamilyStoreTest extends SchemaLoader
     @Test
     public void testIndexScanWithLimitOne()
     {
+        ColumnFamilyStore cfs = Keyspace.open("Keyspace1").getColumnFamilyStore("Indexed1");
         RowMutation rm;
 
+        CellName nobirthdate = cellname("notbirthdate");
+        CellName birthdate = cellname("birthdate");
+
         rm = new RowMutation("Keyspace1", ByteBufferUtil.bytes("kk1"));
-        rm.add("Indexed1", ByteBufferUtil.bytes("notbirthdate"), ByteBufferUtil.bytes(1L), 0);
-        rm.add("Indexed1", ByteBufferUtil.bytes("birthdate"), ByteBufferUtil.bytes(1L), 0);
+        rm.add("Indexed1", nobirthdate, ByteBufferUtil.bytes(1L), 0);
+        rm.add("Indexed1", birthdate, ByteBufferUtil.bytes(1L), 0);
         rm.apply();
 
         rm = new RowMutation("Keyspace1", ByteBufferUtil.bytes("kk2"));
-        rm.add("Indexed1", ByteBufferUtil.bytes("notbirthdate"), ByteBufferUtil.bytes(2L), 0);
-        rm.add("Indexed1", ByteBufferUtil.bytes("birthdate"), ByteBufferUtil.bytes(1L), 0);
+        rm.add("Indexed1", nobirthdate, ByteBufferUtil.bytes(2L), 0);
+        rm.add("Indexed1", birthdate, ByteBufferUtil.bytes(1L), 0);
         rm.apply();
 
         rm = new RowMutation("Keyspace1", ByteBufferUtil.bytes("kk3"));
-        rm.add("Indexed1", ByteBufferUtil.bytes("notbirthdate"), ByteBufferUtil.bytes(2L), 0);
-        rm.add("Indexed1", ByteBufferUtil.bytes("birthdate"), ByteBufferUtil.bytes(1L), 0);
+        rm.add("Indexed1", nobirthdate, ByteBufferUtil.bytes(2L), 0);
+        rm.add("Indexed1", birthdate, ByteBufferUtil.bytes(1L), 0);
         rm.apply();
 
         rm = new RowMutation("Keyspace1", ByteBufferUtil.bytes("kk4"));
-        rm.add("Indexed1", ByteBufferUtil.bytes("notbirthdate"), ByteBufferUtil.bytes(2L), 0);
-        rm.add("Indexed1", ByteBufferUtil.bytes("birthdate"), ByteBufferUtil.bytes(1L), 0);
+        rm.add("Indexed1", nobirthdate, ByteBufferUtil.bytes(2L), 0);
+        rm.add("Indexed1", birthdate, ByteBufferUtil.bytes(1L), 0);
         rm.apply();
 
         // basic single-expression query
@@ -631,7 +630,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         List<IndexExpression> clause = Arrays.asList(expr1, expr2);
         IDiskAtomFilter filter = new IdentityQueryFilter();
         Range<RowPosition> range = Util.range("", "");
-        List<Row> rows = Keyspace.open("Keyspace1").getColumnFamilyStore("Indexed1").search(range, clause, filter, 1);
+        List<Row> rows = cfs.search(range, clause, filter, 1);
 
         assert rows != null;
         assert rows.size() == 1 : StringUtils.join(rows, ",");
@@ -641,14 +640,14 @@ public class ColumnFamilyStoreTest extends SchemaLoader
     public void testIndexCreate() throws IOException, InterruptedException, ExecutionException
     {
         Keyspace keyspace = Keyspace.open("Keyspace1");
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("Indexed2");
 
         // 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("k1"));
-        rm.add("Indexed2", ByteBufferUtil.bytes("birthdate"), ByteBufferUtil.bytes(1L), 1);
+        rm.add("Indexed2", cellname("birthdate"), ByteBufferUtil.bytes(1L), 1);
         rm.apply();
 
-        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("Indexed2");
         ColumnDefinition old = cfs.metadata.getColumnDefinition(ByteBufferUtil.bytes("birthdate"));
         ColumnDefinition cd = ColumnDefinition.regularDef(cfs.metadata, old.name.bytes, old.type, null).setIndex("birthdate_index", IndexType.KEYS, null);
         Future<?> future = cfs.indexManager.addIndexedColumn(cd);
@@ -686,7 +685,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
 
         List<Row> result = cfs.getRangeSlice(Util.bounds("key1", "key2"),
                                              null,
-                                             new NamesQueryFilter(ByteBufferUtil.bytes("asdf")),
+                                             Util.namesFilter(cfs, "asdf"),
                                              10);
         assertEquals(2, result.size());
         assert result.get(0).key.key.equals(ByteBufferUtil.bytes("key1"));
@@ -704,16 +703,16 @@ public class ColumnFamilyStoreTest extends SchemaLoader
 
         // create an isolated sstable.
         putColsSuper(cfs, key, scfName,
-                new Column(getBytes(1L), ByteBufferUtil.bytes("val1"), 1),
-                new Column(getBytes(2L), ByteBufferUtil.bytes("val2"), 1),
-                new Column(getBytes(3L), ByteBufferUtil.bytes("val3"), 1));
+                new Column(cellname(1L), ByteBufferUtil.bytes("val1"), 1),
+                new Column(cellname(2L), ByteBufferUtil.bytes("val2"), 1),
+                new Column(cellname(3L), ByteBufferUtil.bytes("val3"), 1));
         cfs.forceBlockingFlush();
 
         // insert, don't flush.
         putColsSuper(cfs, key, scfName,
-                new Column(getBytes(4L), ByteBufferUtil.bytes("val4"), 1),
-                new Column(getBytes(5L), ByteBufferUtil.bytes("val5"), 1),
-                new Column(getBytes(6L), ByteBufferUtil.bytes("val6"), 1));
+                new Column(cellname(4L), ByteBufferUtil.bytes("val4"), 1),
+                new Column(cellname(5L), ByteBufferUtil.bytes("val5"), 1),
+                new Column(cellname(6L), ByteBufferUtil.bytes("val6"), 1));
 
         // verify insert.
         final SlicePredicate sp = new SlicePredicate();
@@ -740,17 +739,17 @@ public class ColumnFamilyStoreTest extends SchemaLoader
 
         // late insert.
         putColsSuper(cfs, key, scfName,
-                new Column(getBytes(4L), ByteBufferUtil.bytes("val4"), 1L),
-                new Column(getBytes(7L), ByteBufferUtil.bytes("val7"), 1L));
+                new Column(cellname(4L), ByteBufferUtil.bytes("val4"), 1L),
+                new Column(cellname(7L), ByteBufferUtil.bytes("val7"), 1L));
 
         // re-verify delete.
         assertRowAndColCount(1, 0, false, cfs.getRangeSlice(Util.range("f", "g"), null, ThriftValidation.asIFilter(sp, cfs.metadata, scfName), 100));
 
         // make sure new writes are recognized.
         putColsSuper(cfs, key, scfName,
-                new Column(getBytes(3L), ByteBufferUtil.bytes("val3"), 3),
-                new Column(getBytes(8L), ByteBufferUtil.bytes("val8"), 3),
-                new Column(getBytes(9L), ByteBufferUtil.bytes("val9"), 3));
+                new Column(cellname(3L), ByteBufferUtil.bytes("val3"), 3),
+                new Column(cellname(8L), ByteBufferUtil.bytes("val8"), 3),
+                new Column(cellname(9L), ByteBufferUtil.bytes("val9"), 3));
         assertRowAndColCount(1, 3, false, cfs.getRangeSlice(Util.range("f", "g"), null, ThriftValidation.asIFilter(sp, cfs.metadata, scfName), 100));
     }
 
@@ -770,7 +769,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
     {
         StringBuilder sb = new StringBuilder();
         for (Column col : cf.getSortedColumns())
-            sb.append(String.format("(%s,%s,%d),", ByteBufferUtil.string(col.name()), ByteBufferUtil.string(col.value()), col.timestamp()));
+            sb.append(String.format("(%s,%s,%d),", ByteBufferUtil.string(col.name().toByteBuffer()), ByteBufferUtil.string(col.value()), col.timestamp()));
         return sb.toString();
     }
 
@@ -778,7 +777,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
     {
         ColumnFamily cf = TreeMapBackedSortedColumns.factory.create(cfs.keyspace.getName(), cfs.name);
         for (Column col : cols)
-            cf.addColumn(col.withUpdatedName(CompositeType.build(scfName, col.name())));
+            cf.addColumn(col.withUpdatedName(CellNames.compositeDense(scfName, col.name().toByteBuffer())));
         RowMutation rm = new RowMutation(cfs.keyspace.getName(), key.key, cf);
         rm.apply();
     }
@@ -851,15 +850,16 @@ public class ColumnFamilyStoreTest extends SchemaLoader
 
     private ColumnFamilyStore insertKey1Key2() throws IOException, ExecutionException, InterruptedException
     {
+        ColumnFamilyStore cfs = Keyspace.open("Keyspace2").getColumnFamilyStore("Standard1");
         List<IMutation> rms = new LinkedList<IMutation>();
         RowMutation rm;
         rm = new RowMutation("Keyspace2", ByteBufferUtil.bytes("key1"));
-        rm.add("Standard1", ByteBufferUtil.bytes("Column1"), ByteBufferUtil.bytes("asdf"), 0);
+        rm.add("Standard1", cellname("Column1"), ByteBufferUtil.bytes("asdf"), 0);
         rms.add(rm);
         Util.writeColumnFamily(rms);
 
         rm = new RowMutation("Keyspace2", ByteBufferUtil.bytes("key2"));
-        rm.add("Standard1", ByteBufferUtil.bytes("Column1"), ByteBufferUtil.bytes("asdf"), 0);
+        rm.add("Standard1", cellname("Column1"), ByteBufferUtil.bytes("asdf"), 0);
         rms.add(rm);
         return Util.writeColumnFamily(rms);
     }
@@ -890,24 +890,24 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         DecoratedKey key = Util.dk("slice-get-uuid-type");
 
         // Insert a row with one supercolumn and multiple subcolumns
-        putColsSuper(cfs, key, superColName, new Column(ByteBufferUtil.bytes("a"), ByteBufferUtil.bytes("A"), 1),
-                                             new Column(ByteBufferUtil.bytes("b"), ByteBufferUtil.bytes("B"), 1));
+        putColsSuper(cfs, key, superColName, new Column(cellname("a"), ByteBufferUtil.bytes("A"), 1),
+                                             new Column(cellname("b"), ByteBufferUtil.bytes("B"), 1));
 
         // Get the entire supercolumn like normal
         ColumnFamily cfGet = cfs.getColumnFamily(QueryFilter.getIdentityFilter(key, cfName, System.currentTimeMillis()));
-        assertEquals(ByteBufferUtil.bytes("A"), cfGet.getColumn(CompositeType.build(superColName, ByteBufferUtil.bytes("a"))).value());
-        assertEquals(ByteBufferUtil.bytes("B"), cfGet.getColumn(CompositeType.build(superColName, ByteBufferUtil.bytes("b"))).value());
+        assertEquals(ByteBufferUtil.bytes("A"), cfGet.getColumn(CellNames.compositeDense(superColName, ByteBufferUtil.bytes("a"))).value());
+        assertEquals(ByteBufferUtil.bytes("B"), cfGet.getColumn(CellNames.compositeDense(superColName, ByteBufferUtil.bytes("b"))).value());
 
         // Now do the SliceByNamesCommand on the supercolumn, passing both subcolumns in as columns to get
-        SortedSet<ByteBuffer> sliceColNames = new TreeSet<ByteBuffer>(cfs.metadata.comparator);
-        sliceColNames.add(CompositeType.build(superColName, ByteBufferUtil.bytes("a")));
-        sliceColNames.add(CompositeType.build(superColName, ByteBufferUtil.bytes("b")));
+        SortedSet<CellName> sliceColNames = new TreeSet<CellName>(cfs.metadata.comparator);
+        sliceColNames.add(CellNames.compositeDense(superColName, ByteBufferUtil.bytes("a")));
+        sliceColNames.add(CellNames.compositeDense(superColName, ByteBufferUtil.bytes("b")));
         SliceByNamesReadCommand cmd = new SliceByNamesReadCommand(keyspaceName, key.key, cfName, System.currentTimeMillis(), new NamesQueryFilter(sliceColNames));
         ColumnFamily cfSliced = cmd.getRow(keyspace).cf;
 
         // Make sure the slice returns the same as the straight get
-        assertEquals(ByteBufferUtil.bytes("A"), cfSliced.getColumn(CompositeType.build(superColName, ByteBufferUtil.bytes("a"))).value());
-        assertEquals(ByteBufferUtil.bytes("B"), cfSliced.getColumn(CompositeType.build(superColName, ByteBufferUtil.bytes("b"))).value());
+        assertEquals(ByteBufferUtil.bytes("A"), cfSliced.getColumn(CellNames.compositeDense(superColName, ByteBufferUtil.bytes("a"))).value());
+        assertEquals(ByteBufferUtil.bytes("B"), cfSliced.getColumn(CellNames.compositeDense(superColName, ByteBufferUtil.bytes("b"))).value());
     }
 
     @Test
@@ -916,7 +916,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         String keyspaceName = "Keyspace1";
         String cfName= "Standard1";
         DecoratedKey key = Util.dk("slice-name-old-metadata");
-        ByteBuffer cname = ByteBufferUtil.bytes("c1");
+        CellName cname = cellname("c1");
         Keyspace keyspace = Keyspace.open(keyspaceName);
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfName);
         cfs.clearUnsafe();
@@ -938,7 +938,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         putColsStandard(cfs, key, new Column(cname, ByteBufferUtil.bytes("b"), 1));
 
         // Test fetching the column by name returns the first column
-        SliceByNamesReadCommand cmd = new SliceByNamesReadCommand(keyspaceName, key.key, cfName, System.currentTimeMillis(), new NamesQueryFilter(cname));
+        SliceByNamesReadCommand cmd = new SliceByNamesReadCommand(keyspaceName, key.key, cfName, System.currentTimeMillis(), new NamesQueryFilter(FBUtilities.singleton(cname, cfs.getComparator())));
         ColumnFamily cf = cmd.getRow(keyspace).cf;
         Column column = cf.getColumn(cname);
         assert column.value().equals(ByteBufferUtil.bytes("a")) : "expecting a, got " + ByteBufferUtil.string(column.value());
@@ -1138,11 +1138,11 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         assertColumnNames(row2, "c0", "c1");
 
         // Paging within bounds
-        SliceQueryFilter sf = new SliceQueryFilter(ByteBufferUtil.bytes("c1"),
-                                                   ByteBufferUtil.bytes("c2"),
+        SliceQueryFilter sf = new SliceQueryFilter(cellname("c1"),
+                                                   cellname("c2"),
                                                    false,
                                                    0);
-        rows = cfs.getRangeSlice(cfs.makeExtendedFilter(new Bounds<RowPosition>(ka, kc), sf, ByteBufferUtil.bytes("c2"), ByteBufferUtil.bytes("c1"), null, 2, System.currentTimeMillis()));
+        rows = cfs.getRangeSlice(cfs.makeExtendedFilter(new Bounds<RowPosition>(ka, kc), sf, cellname("c2"), cellname("c1"), null, 2, System.currentTimeMillis()));
         assert rows.size() == 2 : "Expected 2 rows, got " + toString(rows);
         iter = rows.iterator();
         row1 = iter.next();
@@ -1150,7 +1150,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         assertColumnNames(row1, "c2");
         assertColumnNames(row2, "c1");
 
-        rows = cfs.getRangeSlice(cfs.makeExtendedFilter(new Bounds<RowPosition>(kb, kc), sf, ByteBufferUtil.bytes("c1"), ByteBufferUtil.bytes("c1"), null, 10, System.currentTimeMillis()));
+        rows = cfs.getRangeSlice(cfs.makeExtendedFilter(new Bounds<RowPosition>(kb, kc), sf, cellname("c1"), cellname("c1"), null, 10, System.currentTimeMillis()));
         assert rows.size() == 2 : "Expected 2 rows, got " + toString(rows);
         iter = rows.iterator();
         row1 = iter.next();
@@ -1172,7 +1172,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
                 if (row.cf != null && !row.cf.isEmpty())
                 {
                     for (Column c : row.cf)
-                        sb.append(" ").append(ByteBufferUtil.string(c.name()));
+                        sb.append(" ").append(row.cf.getComparator().getString(c.name()));
                 }
                 sb.append("} ");
             }
@@ -1195,9 +1195,9 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         while (columns.hasNext())
         {
             Column c = columns.next();
-            assert names.hasNext() : "Got more columns that expected (first unexpected column: " + ByteBufferUtil.string(c.name()) + ")";
+            assert names.hasNext() : "Got more columns that expected (first unexpected column: " + ByteBufferUtil.string(c.name().toByteBuffer()) + ")";
             String n = names.next();
-            assert c.name().equals(ByteBufferUtil.bytes(n)) : "Expected " + n + ", got " + ByteBufferUtil.string(c.name());
+            assert c.name().toByteBuffer().equals(ByteBufferUtil.bytes(n)) : "Expected " + n + ", got " + ByteBufferUtil.string(c.name().toByteBuffer());
         }
         assert !names.hasNext() : "Missing expected column " + names.next();
     }
@@ -1273,7 +1273,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         {
             ByteBuffer key = ByteBufferUtil.bytes(String.valueOf("k" + i));
             RowMutation rm = new RowMutation("Keyspace1", key);
-            rm.add("Indexed1", ByteBufferUtil.bytes("birthdate"), LongType.instance.decompose(1L), System.currentTimeMillis());
+            rm.add("Indexed1", cellname("birthdate"), LongType.instance.decompose(1L), System.currentTimeMillis());
             rm.apply();
         }
 
@@ -1293,18 +1293,18 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         // in order not to change thrift interfaces at this stage we build SliceQueryFilter
         // directly instead of using QueryFilter to build it for us
         ColumnSlice[] ranges = new ColumnSlice[] {
-                new ColumnSlice(ByteBuffer.wrap(EMPTY_BYTE_ARRAY), bytes("colA")),
-                new ColumnSlice(bytes("colC"), bytes("colE")),
-                new ColumnSlice(bytes("colF"), bytes("colF")),
-                new ColumnSlice(bytes("colG"), bytes("colG")),
-                new ColumnSlice(bytes("colI"), ByteBuffer.wrap(EMPTY_BYTE_ARRAY)) };
+                new ColumnSlice(Composites.EMPTY, cellname("colA")),
+                new ColumnSlice(cellname("colC"), cellname("colE")),
+                new ColumnSlice(cellname("colF"), cellname("colF")),
+                new ColumnSlice(cellname("colG"), cellname("colG")),
+                new ColumnSlice(cellname("colI"), Composites.EMPTY) };
 
         ColumnSlice[] rangesReversed = new ColumnSlice[] {
-                new ColumnSlice(ByteBuffer.wrap(EMPTY_BYTE_ARRAY), bytes("colI")),
-                new ColumnSlice(bytes("colG"), bytes("colG")),
-                new ColumnSlice(bytes("colF"), bytes("colF")),
-                new ColumnSlice(bytes("colE"), bytes("colC")),
-                new ColumnSlice(bytes("colA"), ByteBuffer.wrap(EMPTY_BYTE_ARRAY)) };
+                new ColumnSlice(Composites.EMPTY, cellname("colI")),
+                new ColumnSlice(cellname("colG"), cellname("colG")),
+                new ColumnSlice(cellname("colF"), cellname("colF")),
+                new ColumnSlice(cellname("colE"), cellname("colC")),
+                new ColumnSlice(cellname("colA"), Composites.EMPTY) };
 
         String tableName = "Keyspace1";
         String cfName = "Standard1";
@@ -1316,7 +1316,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         Column[] cols = new Column[letters.length];
         for (int i = 0; i < cols.length; i++)
         {
-            cols[i] = new Column(ByteBufferUtil.bytes("col" + letters[i].toUpperCase()),
+            cols[i] = new Column(cellname("col" + letters[i].toUpperCase()),
                     ByteBuffer.wrap(new byte[1]), 1);
         }
 
@@ -1342,18 +1342,18 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         // in order not to change thrift interfaces at this stage we build SliceQueryFilter
         // directly instead of using QueryFilter to build it for us
         ColumnSlice[] ranges = new ColumnSlice[] {
-                new ColumnSlice(ByteBuffer.wrap(EMPTY_BYTE_ARRAY), bytes("colA")),
-                new ColumnSlice(bytes("colC"), bytes("colE")),
-                new ColumnSlice(bytes("colF"), bytes("colF")),
-                new ColumnSlice(bytes("colG"), bytes("colG")),
-                new ColumnSlice(bytes("colI"), ByteBuffer.wrap(EMPTY_BYTE_ARRAY)) };
+                new ColumnSlice(Composites.EMPTY, cellname("colA")),
+                new ColumnSlice(cellname("colC"), cellname("colE")),
+                new ColumnSlice(cellname("colF"), cellname("colF")),
+                new ColumnSlice(cellname("colG"), cellname("colG")),
+                new ColumnSlice(cellname("colI"), Composites.EMPTY) };
 
         ColumnSlice[] rangesReversed = new ColumnSlice[] {
-                new ColumnSlice(ByteBuffer.wrap(EMPTY_BYTE_ARRAY), bytes("colI")),
-                new ColumnSlice(bytes("colG"), bytes("colG")),
-                new ColumnSlice(bytes("colF"), bytes("colF")),
-                new ColumnSlice(bytes("colE"), bytes("colC")),
-                new ColumnSlice(bytes("colA"), ByteBuffer.wrap(EMPTY_BYTE_ARRAY)) };
+                new ColumnSlice(Composites.EMPTY,  cellname("colI")),
+                new ColumnSlice(cellname("colG"), cellname("colG")),
+                new ColumnSlice(cellname("colF"), cellname("colF")),
+                new ColumnSlice(cellname("colE"), cellname("colC")),
+                new ColumnSlice(cellname("colA"), Composites.EMPTY) };
 
         String tableName = "Keyspace1";
         String cfName = "Standard1";
@@ -1365,7 +1365,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         Column[] cols = new Column[letters.length];
         for (int i = 0; i < cols.length; i++)
         {
-            cols[i] = new Column(ByteBufferUtil.bytes("col" + letters[i].toUpperCase()),
+            cols[i] = new Column(cellname("col" + letters[i].toUpperCase()),
                     ByteBuffer.wrap(new byte[1366]), 1);
         }
 
@@ -1391,18 +1391,18 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         // in order not to change thrift interfaces at this stage we build SliceQueryFilter
         // directly instead of using QueryFilter to build it for us
         ColumnSlice[] ranges = new ColumnSlice[] {
-                new ColumnSlice(ByteBuffer.wrap(EMPTY_BYTE_ARRAY), bytes("colA")),
-                new ColumnSlice(bytes("colC"), bytes("colE")),
-                new ColumnSlice(bytes("colF"), bytes("colF")),
-                new ColumnSlice(bytes("colG"), bytes("colG")),
-                new ColumnSlice(bytes("colI"), ByteBuffer.wrap(EMPTY_BYTE_ARRAY)) };
+                new ColumnSlice(Composites.EMPTY, cellname("colA")),
+                new ColumnSlice(cellname("colC"), cellname("colE")),
+                new ColumnSlice(cellname("colF"), cellname("colF")),
+                new ColumnSlice(cellname("colG"), cellname("colG")),
+                new ColumnSlice(cellname("colI"), Composites.EMPTY) };
 
         ColumnSlice[] rangesReversed = new ColumnSlice[] {
-                new ColumnSlice(ByteBuffer.wrap(EMPTY_BYTE_ARRAY), bytes("colI")),
-                new ColumnSlice(bytes("colG"), bytes("colG")),
-                new ColumnSlice(bytes("colF"), bytes("colF")),
-                new ColumnSlice(bytes("colE"), bytes("colC")),
-                new ColumnSlice(bytes("colA"), ByteBuffer.wrap(EMPTY_BYTE_ARRAY)) };
+                new ColumnSlice(Composites.EMPTY, cellname("colI")),
+                new ColumnSlice(cellname("colG"), cellname("colG")),
+                new ColumnSlice(cellname("colF"), cellname("colF")),
+                new ColumnSlice(cellname("colE"), cellname("colC")),
+                new ColumnSlice(cellname("colA"), Composites.EMPTY) };
 
         String tableName = "Keyspace1";
         String cfName = "Standard1";
@@ -1414,7 +1414,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         Column[] cols = new Column[letters.length];
         for (int i = 0; i < cols.length; i++)
         {
-            cols[i] = new Column(ByteBufferUtil.bytes("col" + letters[i].toUpperCase()),
+            cols[i] = new Column(cellname("col" + letters[i].toUpperCase()),
                     ByteBuffer.wrap(new byte[1]), 1);
         }
 
@@ -1441,18 +1441,18 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         // in order not to change thrift interfaces at this stage we build SliceQueryFilter
         // directly instead of using QueryFilter to build it for us
         ColumnSlice[] ranges = new ColumnSlice[] {
-                new ColumnSlice(ByteBuffer.wrap(EMPTY_BYTE_ARRAY), bytes("colA")),
-                new ColumnSlice(bytes("colC"), bytes("colE")),
-                new ColumnSlice(bytes("colF"), bytes("colF")),
-                new ColumnSlice(bytes("colG"), bytes("colG")),
-                new ColumnSlice(bytes("colI"), ByteBuffer.wrap(EMPTY_BYTE_ARRAY)) };
+                new ColumnSlice(Composites.EMPTY, cellname("colA")),
+                new ColumnSlice(cellname("colC"), cellname("colE")),
+                new ColumnSlice(cellname("colF"), cellname("colF")),
+                new ColumnSlice(cellname("colG"), cellname("colG")),
+                new ColumnSlice(cellname("colI"), Composites.EMPTY) };
 
         ColumnSlice[] rangesReversed = new ColumnSlice[] {
-                new ColumnSlice(ByteBuffer.wrap(EMPTY_BYTE_ARRAY), bytes("colI")),
-                new ColumnSlice(bytes("colG"), bytes("colG")),
-                new ColumnSlice(bytes("colF"), bytes("colF")),
-                new ColumnSlice(bytes("colE"), bytes("colC")),
-                new ColumnSlice(bytes("colA"), ByteBuffer.wrap(EMPTY_BYTE_ARRAY)) };
+                new ColumnSlice(Composites.EMPTY, cellname("colI")),
+                new ColumnSlice(cellname("colG"), cellname("colG")),
+                new ColumnSlice(cellname("colF"), cellname("colF")),
+                new ColumnSlice(cellname("colE"), cellname("colC")),
+                new ColumnSlice(cellname("colA"), Composites.EMPTY) };
 
         String tableName = "Keyspace1";
         String cfName = "Standard1";
@@ -1464,7 +1464,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         Column[] cols = new Column[letters.length];
         for (int i = 0; i < cols.length; i++)
         {
-            cols[i] = new Column(ByteBufferUtil.bytes("col" + letters[i].toUpperCase()),
+            cols[i] = new Column(cellname("col" + letters[i].toUpperCase()),
                     ByteBuffer.wrap(new byte[1366]), 1);
         }
 
@@ -1491,16 +1491,16 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         // in order not to change thrift interfaces at this stage we build SliceQueryFilter
         // directly instead of using QueryFilter to build it for us
         ColumnSlice[] ranges = new ColumnSlice[] {
-                new ColumnSlice(ByteBuffer.wrap(EMPTY_BYTE_ARRAY), bytes("colA")),
-                new ColumnSlice(bytes("colC"), bytes("colE")),
-                new ColumnSlice(bytes("colG"), bytes("colG")),
-                new ColumnSlice(bytes("colI"), ByteBuffer.wrap(EMPTY_BYTE_ARRAY)) };
+                new ColumnSlice(Composites.EMPTY, cellname("colA")),
+                new ColumnSlice(cellname("colC"), cellname("colE")),
+                new ColumnSlice(cellname("colG"), cellname("colG")),
+                new ColumnSlice(cellname("colI"), Composites.EMPTY) };
 
         ColumnSlice[] rangesReversed = new ColumnSlice[] {
-                new ColumnSlice(ByteBuffer.wrap(EMPTY_BYTE_ARRAY), bytes("colI")),
-                new ColumnSlice(bytes("colG"), bytes("colG")),
-                new ColumnSlice(bytes("colE"), bytes("colC")),
-                new ColumnSlice(bytes("colA"), ByteBuffer.wrap(EMPTY_BYTE_ARRAY)) };
+                new ColumnSlice(Composites.EMPTY, cellname("colI")),
+                new ColumnSlice(cellname("colG"), cellname("colG")),
+                new ColumnSlice(cellname("colE"), cellname("colC")),
+                new ColumnSlice(cellname("colA"), Composites.EMPTY) };
 
         String keyspaceName = "Keyspace1";
         String cfName = "Standard1";
@@ -1512,7 +1512,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         Column[] cols = new Column[letters.length];
         for (int i = 0; i < cols.length; i++)
         {
-            cols[i] = new Column(ByteBufferUtil.bytes("col" + letters[i].toUpperCase()),
+            cols[i] = new Column(cellname("col" + letters[i].toUpperCase()),
                     // use 1366 so that three cols make an index segment
                     ByteBuffer.wrap(new byte[1366]), 1);
         }
@@ -1631,7 +1631,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         Column[] cols = new Column[12];
         for (int i = 0; i < cols.length; i++)
         {
-            cols[i] = new Column(ByteBufferUtil.bytes("col" + letters[i]), ByteBuffer.wrap(new byte[valueSize]), 1);
+            cols[i] = new Column(cellname("col" + letters[i]), ByteBuffer.wrap(new byte[valueSize]), 1);
         }
 
         for (int i = 0; i < 12; i++)
@@ -1654,35 +1654,37 @@ public class ColumnFamilyStoreTest extends SchemaLoader
 
     private void testMultiRangeSlicesBehavior(ColumnFamilyStore cfs)
     {
+        CellNameType type = cfs.getComparator();
+
         // in order not to change thrift interfaces at this stage we build SliceQueryFilter
         // directly instead of using QueryFilter to build it for us
         ColumnSlice[] startMiddleAndEndRanges = new ColumnSlice[] {
-                new ColumnSlice(ByteBuffer.wrap(EMPTY_BYTE_ARRAY), bytes("colc")),
-                new ColumnSlice(bytes("colf"), bytes("colg")),
-                new ColumnSlice(bytes("colj"), ByteBuffer.wrap(EMPTY_BYTE_ARRAY)) };
+                new ColumnSlice(Composites.EMPTY, cellname("colc")),
+                new ColumnSlice(cellname("colf"), cellname("colg")),
+                new ColumnSlice(cellname("colj"), Composites.EMPTY) };
 
         ColumnSlice[] startMiddleAndEndRangesReversed = new ColumnSlice[] {
-                new ColumnSlice(ByteBuffer.wrap(EMPTY_BYTE_ARRAY), bytes("colj")),
-                new ColumnSlice(bytes("colg"), bytes("colf")),
-                new ColumnSlice(bytes("colc"), ByteBuffer.wrap(EMPTY_BYTE_ARRAY)) };
+                new ColumnSlice(Composites.EMPTY, cellname("colj")),
+                new ColumnSlice(cellname("colg"), cellname("colf")),
+                new ColumnSlice(cellname("colc"), Composites.EMPTY) };
 
         ColumnSlice[] startOnlyRange =
-                new ColumnSlice[] { new ColumnSlice(ByteBuffer.wrap(EMPTY_BYTE_ARRAY), bytes("colc")) };
+                new ColumnSlice[] { new ColumnSlice(Composites.EMPTY, cellname("colc")) };
 
         ColumnSlice[] startOnlyRangeReversed =
-                new ColumnSlice[] { new ColumnSlice(bytes("colc"), ByteBuffer.wrap(EMPTY_BYTE_ARRAY)) };
+                new ColumnSlice[] { new ColumnSlice(cellname("colc"), Composites.EMPTY) };
 
         ColumnSlice[] middleOnlyRanges =
-                new ColumnSlice[] { new ColumnSlice(bytes("colf"), bytes("colg")) };
+                new ColumnSlice[] { new ColumnSlice(cellname("colf"), cellname("colg")) };
 
         ColumnSlice[] middleOnlyRangesReversed =
-                new ColumnSlice[] { new ColumnSlice(bytes("colg"), bytes("colf")) };
+                new ColumnSlice[] { new ColumnSlice(cellname("colg"), cellname("colf")) };
 
         ColumnSlice[] endOnlyRanges =
-                new ColumnSlice[] { new ColumnSlice(bytes("colj"), ByteBuffer.wrap(EMPTY_BYTE_ARRAY)) };
+                new ColumnSlice[] { new ColumnSlice(cellname("colj"), Composites.EMPTY) };
 
         ColumnSlice[] endOnlyRangesReversed =
-                new ColumnSlice[] { new ColumnSlice(ByteBuffer.wrap(EMPTY_BYTE_ARRAY), bytes("colj")) };
+                new ColumnSlice[] { new ColumnSlice(Composites.EMPTY, cellname("colj")) };
 
         SliceQueryFilter startOnlyFilter = new SliceQueryFilter(startOnlyRange, false,
                 Integer.MAX_VALUE);
@@ -1867,7 +1869,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         {
             public String apply(Column arg0)
             {
-                return new String(arg0.name().array());
+                return Util.string(arg0.name().toByteBuffer());
             }
         }), String.class);
 
@@ -1877,7 +1879,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         int i = 0;
         for (Column col : cols)
         {
-            assertEquals(colNames[i++], new String(col.name().array()));
+            assertEquals(colNames[i++], Util.string(col.name().toByteBuffer()));
         }
     }
 
@@ -1891,7 +1893,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
                 {
                     public String apply(Column arg0)
                     {
-                        return new String(arg0.name().array());
+                        return Util.string(arg0.name().toByteBuffer());
                     }
                 }));
         System.err.println("Filtered:");
@@ -1899,7 +1901,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         {
             public String apply(Column arg0)
             {
-                return new String(arg0.name().array());
+                return Util.string(arg0.name().toByteBuffer());
             }
         });
         System.err.println("Row key: " + rowKey + " Cols: " + transformed);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/362cc053/test/unit/org/apache/cassandra/db/ColumnFamilyTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/ColumnFamilyTest.java b/test/unit/org/apache/cassandra/db/ColumnFamilyTest.java
index a01c25c..42f6066 100644
--- a/test/unit/org/apache/cassandra/db/ColumnFamilyTest.java
+++ b/test/unit/org/apache/cassandra/db/ColumnFamilyTest.java
@@ -33,6 +33,7 @@ import org.apache.cassandra.io.sstable.ColumnStats;
 import org.apache.cassandra.io.util.DataOutputBuffer;
 import org.apache.cassandra.net.MessagingService;
 import static org.apache.cassandra.Util.column;
+import static org.apache.cassandra.Util.cellname;
 import static org.junit.Assert.assertEquals;
 
 import org.apache.cassandra.utils.ByteBufferUtil;
@@ -87,7 +88,7 @@ public class ColumnFamilyTest extends SchemaLoader
         cf = ColumnFamily.serializer.deserialize(new DataInputStream(bufIn), version);
         for (String cName : map.navigableKeySet())
         {
-            ByteBuffer val = cf.getColumn(ByteBufferUtil.bytes(cName)).value();
+            ByteBuffer val = cf.getColumn(cellname(cName)).value();
             assert new String(val.array(),val.position(),val.remaining()).equals(map.get(cName));
         }
         assert Iterables.size(cf.getColumnNames()) == map.size();
@@ -115,7 +116,7 @@ public class ColumnFamilyTest extends SchemaLoader
         cf.addColumn(column("col1", "val2", 2)); // same timestamp, new value
         cf.addColumn(column("col1", "val3", 1)); // older timestamp -- should be ignored
 
-        assert ByteBufferUtil.bytes("val2").equals(cf.getColumn(ByteBufferUtil.bytes("col1")).value());
+        assert ByteBufferUtil.bytes("val2").equals(cf.getColumn(cellname("col1")).value());
     }
 
     @Test
@@ -127,30 +128,30 @@ public class ColumnFamilyTest extends SchemaLoader
         ByteBuffer val = ByteBufferUtil.bytes("sample value");
         ByteBuffer val2 = ByteBufferUtil.bytes("x value ");
 
-        cf_new.addColumn(ByteBufferUtil.bytes("col1"), val, 3);
-        cf_new.addColumn(ByteBufferUtil.bytes("col2"), val, 4);
+        cf_new.addColumn(cellname("col1"), val, 3);
+        cf_new.addColumn(cellname("col2"), val, 4);
 
-        cf_old.addColumn(ByteBufferUtil.bytes("col2"), val2, 1);
-        cf_old.addColumn(ByteBufferUtil.bytes("col3"), val2, 2);
+        cf_old.addColumn(cellname("col2"), val2, 1);
+        cf_old.addColumn(cellname("col3"), val2, 2);
 
         cf_result.addAll(cf_new, HeapAllocator.instance);
         cf_result.addAll(cf_old, HeapAllocator.instance);
 
         assert 3 == cf_result.getColumnCount() : "Count is " + cf_new.getColumnCount();
         //addcolumns will only add if timestamp >= old timestamp
-        assert val.equals(cf_result.getColumn(ByteBufferUtil.bytes("col2")).value());
+        assert val.equals(cf_result.getColumn(cellname("col2")).value());
 
         // check that tombstone wins timestamp ties
-        cf_result.addTombstone(ByteBufferUtil.bytes("col1"), 0, 3);
-        assert cf_result.getColumn(ByteBufferUtil.bytes("col1")).isMarkedForDelete(System.currentTimeMillis());
-        cf_result.addColumn(ByteBufferUtil.bytes("col1"), val2, 3);
-        assert cf_result.getColumn(ByteBufferUtil.bytes("col1")).isMarkedForDelete(System.currentTimeMillis());
+        cf_result.addTombstone(cellname("col1"), 0, 3);
+        assert cf_result.getColumn(cellname("col1")).isMarkedForDelete(System.currentTimeMillis());
+        cf_result.addColumn(cellname("col1"), val2, 3);
+        assert cf_result.getColumn(cellname("col1")).isMarkedForDelete(System.currentTimeMillis());
 
         // check that column value wins timestamp ties in absence of tombstone
-        cf_result.addColumn(ByteBufferUtil.bytes("col3"), val, 2);
-        assert cf_result.getColumn(ByteBufferUtil.bytes("col3")).value().equals(val2);
-        cf_result.addColumn(ByteBufferUtil.bytes("col3"), ByteBufferUtil.bytes("z"), 2);
-        assert cf_result.getColumn(ByteBufferUtil.bytes("col3")).value().equals(ByteBufferUtil.bytes("z"));
+        cf_result.addColumn(cellname("col3"), val, 2);
+        assert cf_result.getColumn(cellname("col3")).value().equals(val2);
+        cf_result.addColumn(cellname("col3"), ByteBufferUtil.bytes("z"), 2);
+        assert cf_result.getColumn(cellname("col3")).value().equals(ByteBufferUtil.bytes("z"));
     }
 
     @Test

http://git-wip-us.apache.org/repos/asf/cassandra/blob/362cc053/test/unit/org/apache/cassandra/db/CommitLogTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/CommitLogTest.java b/test/unit/org/apache/cassandra/db/CommitLogTest.java
index 6c1b56b..587e731 100644
--- a/test/unit/org/apache/cassandra/db/CommitLogTest.java
+++ b/test/unit/org/apache/cassandra/db/CommitLogTest.java
@@ -29,6 +29,7 @@ import org.junit.Assert;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.Util;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.commitlog.CommitLog;
 import org.apache.cassandra.db.commitlog.CommitLogDescriptor;
@@ -99,7 +100,7 @@ public class CommitLogTest extends SchemaLoader
         CommitLog.instance.resetUnsafe();
         // Roughly 32 MB mutation
         RowMutation rm = new RowMutation("Keyspace1", bytes("k"));
-        rm.add("Standard1", bytes("c1"), ByteBuffer.allocate(DatabaseDescriptor.getCommitLogSegmentSize()/4), 0);
+        rm.add("Standard1", Util.cellname("c1"), ByteBuffer.allocate(DatabaseDescriptor.getCommitLogSegmentSize()/4), 0);
 
         // Adding it 5 times
         CommitLog.instance.add(rm);
@@ -110,7 +111,7 @@ public class CommitLogTest extends SchemaLoader
 
         // Adding new mutation on another CF
         RowMutation rm2 = new RowMutation("Keyspace1", bytes("k"));
-        rm2.add("Standard2", bytes("c1"), ByteBuffer.allocate(4), 0);
+        rm2.add("Standard2", Util.cellname("c1"), ByteBuffer.allocate(4), 0);
         CommitLog.instance.add(rm2);
 
         assert CommitLog.instance.activeSegments() == 2 : "Expecting 2 segments, got " + CommitLog.instance.activeSegments();
@@ -129,7 +130,7 @@ public class CommitLogTest extends SchemaLoader
         CommitLog.instance.resetUnsafe();
         // Roughly 32 MB mutation
         RowMutation rm = new RowMutation("Keyspace1", bytes("k"));
-        rm.add("Standard1", bytes("c1"), ByteBuffer.allocate((DatabaseDescriptor.getCommitLogSegmentSize()/4) - 1), 0);
+        rm.add("Standard1", Util.cellname("c1"), ByteBuffer.allocate((DatabaseDescriptor.getCommitLogSegmentSize()/4) - 1), 0);
 
         // Adding it twice (won't change segment)
         CommitLog.instance.add(rm);
@@ -146,7 +147,7 @@ public class CommitLogTest extends SchemaLoader
 
         // Adding new mutation on another CF, large enough (including CL entry overhead) that a new segment is created
         RowMutation rm2 = new RowMutation("Keyspace1", bytes("k"));
-        rm2.add("Standard2", bytes("c1"), ByteBuffer.allocate((DatabaseDescriptor.getCommitLogSegmentSize()/2) - 100), 0);
+        rm2.add("Standard2", Util.cellname("c1"), ByteBuffer.allocate((DatabaseDescriptor.getCommitLogSegmentSize()/2) - 100), 0);
         CommitLog.instance.add(rm2);
         // also forces a new segment, since each entry-with-overhead is just under half the CL size
         CommitLog.instance.add(rm2);
@@ -172,7 +173,7 @@ public class CommitLogTest extends SchemaLoader
         CommitLog.instance.resetUnsafe();
 
         RowMutation rm = new RowMutation("Keyspace1", bytes("k"));
-        rm.add("Standard1", bytes("c1"), ByteBuffer.allocate((DatabaseDescriptor.getCommitLogSegmentSize()) - 83), 0);
+        rm.add("Standard1", Util.cellname("c1"), ByteBuffer.allocate((DatabaseDescriptor.getCommitLogSegmentSize()) - 83), 0);
         CommitLog.instance.add(rm);
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/362cc053/test/unit/org/apache/cassandra/db/CounterColumnTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/CounterColumnTest.java b/test/unit/org/apache/cassandra/db/CounterColumnTest.java
index a2add18..0ca2636 100644
--- a/test/unit/org/apache/cassandra/db/CounterColumnTest.java
+++ b/test/unit/org/apache/cassandra/db/CounterColumnTest.java
@@ -31,7 +31,9 @@ import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
+import org.apache.cassandra.db.composites.*;
 import org.apache.cassandra.db.context.CounterContext;
+import org.apache.cassandra.db.marshal.UTF8Type;
 import static org.apache.cassandra.db.context.CounterContext.ContextState;
 import org.apache.cassandra.io.util.DataOutputBuffer;
 import org.apache.cassandra.utils.*;
@@ -59,7 +61,7 @@ public class CounterColumnTest extends SchemaLoader
     public void testCreate()
     {
         long delta = 3L;
-        CounterUpdateColumn cuc = new CounterUpdateColumn(ByteBufferUtil.bytes("x"), delta, 1L);
+        CounterUpdateColumn cuc = new CounterUpdateColumn(Util.cellname("x"), delta, 1L);
         CounterColumn column = cuc.localCopy(Keyspace.open("Keyspace5").getColumnFamilyStore("Counter1"));
 
         Assert.assertEquals(delta, column.total());
@@ -80,33 +82,33 @@ public class CounterColumnTest extends SchemaLoader
         ByteBuffer context;
 
         // tombstone + tombstone
-        left  = new DeletedColumn(ByteBufferUtil.bytes("x"), 1, 1L);
-        right = new DeletedColumn(ByteBufferUtil.bytes("x"), 2, 2L);
+        left  = new DeletedColumn(Util.cellname("x"), 1, 1L);
+        right = new DeletedColumn(Util.cellname("x"), 2, 2L);
 
         Assert.assertEquals(left.reconcile(right).getMarkedForDeleteAt(), right.getMarkedForDeleteAt());
         Assert.assertEquals(right.reconcile(left).getMarkedForDeleteAt(), right.getMarkedForDeleteAt());
 
         // tombstone > live
-        left  = new DeletedColumn(ByteBufferUtil.bytes("x"), 1, 2L);
-        right = new CounterColumn(ByteBufferUtil.bytes("x"), 0L, 1L);
+        left  = new DeletedColumn(Util.cellname("x"), 1, 2L);
+        right = new CounterColumn(Util.cellname("x"), 0L, 1L);
 
         Assert.assertEquals(left.reconcile(right), left);
 
         // tombstone < live last delete
-        left  = new DeletedColumn(ByteBufferUtil.bytes("x"), 1, 1L);
-        right = new CounterColumn(ByteBufferUtil.bytes("x"), 0L, 4L, 2L);
+        left  = new DeletedColumn(Util.cellname("x"), 1, 1L);
+        right = new CounterColumn(Util.cellname("x"), 0L, 4L, 2L);
 
         Assert.assertEquals(left.reconcile(right), right);
 
         // tombstone == live last delete
-        left  = new DeletedColumn(ByteBufferUtil.bytes("x"), 1, 2L);
-        right = new CounterColumn(ByteBufferUtil.bytes("x"), 0L, 4L, 2L);
+        left  = new DeletedColumn(Util.cellname("x"), 1, 2L);
+        right = new CounterColumn(Util.cellname("x"), 0L, 4L, 2L);
 
         Assert.assertEquals(left.reconcile(right), right);
 
         // tombstone > live last delete
-        left  = new DeletedColumn(ByteBufferUtil.bytes("x"), 1, 4L);
-        right = new CounterColumn(ByteBufferUtil.bytes("x"), 0L, 9L, 1L);
+        left  = new DeletedColumn(Util.cellname("x"), 1, 4L);
+        right = new CounterColumn(Util.cellname("x"), 0L, 9L, 1L);
 
         reconciled = left.reconcile(right);
         Assert.assertEquals(reconciled.name(), right.name());
@@ -115,26 +117,26 @@ public class CounterColumnTest extends SchemaLoader
         Assert.assertEquals(((CounterColumn)reconciled).timestampOfLastDelete(), left.getMarkedForDeleteAt());
 
         // live < tombstone
-        left  = new CounterColumn(ByteBufferUtil.bytes("x"), 0L, 1L);
-        right = new DeletedColumn(ByteBufferUtil.bytes("x"), 1, 2L);
+        left  = new CounterColumn(Util.cellname("x"), 0L, 1L);
+        right = new DeletedColumn(Util.cellname("x"), 1, 2L);
 
         Assert.assertEquals(left.reconcile(right), right);
 
         // live last delete > tombstone
-        left  = new CounterColumn(ByteBufferUtil.bytes("x"), 0L, 4L, 2L);
-        right = new DeletedColumn(ByteBufferUtil.bytes("x"), 1, 1L);
+        left  = new CounterColumn(Util.cellname("x"), 0L, 4L, 2L);
+        right = new DeletedColumn(Util.cellname("x"), 1, 1L);
 
         Assert.assertEquals(left.reconcile(right), left);
 
         // live last delete == tombstone
-        left  = new CounterColumn(ByteBufferUtil.bytes("x"), 0L, 4L, 2L);
-        right = new DeletedColumn(ByteBufferUtil.bytes("x"), 1, 2L);
+        left  = new CounterColumn(Util.cellname("x"), 0L, 4L, 2L);
+        right = new DeletedColumn(Util.cellname("x"), 1, 2L);
 
         Assert.assertEquals(left.reconcile(right), left);
 
         // live last delete < tombstone
-        left  = new CounterColumn(ByteBufferUtil.bytes("x"), 0L, 9L, 1L);
-        right = new DeletedColumn(ByteBufferUtil.bytes("x"), 1, 4L);
+        left  = new CounterColumn(Util.cellname("x"), 0L, 9L, 1L);
+        right = new DeletedColumn(Util.cellname("x"), 1, 4L);
 
         reconciled = left.reconcile(right);
         Assert.assertEquals(reconciled.name(), left.name());
@@ -143,20 +145,20 @@ public class CounterColumnTest extends SchemaLoader
         Assert.assertEquals(((CounterColumn)reconciled).timestampOfLastDelete(), right.getMarkedForDeleteAt());
 
         // live < live last delete
-        left  = new CounterColumn(ByteBufferUtil.bytes("x"), cc.create(CounterId.fromInt(1), 2L, 3L, false), 1L, Long.MIN_VALUE);
-        right = new CounterColumn(ByteBufferUtil.bytes("x"), cc.create(CounterId.fromInt(1), 1L, 1L, false), 4L, 3L);
+        left  = new CounterColumn(Util.cellname("x"), cc.create(CounterId.fromInt(1), 2L, 3L, false), 1L, Long.MIN_VALUE);
+        right = new CounterColumn(Util.cellname("x"), cc.create(CounterId.fromInt(1), 1L, 1L, false), 4L, 3L);
 
         Assert.assertEquals(left.reconcile(right), right);
 
         // live last delete > live
-        left  = new CounterColumn(ByteBufferUtil.bytes("x"), cc.create(CounterId.fromInt(1), 2L, 3L, false), 6L, 5L);
-        right = new CounterColumn(ByteBufferUtil.bytes("x"), cc.create(CounterId.fromInt(1), 1L, 1L, false), 4L, 3L);
+        left  = new CounterColumn(Util.cellname("x"), cc.create(CounterId.fromInt(1), 2L, 3L, false), 6L, 5L);
+        right = new CounterColumn(Util.cellname("x"), cc.create(CounterId.fromInt(1), 1L, 1L, false), 4L, 3L);
 
         Assert.assertEquals(left.reconcile(right), left);
 
         // live + live
-        left = new CounterColumn(ByteBufferUtil.bytes("x"), cc.create(CounterId.fromInt(1), 1L, 1L, false), 4L, Long.MIN_VALUE);
-        right = new CounterColumn(ByteBufferUtil.bytes("x"), cc.create(CounterId.fromInt(1), 2L, 3L, false), 1L, Long.MIN_VALUE);
+        left = new CounterColumn(Util.cellname("x"), cc.create(CounterId.fromInt(1), 1L, 1L, false), 4L, Long.MIN_VALUE);
+        right = new CounterColumn(Util.cellname("x"), cc.create(CounterId.fromInt(1), 2L, 3L, false), 1L, Long.MIN_VALUE);
 
         reconciled = left.reconcile(right);
         Assert.assertEquals(reconciled.name(), left.name());
@@ -164,7 +166,7 @@ public class CounterColumnTest extends SchemaLoader
         Assert.assertEquals(4L, reconciled.timestamp());
 
         left = reconciled;
-        right = new CounterColumn(ByteBufferUtil.bytes("x"), cc.create(CounterId.fromInt(2), 1L, 5L, false), 2L, Long.MIN_VALUE);
+        right = new CounterColumn(Util.cellname("x"), cc.create(CounterId.fromInt(2), 1L, 5L, false), 2L, Long.MIN_VALUE);
 
         reconciled = left.reconcile(right);
         Assert.assertEquals(reconciled.name(), left.name());
@@ -172,7 +174,7 @@ public class CounterColumnTest extends SchemaLoader
         Assert.assertEquals(4L, reconciled.timestamp());
 
         left = reconciled;
-        right = new CounterColumn(ByteBufferUtil.bytes("x"), cc.create(CounterId.fromInt(2), 2L, 2L, false), 6L, Long.MIN_VALUE);
+        right = new CounterColumn(Util.cellname("x"), cc.create(CounterId.fromInt(2), 2L, 2L, false), 6L, Long.MIN_VALUE);
 
         reconciled = left.reconcile(right);
         Assert.assertEquals(reconciled.name(), left.name());
@@ -205,15 +207,15 @@ public class CounterColumnTest extends SchemaLoader
         CounterColumn rightCol;
 
         // timestamp
-        leftCol = new CounterColumn(ByteBufferUtil.bytes("x"), 0, 1L);
-        rightCol = new CounterColumn(ByteBufferUtil.bytes("x"), 0, 2L);
+        leftCol = new CounterColumn(Util.cellname("x"), 0, 1L);
+        rightCol = new CounterColumn(Util.cellname("x"), 0, 2L);
 
         Assert.assertEquals(rightCol, leftCol.diff(rightCol));
         Assert.assertNull(rightCol.diff(leftCol));
 
         // timestampOfLastDelete
-        leftCol = new CounterColumn(ByteBufferUtil.bytes("x"), 0, 1L, 1L);
-        rightCol = new CounterColumn(ByteBufferUtil.bytes("x"), 0, 1L, 2L);
+        leftCol = new CounterColumn(Util.cellname("x"), 0, 1L, 1L);
+        rightCol = new CounterColumn(Util.cellname("x"), 0, 1L, 2L);
 
         Assert.assertEquals(rightCol, leftCol.diff(rightCol));
         Assert.assertNull(rightCol.diff(leftCol));
@@ -225,8 +227,8 @@ public class CounterColumnTest extends SchemaLoader
         left.writeElement(CounterId.fromInt(9), 1L, 0L);
         right = new ContextState(ByteBufferUtil.clone(left.context), 2);
 
-        leftCol  = new CounterColumn(ByteBufferUtil.bytes("x"), left.context,  1L);
-        rightCol = new CounterColumn(ByteBufferUtil.bytes("x"), right.context, 1L);
+        leftCol  = new CounterColumn(Util.cellname("x"), left.context,  1L);
+        rightCol = new CounterColumn(Util.cellname("x"), right.context, 1L);
         Assert.assertNull(leftCol.diff(rightCol));
 
         // greater than: left has superset of nodes (counts equal)
@@ -241,8 +243,8 @@ public class CounterColumnTest extends SchemaLoader
         right.writeElement(CounterId.fromInt(6), 2L, 0L);
         right.writeElement(CounterId.fromInt(9), 1L, 0L);
 
-        leftCol  = new CounterColumn(ByteBufferUtil.bytes("x"), left.context,  1L);
-        rightCol = new CounterColumn(ByteBufferUtil.bytes("x"), right.context, 1L);
+        leftCol  = new CounterColumn(Util.cellname("x"), left.context,  1L);
+        rightCol = new CounterColumn(Util.cellname("x"), right.context, 1L);
         Assert.assertNull(leftCol.diff(rightCol));
 
         // less than: right has subset of nodes (counts equal)
@@ -259,8 +261,8 @@ public class CounterColumnTest extends SchemaLoader
         right.writeElement(CounterId.fromInt(6), 1L, 0L);
         right.writeElement(CounterId.fromInt(9), 1L, 0L);
 
-        leftCol  = new CounterColumn(ByteBufferUtil.bytes("x"), left.context,  1L);
-        rightCol = new CounterColumn(ByteBufferUtil.bytes("x"), right.context, 1L);
+        leftCol  = new CounterColumn(Util.cellname("x"), left.context,  1L);
+        rightCol = new CounterColumn(Util.cellname("x"), right.context, 1L);
         Assert.assertEquals(rightCol, leftCol.diff(rightCol));
         Assert.assertEquals(leftCol, rightCol.diff(leftCol));
     }
@@ -275,20 +277,21 @@ public class CounterColumnTest extends SchemaLoader
         state.writeElement(CounterId.fromInt(3), 4L, 4L);
         state.writeElement(CounterId.fromInt(4), 4L, 4L, true);
 
-        CounterColumn original = new CounterColumn(ByteBufferUtil.bytes("x"), state.context, 1L);
+        CellNameType type = new SimpleDenseCellNameType(UTF8Type.instance);
+        CounterColumn original = new CounterColumn(Util.cellname("x"), state.context, 1L);
         byte[] serialized;
         try (DataOutputBuffer bufOut = new DataOutputBuffer())
         {
-            Column.serializer.serialize(original, bufOut);
+            type.columnSerializer().serialize(original, bufOut);
             serialized = bufOut.getData();
         }
 
         ByteArrayInputStream bufIn = new ByteArrayInputStream(serialized, 0, serialized.length);
-        CounterColumn deserialized = (CounterColumn) Column.serializer.deserialize(new DataInputStream(bufIn));
+        CounterColumn deserialized = (CounterColumn) type.columnSerializer().deserialize(new DataInputStream(bufIn));
         Assert.assertEquals(original, deserialized);
 
         bufIn = new ByteArrayInputStream(serialized, 0, serialized.length);
-        CounterColumn deserializedOnRemote = (CounterColumn) Column.serializer.deserialize(new DataInputStream(bufIn), ColumnSerializer.Flag.FROM_REMOTE);
+        CounterColumn deserializedOnRemote = (CounterColumn) type.columnSerializer().deserialize(new DataInputStream(bufIn), ColumnSerializer.Flag.FROM_REMOTE);
         Assert.assertEquals(deserializedOnRemote.name(), original.name());
         Assert.assertEquals(deserializedOnRemote.total(), original.total());
         Assert.assertEquals(deserializedOnRemote.value(), cc.clearAllDelta(original.value()));
@@ -309,8 +312,8 @@ public class CounterColumnTest extends SchemaLoader
         state.writeElement(CounterId.fromInt(3), 4L, 4L);
         state.writeElement(CounterId.fromInt(4), 4L, 4L, true);
 
-        CounterColumn original = new CounterColumn(ByteBufferUtil.bytes("x"), state.context, 1L);
-        CounterColumn cleared = new CounterColumn(ByteBufferUtil.bytes("x"), cc.clearAllDelta(state.context), 1L);
+        CounterColumn original = new CounterColumn(Util.cellname("x"), state.context, 1L);
+        CounterColumn cleared = new CounterColumn(Util.cellname("x"), cc.clearAllDelta(state.context), 1L);
 
         original.updateDigest(digest1);
         cleared.updateDigest(digest2);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/362cc053/test/unit/org/apache/cassandra/db/CounterMutationTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/CounterMutationTest.java b/test/unit/org/apache/cassandra/db/CounterMutationTest.java
index 3cbd030..74deb37 100644
--- a/test/unit/org/apache/cassandra/db/CounterMutationTest.java
+++ b/test/unit/org/apache/cassandra/db/CounterMutationTest.java
@@ -42,7 +42,7 @@ public class CounterMutationTest extends SchemaLoader
         CounterId id1 = CounterId.getLocalId();
 
         rm = new RowMutation("Keyspace1", ByteBufferUtil.bytes("key1"));
-        rm.addCounter("Counter1", ByteBufferUtil.bytes("Column1"), 3);
+        rm.addCounter("Counter1", Util.cellname("Column1"), 3);
         cm = new CounterMutation(rm, ConsistencyLevel.ONE);
         cm.apply();
 
@@ -50,7 +50,7 @@ public class CounterMutationTest extends SchemaLoader
         CounterId id2 = CounterId.getLocalId();
 
         rm = new RowMutation("Keyspace1", ByteBufferUtil.bytes("key1"));
-        rm.addCounter("Counter1", ByteBufferUtil.bytes("Column1"), 4);
+        rm.addCounter("Counter1", Util.cellname("Column1"), 4);
         cm = new CounterMutation(rm, ConsistencyLevel.ONE);
         cm.apply();
 
@@ -58,8 +58,8 @@ public class CounterMutationTest extends SchemaLoader
         CounterId id3 = CounterId.getLocalId();
 
         rm = new RowMutation("Keyspace1", ByteBufferUtil.bytes("key1"));
-        rm.addCounter("Counter1", ByteBufferUtil.bytes("Column1"), 5);
-        rm.addCounter("Counter1", ByteBufferUtil.bytes("Column2"), 1);
+        rm.addCounter("Counter1", Util.cellname("Column1"), 5);
+        rm.addCounter("Counter1", Util.cellname("Column2"), 1);
         cm = new CounterMutation(rm, ConsistencyLevel.ONE);
         cm.apply();
 
@@ -69,7 +69,7 @@ public class CounterMutationTest extends SchemaLoader
         // First merges old shards
         CounterColumn.mergeAndRemoveOldShards(dk, cf, Integer.MIN_VALUE, Integer.MAX_VALUE, false);
         long now = System.currentTimeMillis();
-        Column c = cf.getColumn(ByteBufferUtil.bytes("Column1"));
+        Column c = cf.getColumn(Util.cellname("Column1"));
         assert c != null;
         assert c instanceof CounterColumn;
         assert ((CounterColumn)c).total() == 12L;
@@ -89,7 +89,7 @@ public class CounterMutationTest extends SchemaLoader
 
         // Then collect old shards
         CounterColumn.mergeAndRemoveOldShards(dk, cf, Integer.MAX_VALUE, Integer.MIN_VALUE, false);
-        c = cf.getColumn(ByteBufferUtil.bytes("Column1"));
+        c = cf.getColumn(Util.cellname("Column1"));
         assert c != null;
         assert c instanceof CounterColumn;
         assert ((CounterColumn)c).total() == 12L;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/362cc053/test/unit/org/apache/cassandra/db/HintedHandOffTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/HintedHandOffTest.java b/test/unit/org/apache/cassandra/db/HintedHandOffTest.java
index 23db4bb..4978c57 100644
--- a/test/unit/org/apache/cassandra/db/HintedHandOffTest.java
+++ b/test/unit/org/apache/cassandra/db/HintedHandOffTest.java
@@ -29,6 +29,7 @@ import java.util.concurrent.TimeUnit;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.Util;
 import org.apache.cassandra.cql3.UntypedResultSet;
 import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy;
@@ -63,7 +64,7 @@ public class HintedHandOffTest extends SchemaLoader
 
         // insert 1 hint
         RowMutation rm = new RowMutation(KEYSPACE4, ByteBufferUtil.bytes(1));
-        rm.add(STANDARD1_CF, ByteBufferUtil.bytes(String.valueOf(COLUMN1)), ByteBufferUtil.EMPTY_BYTE_BUFFER, System.currentTimeMillis());
+        rm.add(STANDARD1_CF, Util.cellname(COLUMN1), ByteBufferUtil.EMPTY_BYTE_BUFFER, System.currentTimeMillis());
 
         HintedHandOffManager.instance.hintFor(rm, HintedHandOffManager.calculateHintTTL(rm), UUID.randomUUID()).apply();
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/362cc053/test/unit/org/apache/cassandra/db/KeyCacheTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/KeyCacheTest.java b/test/unit/org/apache/cassandra/db/KeyCacheTest.java
index 1f41860..dc5b22c 100644
--- a/test/unit/org/apache/cassandra/db/KeyCacheTest.java
+++ b/test/unit/org/apache/cassandra/db/KeyCacheTest.java
@@ -28,6 +28,7 @@ import org.junit.Test;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
 import org.apache.cassandra.cache.KeyCacheKey;
+import org.apache.cassandra.db.composites.*;
 import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.db.filter.QueryFilter;
 import org.apache.cassandra.service.CacheService;
@@ -117,10 +118,10 @@ public class KeyCacheTest extends SchemaLoader
 
         // inserts
         rm = new RowMutation(KEYSPACE1, key1.key);
-        rm.add(COLUMN_FAMILY1, ByteBufferUtil.bytes("1"), ByteBufferUtil.EMPTY_BYTE_BUFFER, 0);
+        rm.add(COLUMN_FAMILY1, Util.cellname("1"), ByteBufferUtil.EMPTY_BYTE_BUFFER, 0);
         rm.apply();
         rm = new RowMutation(KEYSPACE1, key2.key);
-        rm.add(COLUMN_FAMILY1, ByteBufferUtil.bytes("2"), ByteBufferUtil.EMPTY_BYTE_BUFFER, 0);
+        rm.add(COLUMN_FAMILY1, Util.cellname("2"), ByteBufferUtil.EMPTY_BYTE_BUFFER, 0);
         rm.apply();
 
         // to make sure we have SSTable
@@ -129,16 +130,16 @@ public class KeyCacheTest extends SchemaLoader
         // reads to cache key position
         cfs.getColumnFamily(QueryFilter.getSliceFilter(key1,
                                                        COLUMN_FAMILY1,
-                                                       ByteBufferUtil.EMPTY_BYTE_BUFFER,
-                                                       ByteBufferUtil.EMPTY_BYTE_BUFFER,
+                                                       Composites.EMPTY,
+                                                       Composites.EMPTY,
                                                        false,
                                                        10,
                                                        System.currentTimeMillis()));
 
         cfs.getColumnFamily(QueryFilter.getSliceFilter(key2,
                                                        COLUMN_FAMILY1,
-                                                       ByteBufferUtil.EMPTY_BYTE_BUFFER,
-                                                       ByteBufferUtil.EMPTY_BYTE_BUFFER,
+                                                       Composites.EMPTY,
+                                                       Composites.EMPTY,
                                                        false,
                                                        10,
                                                        System.currentTimeMillis()));
@@ -153,16 +154,16 @@ public class KeyCacheTest extends SchemaLoader
         // re-read same keys to verify that key cache didn't grow further
         cfs.getColumnFamily(QueryFilter.getSliceFilter(key1,
                                                        COLUMN_FAMILY1,
-                                                       ByteBufferUtil.EMPTY_BYTE_BUFFER,
-                                                       ByteBufferUtil.EMPTY_BYTE_BUFFER,
+                                                       Composites.EMPTY,
+                                                       Composites.EMPTY,
                                                        false,
                                                        10,
                                                        System.currentTimeMillis()));
 
         cfs.getColumnFamily(QueryFilter.getSliceFilter(key2,
                                                        COLUMN_FAMILY1,
-                                                       ByteBufferUtil.EMPTY_BYTE_BUFFER,
-                                                       ByteBufferUtil.EMPTY_BYTE_BUFFER,
+                                                       Composites.EMPTY,
+                                                       Composites.EMPTY,
                                                        false,
                                                        10,
                                                        System.currentTimeMillis()));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/362cc053/test/unit/org/apache/cassandra/db/KeyCollisionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/KeyCollisionTest.java b/test/unit/org/apache/cassandra/db/KeyCollisionTest.java
index 345febd..0f3e6d9 100644
--- a/test/unit/org/apache/cassandra/db/KeyCollisionTest.java
+++ b/test/unit/org/apache/cassandra/db/KeyCollisionTest.java
@@ -26,6 +26,7 @@ import java.util.*;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.Util;
 import org.apache.cassandra.db.columniterator.IdentityQueryFilter;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.marshal.IntegerType;
@@ -89,7 +90,7 @@ public class KeyCollisionTest extends SchemaLoader
     {
         RowMutation rm;
         rm = new RowMutation(KEYSPACE, ByteBufferUtil.bytes(key));
-        rm.add(CF, ByteBufferUtil.bytes("column"), ByteBufferUtil.bytes("asdf"), 0);
+        rm.add(CF, Util.cellname("column"), ByteBufferUtil.bytes("asdf"), 0);
         rm.apply();
     }