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/01/02 13:41:16 UTC

[3/9] Replace supercolumns internally by composites

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a005df3/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 0ea4f71..f2184fe 100644
--- a/test/unit/org/apache/cassandra/db/ArrayBackedSortedColumnsTest.java
+++ b/test/unit/org/apache/cassandra/db/ArrayBackedSortedColumnsTest.java
@@ -52,7 +52,7 @@ public class ArrayBackedSortedColumnsTest
         for (int i = 0; i < values.length; ++i)
             map.addColumn(new Column(ByteBufferUtil.bytes(values[reversed ? values.length - 1 - i : i])), HeapAllocator.instance);
 
-        Iterator<IColumn> iter = map.iterator();
+        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));
@@ -79,9 +79,9 @@ public class ArrayBackedSortedColumnsTest
         for (int i = 0; i < values2.length; ++i)
             map2.addColumn(new Column(ByteBufferUtil.bytes(values2[reversed ? values2.length - 1 - i : i])), HeapAllocator.instance);
 
-        map2.addAll(map, HeapAllocator.instance, Functions.<IColumn>identity());
+        map2.addAll(map, HeapAllocator.instance, Functions.<Column>identity());
 
-        Iterator<IColumn> iter = map2.iterator();
+        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));
@@ -102,10 +102,10 @@ public class ArrayBackedSortedColumnsTest
         ISortedColumns map = ArrayBackedSortedColumns.factory().create(BytesType.instance, reversed);
         int[] values = new int[]{ 1, 2, 3, 5, 9 };
 
-        List<IColumn> sorted = new ArrayList<IColumn>();
+        List<Column> sorted = new ArrayList<Column>();
         for (int v : values)
             sorted.add(new Column(ByteBufferUtil.bytes(v)));
-        List<IColumn> reverseSorted = new ArrayList<IColumn>(sorted);
+        List<Column> reverseSorted = new ArrayList<Column>(sorted);
         Collections.reverse(reverseSorted);
 
         for (int i = 0; i < values.length; ++i)
@@ -177,7 +177,7 @@ public class ArrayBackedSortedColumnsTest
             fail("The collection don't have the same size");
     }
 
-    private void assertSame(int[] names, Iterator<IColumn> iter)
+    private void assertSame(int[] names, Iterator<Column> iter)
     {
         for (int name : names)
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a005df3/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 4de36ac..b4911d7 100644
--- a/test/unit/org/apache/cassandra/db/CleanupTest.java
+++ b/test/unit/org/apache/cassandra/db/CleanupTest.java
@@ -33,7 +33,6 @@ import org.apache.cassandra.db.filter.IDiskAtomFilter;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.db.columniterator.IdentityQueryFilter;
 import org.apache.cassandra.db.compaction.CompactionManager;
-import org.apache.cassandra.db.filter.QueryPath;
 import org.apache.cassandra.db.index.SecondaryIndex;
 import org.apache.cassandra.dht.BytesToken;
 import org.apache.cassandra.dht.IPartitioner;
@@ -151,7 +150,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(TABLE1, ByteBufferUtil.bytes(key));
-            rm.add(new QueryPath(cfs.name, null, COLUMN), VALUE, System.currentTimeMillis());
+            rm.add(cfs.name, COLUMN, VALUE, System.currentTimeMillis());
             rm.applyUnsafe();
         }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a005df3/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 4a8e426..4ef8d64 100644
--- a/test/unit/org/apache/cassandra/db/CollationControllerTest.java
+++ b/test/unit/org/apache/cassandra/db/CollationControllerTest.java
@@ -26,7 +26,6 @@ import java.util.concurrent.ExecutionException;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
 import org.apache.cassandra.db.filter.QueryFilter;
-import org.apache.cassandra.db.filter.QueryPath;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.junit.Test;
 
@@ -42,36 +41,35 @@ public class CollationControllerTest extends SchemaLoader
         ColumnFamilyStore store = table.getColumnFamilyStore("Standard1");
         RowMutation rm;
         DecoratedKey dk = Util.dk("key1");
-        QueryPath path = new QueryPath("Standard1", null, ByteBufferUtil.bytes("Column1"));
         
         // add data
         rm = new RowMutation("Keyspace1", dk.key);
-        rm.add(path, ByteBufferUtil.bytes("asdf"), 0);
+        rm.add("Standard1", ByteBufferUtil.bytes("Column1"), ByteBufferUtil.bytes("asdf"), 0);
         rm.apply();
         store.forceBlockingFlush();
         
         // remove
         rm = new RowMutation("Keyspace1", dk.key);
-        rm.delete(new QueryPath("Standard1"), 10);
+        rm.delete("Standard1", 10);
         rm.apply();
         
         // add another mutation because sstable maxtimestamp isn't set
         // correctly during flush if the most recent mutation is a row delete
         rm = new RowMutation("Keyspace1", Util.dk("key2").key);
-        rm.add(path, ByteBufferUtil.bytes("zxcv"), 20);
+        rm.add("Standard1", ByteBufferUtil.bytes("Column1"), ByteBufferUtil.bytes("zxcv"), 20);
         rm.apply();
         
         store.forceBlockingFlush();
 
         // add yet one more mutation
         rm = new RowMutation("Keyspace1", dk.key);
-        rm.add(path, ByteBufferUtil.bytes("foobar"), 30);
+        rm.add("Standard1", ByteBufferUtil.bytes("Column1"), ByteBufferUtil.bytes("foobar"), 30);
         rm.apply();
         store.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, path, ByteBufferUtil.bytes("Column1"));
+        QueryFilter filter = QueryFilter.getNamesFilter(dk, "Standard1", ByteBufferUtil.bytes("Column1"));
         CollationController controller = new CollationController(store, false, filter, Integer.MIN_VALUE);
         controller.getTopLevelColumns();
         assertEquals(1, controller.getSstablesIterated());
@@ -79,7 +77,7 @@ public class CollationControllerTest extends SchemaLoader
         // SliceQueryFilter goes down another path (through collectAllData())
         // We will read "only" the last sstable in that case, but because the 2nd sstable has a tombstone that is more
         // recent than the maxTimestamp of the very first sstable we flushed, we should only read the 2 first sstables.
-        filter = QueryFilter.getIdentityFilter(dk, path);
+        filter = QueryFilter.getIdentityFilter(dk, "Standard1");
         controller = new CollationController(store, false, filter, Integer.MIN_VALUE);
         controller.getTopLevelColumns();
         assertEquals(2, controller.getSstablesIterated());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a005df3/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 b6f9817..e77277f 100644
--- a/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
+++ b/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
@@ -32,6 +32,8 @@ import java.util.LinkedList;
 import java.util.List;
 import java.util.Random;
 import java.util.Set;
+import java.util.SortedSet;
+import java.util.TreeSet;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Future;
 
@@ -100,17 +102,17 @@ public class ColumnFamilyStoreTest extends SchemaLoader
 
         RowMutation rm;
         rm = new RowMutation("Keyspace1", ByteBufferUtil.bytes("key1"));
-        rm.add(new QueryPath("Standard1", null, ByteBufferUtil.bytes("Column1")), ByteBufferUtil.bytes("asdf"), 0);
+        rm.add("Standard1", ByteBufferUtil.bytes("Column1"), ByteBufferUtil.bytes("asdf"), 0);
         rm.apply();
         cfs.forceBlockingFlush();
 
         rm = new RowMutation("Keyspace1", ByteBufferUtil.bytes("key1"));
-        rm.add(new QueryPath("Standard1", null, ByteBufferUtil.bytes("Column1")), ByteBufferUtil.bytes("asdf"), 1);
+        rm.add("Standard1", ByteBufferUtil.bytes("Column1"), ByteBufferUtil.bytes("asdf"), 1);
         rm.apply();
         cfs.forceBlockingFlush();
 
         cfs.getRecentSSTablesPerReadHistogram(); // resets counts
-        cfs.getColumnFamily(QueryFilter.getNamesFilter(Util.dk("key1"), new QueryPath("Standard1", null), ByteBufferUtil.bytes("Column1")));
+        cfs.getColumnFamily(QueryFilter.getNamesFilter(Util.dk("key1"), "Standard1", ByteBufferUtil.bytes("Column1")));
         assertEquals(1, cfs.getRecentSSTablesPerReadHistogram()[0]);
     }
 
@@ -124,15 +126,15 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         List<IMutation> rms = new LinkedList<IMutation>();
         RowMutation rm;
         rm = new RowMutation("Keyspace1", ByteBufferUtil.bytes("key1"));
-        rm.add(new QueryPath("Standard1", null, ByteBufferUtil.bytes("Column1")), ByteBufferUtil.bytes("asdf"), 0);
-        rm.add(new QueryPath("Standard1", null, ByteBufferUtil.bytes("Column2")), ByteBufferUtil.bytes("asdf"), 0);
+        rm.add("Standard1", ByteBufferUtil.bytes("Column1"), ByteBufferUtil.bytes("asdf"), 0);
+        rm.add("Standard1", ByteBufferUtil.bytes("Column2"), ByteBufferUtil.bytes("asdf"), 0);
         rms.add(rm);
         Util.writeColumnFamily(rms);
 
         List<SSTableReader> ssTables = table.getAllSSTables();
         assertEquals(1, ssTables.size());
         ssTables.get(0).forceFilterFailures();
-        ColumnFamily cf = cfs.getColumnFamily(QueryFilter.getIdentityFilter(Util.dk("key2"), new QueryPath("Standard1", null, ByteBufferUtil.bytes("Column1"))));
+        ColumnFamily cf = cfs.getColumnFamily(QueryFilter.getIdentityFilter(Util.dk("key2"), "Standard1"));
         assertNull(cf);
     }
 
@@ -144,19 +146,19 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         RowMutation rm;
 
         rm = new RowMutation("Keyspace1", ByteBufferUtil.bytes("key1"));
-        rm.delete(new QueryPath("Standard2", null, null), System.currentTimeMillis());
+        rm.delete("Standard2", System.currentTimeMillis());
         rm.apply();
 
         Runnable r = new WrappedRunnable()
         {
             public void runMayThrow() throws IOException
             {
-                QueryFilter sliceFilter = QueryFilter.getSliceFilter(Util.dk("key1"), new QueryPath("Standard2", null, null), ByteBufferUtil.EMPTY_BYTE_BUFFER, ByteBufferUtil.EMPTY_BYTE_BUFFER, false, 1);
+                QueryFilter sliceFilter = QueryFilter.getSliceFilter(Util.dk("key1"), "Standard2", ByteBufferUtil.EMPTY_BYTE_BUFFER, ByteBufferUtil.EMPTY_BYTE_BUFFER, false, 1);
                 ColumnFamily cf = store.getColumnFamily(sliceFilter);
                 assert cf.isMarkedForDelete();
                 assert cf.isEmpty();
 
-                QueryFilter namesFilter = QueryFilter.getNamesFilter(Util.dk("key1"), new QueryPath("Standard2", null, null), ByteBufferUtil.bytes("a"));
+                QueryFilter namesFilter = QueryFilter.getNamesFilter(Util.dk("key1"), "Standard2", ByteBufferUtil.bytes("a"));
                 cf = store.getColumnFamily(namesFilter);
                 assert cf.isMarkedForDelete();
                 assert cf.isEmpty();
@@ -172,8 +174,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         ColumnFamilyStore cfs = insertKey1Key2();
 
         IPartitioner p = StorageService.getPartitioner();
-        List<Row> result = cfs.getRangeSlice(ByteBufferUtil.EMPTY_BYTE_BUFFER,
-                                             Util.range(p, "key1", "key2"),
+        List<Row> result = cfs.getRangeSlice(Util.range(p, "key1", "key2"),
                                              10,
                                              new NamesQueryFilter(ByteBufferUtil.bytes("asdf")),
                                              null);
@@ -187,23 +188,23 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         RowMutation rm;
 
         rm = new RowMutation("Keyspace1", ByteBufferUtil.bytes("k1"));
-        rm.add(new QueryPath("Indexed1", null, ByteBufferUtil.bytes("notbirthdate")), ByteBufferUtil.bytes(1L), 0);
-        rm.add(new QueryPath("Indexed1", null, ByteBufferUtil.bytes("birthdate")), ByteBufferUtil.bytes(1L), 0);
+        rm.add("Indexed1", ByteBufferUtil.bytes("notbirthdate"), ByteBufferUtil.bytes(1L), 0);
+        rm.add("Indexed1", ByteBufferUtil.bytes("birthdate"), ByteBufferUtil.bytes(1L), 0);
         rm.apply();
 
         rm = new RowMutation("Keyspace1", ByteBufferUtil.bytes("k2"));
-        rm.add(new QueryPath("Indexed1", null, ByteBufferUtil.bytes("notbirthdate")), ByteBufferUtil.bytes(2L), 0);
-        rm.add(new QueryPath("Indexed1", null, ByteBufferUtil.bytes("birthdate")), ByteBufferUtil.bytes(2L), 0);
+        rm.add("Indexed1", ByteBufferUtil.bytes("notbirthdate"), ByteBufferUtil.bytes(2L), 0);
+        rm.add("Indexed1", ByteBufferUtil.bytes("birthdate"), ByteBufferUtil.bytes(2L), 0);
         rm.apply();
 
         rm = new RowMutation("Keyspace1", ByteBufferUtil.bytes("k3"));
-        rm.add(new QueryPath("Indexed1", null, ByteBufferUtil.bytes("notbirthdate")), ByteBufferUtil.bytes(2L), 0);
-        rm.add(new QueryPath("Indexed1", null, ByteBufferUtil.bytes("birthdate")), ByteBufferUtil.bytes(1L), 0);
+        rm.add("Indexed1", ByteBufferUtil.bytes("notbirthdate"), ByteBufferUtil.bytes(2L), 0);
+        rm.add("Indexed1", ByteBufferUtil.bytes("birthdate"), ByteBufferUtil.bytes(1L), 0);
         rm.apply();
 
         rm = new RowMutation("Keyspace1", ByteBufferUtil.bytes("k4aaaa"));
-        rm.add(new QueryPath("Indexed1", null, ByteBufferUtil.bytes("notbirthdate")), ByteBufferUtil.bytes(2L), 0);
-        rm.add(new QueryPath("Indexed1", null, ByteBufferUtil.bytes("birthdate")), ByteBufferUtil.bytes(3L), 0);
+        rm.add("Indexed1", ByteBufferUtil.bytes("notbirthdate"), ByteBufferUtil.bytes(2L), 0);
+        rm.add("Indexed1", ByteBufferUtil.bytes("birthdate"), ByteBufferUtil.bytes(3L), 0);
         rm.apply();
 
         // basic single-expression query
@@ -270,8 +271,8 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         for (int i = 0; i < 100; i++)
         {
             rm = new RowMutation("Keyspace1", ByteBufferUtil.bytes("key" + i));
-            rm.add(new QueryPath("Indexed1", null, ByteBufferUtil.bytes("birthdate")), ByteBufferUtil.bytes(34L), 0);
-            rm.add(new QueryPath("Indexed1", null, ByteBufferUtil.bytes("notbirthdate")), ByteBufferUtil.bytes((long) (i % 2)), 0);
+            rm.add("Indexed1", ByteBufferUtil.bytes("birthdate"), ByteBufferUtil.bytes(34L), 0);
+            rm.add("Indexed1", ByteBufferUtil.bytes("notbirthdate"), ByteBufferUtil.bytes((long) (i % 2)), 0);
             rm.applyUnsafe();
         }
 
@@ -299,7 +300,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         RowMutation rm;
 
         rm = new RowMutation("Keyspace3", ByteBufferUtil.bytes("k1"));
-        rm.add(new QueryPath("Indexed1", null, ByteBufferUtil.bytes("birthdate")), ByteBufferUtil.bytes(1L), 0);
+        rm.add("Indexed1", ByteBufferUtil.bytes("birthdate"), ByteBufferUtil.bytes(1L), 0);
         rm.apply();
 
         IndexExpression expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), IndexOperator.EQ, ByteBufferUtil.bytes(1L));
@@ -314,13 +315,13 @@ public class ColumnFamilyStoreTest extends SchemaLoader
 
         // delete the column directly
         rm = new RowMutation("Keyspace3", ByteBufferUtil.bytes("k1"));
-        rm.delete(new QueryPath("Indexed1", null, ByteBufferUtil.bytes("birthdate")), 1);
+        rm.delete("Indexed1", ByteBufferUtil.bytes("birthdate"), 1);
         rm.apply();
         rows = cfs.search(clause, range, 100, filter);
         assert rows.isEmpty();
 
         // verify that it's not being indexed under the deletion column value either
-        IColumn deletion = rm.getColumnFamilies().iterator().next().iterator().next();
+        Column deletion = rm.getColumnFamilies().iterator().next().iterator().next();
         ByteBuffer deletionLong = ByteBufferUtil.bytes((long) ByteBufferUtil.toInt(deletion.value()));
         IndexExpression expr0 = new IndexExpression(ByteBufferUtil.bytes("birthdate"), IndexOperator.EQ, deletionLong);
         List<IndexExpression> clause0 = Arrays.asList(expr0);
@@ -329,7 +330,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
 
         // resurrect w/ a newer timestamp
         rm = new RowMutation("Keyspace3", ByteBufferUtil.bytes("k1"));
-        rm.add(new QueryPath("Indexed1", null, ByteBufferUtil.bytes("birthdate")), ByteBufferUtil.bytes(1L), 2);
+        rm.add("Indexed1", ByteBufferUtil.bytes("birthdate"), ByteBufferUtil.bytes(1L), 2);
         rm.apply();
         rows = cfs.search(clause, range, 100, filter);
         assert rows.size() == 1 : StringUtils.join(rows, ",");
@@ -338,7 +339,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
 
         // verify that row and delete w/ older timestamp does nothing
         rm = new RowMutation("Keyspace3", ByteBufferUtil.bytes("k1"));
-        rm.delete(new QueryPath("Indexed1"), 1);
+        rm.delete("Indexed1", 1);
         rm.apply();
         rows = cfs.search(clause, range, 100, filter);
         assert rows.size() == 1 : StringUtils.join(rows, ",");
@@ -347,7 +348,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
 
         // similarly, column delete w/ older timestamp should do nothing
         rm = new RowMutation("Keyspace3", ByteBufferUtil.bytes("k1"));
-        rm.delete(new QueryPath("Indexed1", null, ByteBufferUtil.bytes("birthdate")), 1);
+        rm.delete("Indexed1", ByteBufferUtil.bytes("birthdate"), 1);
         rm.apply();
         rows = cfs.search(clause, range, 100, filter);
         assert rows.size() == 1 : StringUtils.join(rows, ",");
@@ -356,30 +357,30 @@ public class ColumnFamilyStoreTest extends SchemaLoader
 
         // delete the entire row (w/ newer timestamp this time)
         rm = new RowMutation("Keyspace3", ByteBufferUtil.bytes("k1"));
-        rm.delete(new QueryPath("Indexed1"), 3);
+        rm.delete("Indexed1", 3);
         rm.apply();
         rows = cfs.search(clause, range, 100, filter);
         assert rows.isEmpty() : StringUtils.join(rows, ",");
 
         // make sure obsolete mutations don't generate an index entry
         rm = new RowMutation("Keyspace3", ByteBufferUtil.bytes("k1"));
-        rm.add(new QueryPath("Indexed1", null, ByteBufferUtil.bytes("birthdate")), ByteBufferUtil.bytes(1L), 3);
+        rm.add("Indexed1", ByteBufferUtil.bytes("birthdate"), ByteBufferUtil.bytes(1L), 3);
         rm.apply();
         rows = cfs.search(clause, range, 100, filter);
         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(new QueryPath("Indexed1", null, ByteBufferUtil.bytes("birthdate")), ByteBufferUtil.bytes(1L), 1);
-        rm.delete(new QueryPath("Indexed1"), 2);
+        rm.add("Indexed1", ByteBufferUtil.bytes("birthdate"), ByteBufferUtil.bytes(1L), 1);
+        rm.delete("Indexed1", 2);
         rm.apply();
         rows = cfs.search(clause, range, 100, filter);
         assert rows.isEmpty() : StringUtils.join(rows, ",");
 
         // try row delete followed by insert in the same mutation
         rm = new RowMutation("Keyspace3", ByteBufferUtil.bytes("k1"));
-        rm.delete(new QueryPath("Indexed1"), 3);
-        rm.add(new QueryPath("Indexed1", null, ByteBufferUtil.bytes("birthdate")), ByteBufferUtil.bytes(1L), 4);
+        rm.delete("Indexed1", 3);
+        rm.add("Indexed1", ByteBufferUtil.bytes("birthdate"), ByteBufferUtil.bytes(1L), 4);
         rm.apply();
         rows = cfs.search(clause, range, 100, filter);
         assert rows.size() == 1 : StringUtils.join(rows, ",");
@@ -395,10 +396,10 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         // 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(new QueryPath("Indexed1", null, ByteBufferUtil.bytes("birthdate")), ByteBufferUtil.bytes(1L), 1);
+        rm.add("Indexed1", ByteBufferUtil.bytes("birthdate"), ByteBufferUtil.bytes(1L), 1);
         rm.apply();
         rm = new RowMutation("Keyspace2", ByteBufferUtil.bytes("k1"));
-        rm.add(new QueryPath("Indexed1", null, ByteBufferUtil.bytes("birthdate")), ByteBufferUtil.bytes(2L), 2);
+        rm.add("Indexed1", ByteBufferUtil.bytes("birthdate"), ByteBufferUtil.bytes(2L), 2);
         rm.apply();
 
         IndexExpression expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), IndexOperator.EQ, ByteBufferUtil.bytes(1L));
@@ -417,7 +418,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(new QueryPath("Indexed1", null, ByteBufferUtil.bytes("birthdate")), ByteBufferUtil.bytes(3L), 0);
+        rm.add("Indexed1", ByteBufferUtil.bytes("birthdate"), ByteBufferUtil.bytes(3L), 0);
         rm.apply();
 
         rows = table.getColumnFamilyStore("Indexed1").search(clause, range, 100, filter);
@@ -444,7 +445,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         // create a row and update the "birthdate" value, test that the index query fetches this version
         RowMutation rm;
         rm = new RowMutation(keySpace, rowKey);
-        rm.add(new QueryPath(cfName, null, colName), val1, 0);
+        rm.add(cfName, colName, val1, 0);
         rm.apply();
         IndexExpression expr = new IndexExpression(colName, IndexOperator.EQ, val1);
         List<IndexExpression> clause = Arrays.asList(expr);
@@ -458,7 +459,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
 
         // now apply another update, but force the index update to be skipped
         rm = new RowMutation(keySpace, rowKey);
-        rm.add(new QueryPath(cfName, null, colName), val2, 1);
+        rm.add(cfName, colName, val2, 1);
         table.apply(rm, true, false);
 
         // Now searching the index for either the old or new value should return 0 rows
@@ -478,7 +479,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         // now, reset back to the original value, still skipping the index update, to
         // make sure the value was expunged from the index when it was discovered to be inconsistent
         rm = new RowMutation(keySpace, rowKey);
-        rm.add(new QueryPath(cfName, null, colName), ByteBufferUtil.bytes(1L), 3);
+        rm.add(cfName, colName, ByteBufferUtil.bytes(1L), 3);
         table.apply(rm, true, false);
 
         expr = new IndexExpression(colName, IndexOperator.EQ, ByteBufferUtil.bytes(1L));
@@ -514,7 +515,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         // create a row and update the author value
         RowMutation rm;
         rm = new RowMutation(keySpace, rowKey);
-        rm.add(new QueryPath(cfName, null , compositeName), val1, 0);
+        rm.add(cfName, compositeName, val1, 0);
         rm.apply();
 
         // test that the index query fetches this version
@@ -532,7 +533,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
 
         // now apply another update, but force the index update to be skipped
         rm = new RowMutation(keySpace, rowKey);
-        rm.add(new QueryPath(cfName, null, compositeName), val2, 1);
+        rm.add(cfName, compositeName, val2, 1);
         table.apply(rm, true, false);
 
         // Now searching the index for either the old or new value should return 0 rows
@@ -552,7 +553,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         // now, reset back to the original value, still skipping the index update, to
         // make sure the value was expunged from the index when it was discovered to be inconsistent
         rm = new RowMutation(keySpace, rowKey);
-        rm.add(new QueryPath(cfName, null , compositeName), val1, 2);
+        rm.add(cfName, compositeName, val1, 2);
         table.apply(rm, true, false);
 
         expr = new IndexExpression(colName, IndexOperator.EQ, val1);
@@ -570,23 +571,23 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         RowMutation rm;
 
         rm = new RowMutation("Keyspace1", ByteBufferUtil.bytes("kk1"));
-        rm.add(new QueryPath("Indexed1", null, ByteBufferUtil.bytes("notbirthdate")), ByteBufferUtil.bytes(1L), 0);
-        rm.add(new QueryPath("Indexed1", null, ByteBufferUtil.bytes("birthdate")), ByteBufferUtil.bytes(1L), 0);
+        rm.add("Indexed1", ByteBufferUtil.bytes("notbirthdate"), ByteBufferUtil.bytes(1L), 0);
+        rm.add("Indexed1", ByteBufferUtil.bytes("birthdate"), ByteBufferUtil.bytes(1L), 0);
         rm.apply();
 
         rm = new RowMutation("Keyspace1", ByteBufferUtil.bytes("kk2"));
-        rm.add(new QueryPath("Indexed1", null, ByteBufferUtil.bytes("notbirthdate")), ByteBufferUtil.bytes(2L), 0);
-        rm.add(new QueryPath("Indexed1", null, ByteBufferUtil.bytes("birthdate")), ByteBufferUtil.bytes(1L), 0);
+        rm.add("Indexed1", ByteBufferUtil.bytes("notbirthdate"), ByteBufferUtil.bytes(2L), 0);
+        rm.add("Indexed1", ByteBufferUtil.bytes("birthdate"), ByteBufferUtil.bytes(1L), 0);
         rm.apply();
 
         rm = new RowMutation("Keyspace1", ByteBufferUtil.bytes("kk3"));
-        rm.add(new QueryPath("Indexed1", null, ByteBufferUtil.bytes("notbirthdate")), ByteBufferUtil.bytes(2L), 0);
-        rm.add(new QueryPath("Indexed1", null, ByteBufferUtil.bytes("birthdate")), ByteBufferUtil.bytes(1L), 0);
+        rm.add("Indexed1", ByteBufferUtil.bytes("notbirthdate"), ByteBufferUtil.bytes(2L), 0);
+        rm.add("Indexed1", ByteBufferUtil.bytes("birthdate"), ByteBufferUtil.bytes(1L), 0);
         rm.apply();
 
         rm = new RowMutation("Keyspace1", ByteBufferUtil.bytes("kk4"));
-        rm.add(new QueryPath("Indexed1", null, ByteBufferUtil.bytes("notbirthdate")), ByteBufferUtil.bytes(2L), 0);
-        rm.add(new QueryPath("Indexed1", null, ByteBufferUtil.bytes("birthdate")), ByteBufferUtil.bytes(1L), 0);
+        rm.add("Indexed1", ByteBufferUtil.bytes("notbirthdate"), ByteBufferUtil.bytes(2L), 0);
+        rm.add("Indexed1", ByteBufferUtil.bytes("birthdate"), ByteBufferUtil.bytes(1L), 0);
         rm.apply();
 
         // basic single-expression query
@@ -610,7 +611,7 @@ public class ColumnFamilyStoreTest 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("k1"));
-        rm.add(new QueryPath("Indexed2", null, ByteBufferUtil.bytes("birthdate")), ByteBufferUtil.bytes(1L), 1);
+        rm.add("Indexed2", ByteBufferUtil.bytes("birthdate"), ByteBufferUtil.bytes(1L), 1);
         rm.apply();
 
         ColumnFamilyStore cfs = table.getColumnFamilyStore("Indexed2");
@@ -651,8 +652,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         ColumnFamilyStore cfs = insertKey1Key2();
 
         IPartitioner p = StorageService.getPartitioner();
-        List<Row> result = cfs.getRangeSlice(ByteBufferUtil.EMPTY_BYTE_BUFFER,
-                                             Util.bounds("key1", "key2"),
+        List<Row> result = cfs.getRangeSlice(Util.bounds("key1", "key2"),
                                              10,
                                              new NamesQueryFilter(ByteBufferUtil.bytes("asdf")),
                                              null);
@@ -690,21 +690,21 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         sp.getSlice_range().setStart(ArrayUtils.EMPTY_BYTE_ARRAY);
         sp.getSlice_range().setFinish(ArrayUtils.EMPTY_BYTE_ARRAY);
 
-        assertRowAndColCount(1, 6, scfName, false, cfs.getRangeSlice(scfName, Util.range("f", "g"), 100, ThriftValidation.asIFilter(sp, cfs.getComparator()), null));
+        assertRowAndColCount(1, 6, scfName, false, cfs.getRangeSlice(Util.range("f", "g"), 100, ThriftValidation.asIFilter(sp, cfs.metadata, scfName), null));
 
         // deeleet.
         RowMutation rm = new RowMutation(table.getName(), key.key);
-        rm.delete(new QueryPath(cfName, scfName), 2);
+        rm.deleteRange(cfName, SuperColumns.startOf(scfName), SuperColumns.endOf(scfName), 2);
         rm.apply();
 
         // verify delete.
-        assertRowAndColCount(1, 0, scfName, false, cfs.getRangeSlice(scfName, Util.range("f", "g"), 100, ThriftValidation.asIFilter(sp, cfs.getComparator()), null));
+        assertRowAndColCount(1, 0, scfName, false, cfs.getRangeSlice(Util.range("f", "g"), 100, ThriftValidation.asIFilter(sp, cfs.metadata, scfName), null));
 
         // flush
         cfs.forceBlockingFlush();
 
         // re-verify delete.
-        assertRowAndColCount(1, 0, scfName, false, cfs.getRangeSlice(scfName, Util.range("f", "g"), 100, ThriftValidation.asIFilter(sp, cfs.getComparator()), null));
+        assertRowAndColCount(1, 0, scfName, false, cfs.getRangeSlice(Util.range("f", "g"), 100, ThriftValidation.asIFilter(sp, cfs.metadata, scfName), null));
 
         // late insert.
         putColsSuper(cfs, key, scfName,
@@ -712,14 +712,14 @@ public class ColumnFamilyStoreTest extends SchemaLoader
                 new Column(getBytes(7L), ByteBufferUtil.bytes("val7"), 1L));
 
         // re-verify delete.
-        assertRowAndColCount(1, 0, scfName, false, cfs.getRangeSlice(scfName, Util.range("f", "g"), 100, ThriftValidation.asIFilter(sp, cfs.getComparator()), null));
+        assertRowAndColCount(1, 0, scfName, false, cfs.getRangeSlice(Util.range("f", "g"), 100, ThriftValidation.asIFilter(sp, cfs.metadata, scfName), null));
 
         // 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));
-        assertRowAndColCount(1, 3, scfName, false, cfs.getRangeSlice(scfName, Util.range("f", "g"), 100, ThriftValidation.asIFilter(sp, cfs.getComparator()), null));
+        assertRowAndColCount(1, 3, scfName, false, cfs.getRangeSlice(Util.range("f", "g"), 100, ThriftValidation.asIFilter(sp, cfs.metadata, scfName), null));
     }
 
     private static void assertRowAndColCount(int rowCount, int colCount, ByteBuffer sc, boolean isDeleted, Collection<Row> rows) throws CharacterCodingException
@@ -728,10 +728,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         for (Row row : rows)
         {
             assert row.cf != null : "cf was null";
-            if (sc != null)
-                assert row.cf.getColumn(sc).getSubColumns().size() == colCount : row.cf.getColumn(sc).getSubColumns().size();
-            else
-                assert row.cf.getColumnCount() == colCount : "colcount " + row.cf.getColumnCount() + "|" + str(row.cf);
+            assert row.cf.getColumnCount() == colCount : "colcount " + row.cf.getColumnCount() + "|" + str(row.cf);
             if (isDeleted)
                 assert row.cf.isMarkedForDelete() : "cf not marked for delete";
         }
@@ -740,7 +737,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
     private static String str(ColumnFamily cf) throws CharacterCodingException
     {
         StringBuilder sb = new StringBuilder();
-        for (IColumn col : cf.getSortedColumns())
+        for (Column col : cf.getSortedColumns())
             sb.append(String.format("(%s,%s,%d),", ByteBufferUtil.string(col.name()), ByteBufferUtil.string(col.value()), col.timestamp()));
         return sb.toString();
     }
@@ -749,10 +746,8 @@ public class ColumnFamilyStoreTest extends SchemaLoader
     {
         RowMutation rm = new RowMutation(cfs.table.getName(), key.key);
         ColumnFamily cf = ColumnFamily.create(cfs.table.getName(), cfs.name);
-        SuperColumn sc = new SuperColumn(scfName, cfs.metadata.subcolumnComparator);
         for (Column col : cols)
-            sc.addColumn(col);
-        cf.addColumn(sc);
+            cf.addColumn(col.withUpdatedName(CompositeType.build(scfName, col.name())));
         rm.add(cf);
         rm.apply();
     }
@@ -785,42 +780,42 @@ public class ColumnFamilyStoreTest extends SchemaLoader
 
         // insert
         putColsStandard(cfs, key, column("col1", "val1", 1), column("col2", "val2", 1));
-        assertRowAndColCount(1, 2, null, false, cfs.getRangeSlice(null, Util.range("f", "g"), 100, ThriftValidation.asIFilter(sp, cfs.getComparator()), null));
+        assertRowAndColCount(1, 2, null, false, cfs.getRangeSlice(Util.range("f", "g"), 100, ThriftValidation.asIFilter(sp, cfs.metadata, null), null));
 
         // flush.
         cfs.forceBlockingFlush();
 
         // insert, don't flush
         putColsStandard(cfs, key, column("col3", "val3", 1), column("col4", "val4", 1));
-        assertRowAndColCount(1, 4, null, false, cfs.getRangeSlice(null, Util.range("f", "g"), 100, ThriftValidation.asIFilter(sp, cfs.getComparator()), null));
+        assertRowAndColCount(1, 4, null, false, cfs.getRangeSlice(Util.range("f", "g"), 100, ThriftValidation.asIFilter(sp, cfs.metadata, null), null));
 
         // delete (from sstable and memtable)
         RowMutation rm = new RowMutation(table.getName(), key.key);
-        rm.delete(new QueryPath(cfs.name, null, null), 2);
+        rm.delete(cfs.name, 2);
         rm.apply();
 
         // verify delete
-        assertRowAndColCount(1, 0, null, true, cfs.getRangeSlice(null, Util.range("f", "g"), 100, ThriftValidation.asIFilter(sp, cfs.getComparator()), null));
+        assertRowAndColCount(1, 0, null, true, cfs.getRangeSlice(Util.range("f", "g"), 100, ThriftValidation.asIFilter(sp, cfs.metadata, null), null));
 
         // flush
         cfs.forceBlockingFlush();
 
         // re-verify delete. // first breakage is right here because of CASSANDRA-1837.
-        assertRowAndColCount(1, 0, null, true, cfs.getRangeSlice(null, Util.range("f", "g"), 100, ThriftValidation.asIFilter(sp, cfs.getComparator()), null));
+        assertRowAndColCount(1, 0, null, true, cfs.getRangeSlice(Util.range("f", "g"), 100, ThriftValidation.asIFilter(sp, cfs.metadata, null), null));
 
         // simulate a 'late' insertion that gets put in after the deletion. should get inserted, but fail on read.
         putColsStandard(cfs, key, column("col5", "val5", 1), column("col2", "val2", 1));
 
         // should still be nothing there because we deleted this row. 2nd breakage, but was undetected because of 1837.
-        assertRowAndColCount(1, 0, null, true, cfs.getRangeSlice(null, Util.range("f", "g"), 100, ThriftValidation.asIFilter(sp, cfs.getComparator()), null));
+        assertRowAndColCount(1, 0, null, true, cfs.getRangeSlice(Util.range("f", "g"), 100, ThriftValidation.asIFilter(sp, cfs.metadata, null), null));
 
         // make sure that new writes are recognized.
         putColsStandard(cfs, key, column("col6", "val6", 3), column("col7", "val7", 3));
-        assertRowAndColCount(1, 2, null, true, cfs.getRangeSlice(null, Util.range("f", "g"), 100, ThriftValidation.asIFilter(sp, cfs.getComparator()), null));
+        assertRowAndColCount(1, 2, null, true, cfs.getRangeSlice(Util.range("f", "g"), 100, ThriftValidation.asIFilter(sp, cfs.metadata, null), null));
 
         // and it remains so after flush. (this wasn't failing before, but it's good to check.)
         cfs.forceBlockingFlush();
-        assertRowAndColCount(1, 2, null, true, cfs.getRangeSlice(null, Util.range("f", "g"), 100, ThriftValidation.asIFilter(sp, cfs.getComparator()), null));
+        assertRowAndColCount(1, 2, null, true, cfs.getRangeSlice(Util.range("f", "g"), 100, ThriftValidation.asIFilter(sp, cfs.metadata, null), null));
     }
 
 
@@ -829,12 +824,12 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         List<IMutation> rms = new LinkedList<IMutation>();
         RowMutation rm;
         rm = new RowMutation("Keyspace2", ByteBufferUtil.bytes("key1"));
-        rm.add(new QueryPath("Standard1", null, ByteBufferUtil.bytes("Column1")), ByteBufferUtil.bytes("asdf"), 0);
+        rm.add("Standard1", ByteBufferUtil.bytes("Column1"), ByteBufferUtil.bytes("asdf"), 0);
         rms.add(rm);
         Util.writeColumnFamily(rms);
 
         rm = new RowMutation("Keyspace2", ByteBufferUtil.bytes("key2"));
-        rm.add(new QueryPath("Standard1", null, ByteBufferUtil.bytes("Column1")), ByteBufferUtil.bytes("asdf"), 0);
+        rm.add("Standard1", ByteBufferUtil.bytes("Column1"), ByteBufferUtil.bytes("asdf"), 0);
         rms.add(rm);
         return Util.writeColumnFamily(rms);
     }
@@ -853,30 +848,6 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         }
     }
 
-    @Test
-    public void testSuperSliceByNamesCommand() throws Throwable
-    {
-        String tableName = "Keyspace1";
-        String cfName= "Super4";
-        ByteBuffer superColName = ByteBufferUtil.bytes("HerpDerp");
-        DecoratedKey key = Util.dk("multiget-slice-resurrection");
-        Table table = Table.open(tableName);
-        ColumnFamilyStore cfs = table.getColumnFamilyStore(cfName);
-
-        // Initially create a SC with 1 subcolumn
-        putColsSuper(cfs, key, superColName, new Column(ByteBufferUtil.bytes("c1"), ByteBufferUtil.bytes("a"), 1));
-        cfs.forceBlockingFlush();
-
-        // Add another column
-        putColsSuper(cfs, key, superColName, new Column(ByteBufferUtil.bytes("c2"), ByteBufferUtil.bytes("b"), 2));
-
-        // Test fetching the supercolumn by name
-        SliceByNamesReadCommand cmd = new SliceByNamesReadCommand(tableName, key.key, new QueryPath(cfName), Collections.singletonList(superColName));
-        ColumnFamily cf = cmd.getRow(table).cf;
-        SuperColumn superColumn = (SuperColumn) cf.getColumn(superColName);
-        assertColumns(superColumn, "c1", "c2");
-    }
-
     // CASSANDRA-3467.  the key here is that supercolumn and subcolumn comparators are different
     @Test
     public void testSliceByNamesCommandOnUUIDTypeSCF() throws Throwable
@@ -893,20 +864,20 @@ public class ColumnFamilyStoreTest extends SchemaLoader
                                              new Column(ByteBufferUtil.bytes("b"), ByteBufferUtil.bytes("B"), 1));
 
         // Get the entire supercolumn like normal
-        IColumn columnGet = cfs.getColumnFamily(QueryFilter.getIdentityFilter(key, new QueryPath(cfName, superColName))).getColumn(superColName);
-        assertEquals(ByteBufferUtil.bytes("A"), columnGet.getSubColumn(ByteBufferUtil.bytes("a")).value());
-        assertEquals(ByteBufferUtil.bytes("B"), columnGet.getSubColumn(ByteBufferUtil.bytes("b")).value());
+        ColumnFamily cfGet = cfs.getColumnFamily(QueryFilter.getIdentityFilter(key, cfName));
+        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());
 
         // Now do the SliceByNamesCommand on the supercolumn, passing both subcolumns in as columns to get
-        ArrayList<ByteBuffer> sliceColNames = new ArrayList<ByteBuffer>();
-        sliceColNames.add(ByteBufferUtil.bytes("a"));
-        sliceColNames.add(ByteBufferUtil.bytes("b"));
-        SliceByNamesReadCommand cmd = new SliceByNamesReadCommand(tableName, key.key, new QueryPath(cfName, superColName), sliceColNames);
-        IColumn columnSliced = cmd.getRow(table).cf.getColumn(superColName);
+        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")));
+        SliceByNamesReadCommand cmd = new SliceByNamesReadCommand(tableName, key.key, cfName, new NamesQueryFilter(sliceColNames));
+        ColumnFamily cfSliced = cmd.getRow(table).cf;
 
         // Make sure the slice returns the same as the straight get
-        assertEquals(ByteBufferUtil.bytes("A"), columnSliced.getSubColumn(ByteBufferUtil.bytes("a")).value());
-        assertEquals(ByteBufferUtil.bytes("B"), columnSliced.getSubColumn(ByteBufferUtil.bytes("b")).value());
+        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());
     }
 
     @Test
@@ -937,7 +908,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(tableName, key.key, new QueryPath(cfName), Collections.singletonList(cname));
+        SliceByNamesReadCommand cmd = new SliceByNamesReadCommand(tableName, key.key, cfName, new NamesQueryFilter(cname));
         ColumnFamily cf = cmd.getRow(table).cf;
         Column column = (Column) cf.getColumn(cname);
         assert column.value().equals(ByteBufferUtil.bytes("a")) : "expecting a, got " + ByteBufferUtil.string(column.value());
@@ -953,6 +924,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         assert columns == expectedCount : "Expected " + expectedCount + " live columns but got " + columns + ": " + rows;
     }
 
+
     @Test
     public void testRangeSliceColumnsLimit() throws Throwable
     {
@@ -977,11 +949,11 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         sp.getSlice_range().setStart(ArrayUtils.EMPTY_BYTE_ARRAY);
         sp.getSlice_range().setFinish(ArrayUtils.EMPTY_BYTE_ARRAY);
 
-        assertTotalColCount(cfs.getRangeSlice(null, Util.range("", ""), 3, ThriftValidation.asIFilter(sp, cfs.getComparator()), null, true, false), 3);
-        assertTotalColCount(cfs.getRangeSlice(null, Util.range("", ""), 5, ThriftValidation.asIFilter(sp, cfs.getComparator()), null, true, false), 5);
-        assertTotalColCount(cfs.getRangeSlice(null, Util.range("", ""), 8, ThriftValidation.asIFilter(sp, cfs.getComparator()), null, true, false), 8);
-        assertTotalColCount(cfs.getRangeSlice(null, Util.range("", ""), 10, ThriftValidation.asIFilter(sp, cfs.getComparator()), null, true, false), 10);
-        assertTotalColCount(cfs.getRangeSlice(null, Util.range("", ""), 100, ThriftValidation.asIFilter(sp, cfs.getComparator()), null, true, false), 11);
+        assertTotalColCount(cfs.getRangeSlice(Util.range("", ""), 3, ThriftValidation.asIFilter(sp, cfs.metadata, null), null, true, false), 3);
+        assertTotalColCount(cfs.getRangeSlice(Util.range("", ""), 5, ThriftValidation.asIFilter(sp, cfs.metadata, null), null, true, false), 5);
+        assertTotalColCount(cfs.getRangeSlice(Util.range("", ""), 8, ThriftValidation.asIFilter(sp, cfs.metadata, null), null, true, false), 8);
+        assertTotalColCount(cfs.getRangeSlice(Util.range("", ""), 10, ThriftValidation.asIFilter(sp, cfs.metadata, null), null, true, false), 10);
+        assertTotalColCount(cfs.getRangeSlice(Util.range("", ""), 100, ThriftValidation.asIFilter(sp, cfs.metadata, null), null, true, false), 11);
 
         // Check that when querying by name, we always include all names for a
         // gien row even if it means returning more columns than requested (this is necesseray for CQL)
@@ -992,11 +964,11 @@ public class ColumnFamilyStoreTest extends SchemaLoader
             ByteBufferUtil.bytes("c2")
         ));
 
-        assertTotalColCount(cfs.getRangeSlice(null, Util.range("", ""), 1, ThriftValidation.asIFilter(sp, cfs.getComparator()), null, true, false), 3);
-        assertTotalColCount(cfs.getRangeSlice(null, Util.range("", ""), 4, ThriftValidation.asIFilter(sp, cfs.getComparator()), null, true, false), 5);
-        assertTotalColCount(cfs.getRangeSlice(null, Util.range("", ""), 5, ThriftValidation.asIFilter(sp, cfs.getComparator()), null, true, false), 5);
-        assertTotalColCount(cfs.getRangeSlice(null, Util.range("", ""), 6, ThriftValidation.asIFilter(sp, cfs.getComparator()), null, true, false), 8);
-        assertTotalColCount(cfs.getRangeSlice(null, Util.range("", ""), 100, ThriftValidation.asIFilter(sp, cfs.getComparator()), null, true, false), 8);
+        assertTotalColCount(cfs.getRangeSlice(Util.range("", ""), 1, ThriftValidation.asIFilter(sp, cfs.metadata, null), null, true, false), 3);
+        assertTotalColCount(cfs.getRangeSlice(Util.range("", ""), 4, ThriftValidation.asIFilter(sp, cfs.metadata, null), null, true, false), 5);
+        assertTotalColCount(cfs.getRangeSlice(Util.range("", ""), 5, ThriftValidation.asIFilter(sp, cfs.metadata, null), null, true, false), 5);
+        assertTotalColCount(cfs.getRangeSlice(Util.range("", ""), 6, ThriftValidation.asIFilter(sp, cfs.metadata, null), null, true, false), 8);
+        assertTotalColCount(cfs.getRangeSlice(Util.range("", ""), 100, ThriftValidation.asIFilter(sp, cfs.metadata, null), null, true, false), 8);
     }
 
     @Test
@@ -1023,13 +995,13 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         sp.getSlice_range().setStart(ArrayUtils.EMPTY_BYTE_ARRAY);
         sp.getSlice_range().setFinish(ArrayUtils.EMPTY_BYTE_ARRAY);
 
-        Collection<Row> rows = cfs.getRangeSlice(null, Util.range("", ""), 3, ThriftValidation.asIFilter(sp, cfs.getComparator()), null, true, true);
+        Collection<Row> rows = cfs.getRangeSlice(Util.range("", ""), 3, ThriftValidation.asIFilter(sp, cfs.metadata, null), null, true, true);
         assert rows.size() == 1 : "Expected 1 row, got " + rows;
         Row row = rows.iterator().next();
         assertColumnNames(row, "c0", "c1", "c2");
 
         sp.getSlice_range().setStart(ByteBufferUtil.getArray(ByteBufferUtil.bytes("c2")));
-        rows = cfs.getRangeSlice(null, Util.range("", ""), 3, ThriftValidation.asIFilter(sp, cfs.getComparator()), null, true, true);
+        rows = cfs.getRangeSlice(Util.range("", ""), 3, ThriftValidation.asIFilter(sp, cfs.metadata, null), null, true, true);
         assert rows.size() == 2 : "Expected 2 rows, got " + rows;
         Iterator<Row> iter = rows.iterator();
         Row row1 = iter.next();
@@ -1038,13 +1010,13 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         assertColumnNames(row2, "c0");
 
         sp.getSlice_range().setStart(ByteBufferUtil.getArray(ByteBufferUtil.bytes("c0")));
-        rows = cfs.getRangeSlice(null, new Bounds<RowPosition>(row2.key, Util.rp("")), 3, ThriftValidation.asIFilter(sp, cfs.getComparator()), null, true, true);
+        rows = cfs.getRangeSlice(new Bounds<RowPosition>(row2.key, Util.rp("")), 3, ThriftValidation.asIFilter(sp, cfs.metadata, null), null, true, true);
         assert rows.size() == 1 : "Expected 1 row, got " + rows;
         row = rows.iterator().next();
         assertColumnNames(row, "c0", "c1", "c2");
 
         sp.getSlice_range().setStart(ByteBufferUtil.getArray(ByteBufferUtil.bytes("c2")));
-        rows = cfs.getRangeSlice(null, new Bounds<RowPosition>(row.key, Util.rp("")), 3, ThriftValidation.asIFilter(sp, cfs.getComparator()), null, true, true);
+        rows = cfs.getRangeSlice(new Bounds<RowPosition>(row.key, Util.rp("")), 3, ThriftValidation.asIFilter(sp, cfs.metadata, null), null, true, true);
         assert rows.size() == 2 : "Expected 2 rows, got " + rows;
         iter = rows.iterator();
         row1 = iter.next();
@@ -1058,12 +1030,12 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         if (row == null || row.cf == null)
             throw new AssertionError("The row should not be empty");
 
-        Iterator<IColumn> columns = row.cf.getSortedColumns().iterator();
+        Iterator<Column> columns = row.cf.getSortedColumns().iterator();
         Iterator<String> names = Arrays.asList(columnNames).iterator();
 
         while (columns.hasNext())
         {
-            IColumn c = columns.next();
+            Column c = columns.next();
             assert names.hasNext() : "Got more columns that expected (first unexpected column: " + ByteBufferUtil.string(c.name()) + ")";
             String n = names.next();
             assert c.name().equals(ByteBufferUtil.bytes(n)) : "Expected " + n + ", got " + ByteBufferUtil.string(c.name());
@@ -1100,30 +1072,30 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         sp.getSlice_range().setCount(1);
         sp.getSlice_range().setStart(ArrayUtils.EMPTY_BYTE_ARRAY);
         sp.getSlice_range().setFinish(ArrayUtils.EMPTY_BYTE_ARRAY);
-        IDiskAtomFilter qf = ThriftValidation.asIFilter(sp, cfs.getComparator());
+        IDiskAtomFilter qf = ThriftValidation.asIFilter(sp, cfs.metadata, null);
 
         List<Row> rows;
 
         // Start and end inclusive
-        rows = cfs.getRangeSlice(null, new Bounds<RowPosition>(rp("2"), rp("7")), 100, qf, null);
+        rows = cfs.getRangeSlice(new Bounds<RowPosition>(rp("2"), rp("7")), 100, qf, null);
         assert rows.size() == 6;
         assert rows.get(0).key.equals(idk(2));
         assert rows.get(rows.size() - 1).key.equals(idk(7));
 
         // Start and end excluded
-        rows = cfs.getRangeSlice(null, new ExcludingBounds<RowPosition>(rp("2"), rp("7")), 100, qf, null);
+        rows = cfs.getRangeSlice(new ExcludingBounds<RowPosition>(rp("2"), rp("7")), 100, qf, null);
         assert rows.size() == 4;
         assert rows.get(0).key.equals(idk(3));
         assert rows.get(rows.size() - 1).key.equals(idk(6));
 
         // Start excluded, end included
-        rows = cfs.getRangeSlice(null, new Range<RowPosition>(rp("2"), rp("7")), 100, qf, null);
+        rows = cfs.getRangeSlice(new Range<RowPosition>(rp("2"), rp("7")), 100, qf, null);
         assert rows.size() == 5;
         assert rows.get(0).key.equals(idk(3));
         assert rows.get(rows.size() - 1).key.equals(idk(7));
 
         // Start included, end excluded
-        rows = cfs.getRangeSlice(null, new IncludingExcludingBounds<RowPosition>(rp("2"), rp("7")), 100, qf, null);
+        rows = cfs.getRangeSlice(new IncludingExcludingBounds<RowPosition>(rp("2"), rp("7")), 100, qf, null);
         assert rows.size() == 5;
         assert rows.get(0).key.equals(idk(2));
         assert rows.get(rows.size() - 1).key.equals(idk(6));
@@ -1142,11 +1114,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         {
             ByteBuffer key = ByteBufferUtil.bytes(String.valueOf("k" + i));
             RowMutation rm = new RowMutation("Keyspace1", key);
-
-            rm.add(new QueryPath("Indexed1", null, ByteBufferUtil.bytes("birthdate")),
-                   LongType.instance.decompose(1L),
-                   System.currentTimeMillis());
-
+            rm.add("Indexed1", ByteBufferUtil.bytes("birthdate"), LongType.instance.decompose(1L), System.currentTimeMillis());
             rm.apply();
         }
 
@@ -1482,16 +1450,16 @@ public class ColumnFamilyStoreTest extends SchemaLoader
     private void findRowGetSlicesAndAssertColsFound(ColumnFamilyStore cfs, SliceQueryFilter filter, String rowKey,
             String... colNames)
     {
-        List<Row> rows = cfs.getRangeSlice(null, new Bounds<RowPosition>(rp(rowKey), rp(rowKey)),
+        List<Row> rows = cfs.getRangeSlice(new Bounds<RowPosition>(rp(rowKey), rp(rowKey)),
                 Integer.MAX_VALUE,
                 filter, null, false, false);
         assertSame("unexpected number of rows ", 1, rows.size());
         Row row = rows.get(0);
-        Collection<IColumn> cols = !filter.isReversed() ? row.cf.getSortedColumns() : row.cf.getReverseSortedColumns();
+        Collection<Column> cols = !filter.isReversed() ? row.cf.getSortedColumns() : row.cf.getReverseSortedColumns();
         // printRow(cfs, new String(row.key.key.array()), cols);
-        String[] returnedColsNames = Iterables.toArray(Iterables.transform(cols, new Function<IColumn, String>()
+        String[] returnedColsNames = Iterables.toArray(Iterables.transform(cols, new Function<Column, String>()
         {
-            public String apply(IColumn arg0)
+            public String apply(Column arg0)
             {
                 return new String(arg0.name().array());
             }
@@ -1501,29 +1469,29 @@ public class ColumnFamilyStoreTest extends SchemaLoader
                 "Columns did not match. Expected: " + Arrays.toString(colNames) + " but got:"
                         + Arrays.toString(returnedColsNames), Arrays.equals(colNames, returnedColsNames));
         int i = 0;
-        for (IColumn col : cols)
+        for (Column col : cols)
         {
             assertEquals(colNames[i++], new String(col.name().array()));
         }
     }
 
-    private void printRow(ColumnFamilyStore cfs, String rowKey, Collection<IColumn> cols)
+    private void printRow(ColumnFamilyStore cfs, String rowKey, Collection<Column> cols)
     {
         DecoratedKey ROW = Util.dk(rowKey);
         System.err.println("Original:");
-        ColumnFamily cf = cfs.getColumnFamily(QueryFilter.getIdentityFilter(ROW, new QueryPath("Standard1")));
+        ColumnFamily cf = cfs.getColumnFamily(QueryFilter.getIdentityFilter(ROW, "Standard1"));
         System.err.println("Row key: " + rowKey + " Cols: "
-                + Iterables.transform(cf.getSortedColumns(), new Function<IColumn, String>()
+                + Iterables.transform(cf.getSortedColumns(), new Function<Column, String>()
                 {
-                    public String apply(IColumn arg0)
+                    public String apply(Column arg0)
                     {
                         return new String(arg0.name().array());
                     }
                 }));
         System.err.println("Filtered:");
-        Iterable<String> transformed = Iterables.transform(cols, new Function<IColumn, String>()
+        Iterable<String> transformed = Iterables.transform(cols, new Function<Column, String>()
         {
-            public String apply(IColumn arg0)
+            public String apply(Column arg0)
             {
                 return new String(arg0.name().array());
             }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a005df3/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 1b41958..21fa727 100644
--- a/test/unit/org/apache/cassandra/db/ColumnFamilyTest.java
+++ b/test/unit/org/apache/cassandra/db/ColumnFamilyTest.java
@@ -29,7 +29,6 @@ import org.junit.Test;
 
 import org.apache.cassandra.io.sstable.ColumnStats;
 import org.apache.cassandra.io.util.DataOutputBuffer;
-import org.apache.cassandra.db.filter.QueryPath;
 import org.apache.cassandra.net.MessagingService;
 import static org.apache.cassandra.Util.column;
 import static org.junit.Assert.assertEquals;
@@ -126,12 +125,11 @@ public class ColumnFamilyTest extends SchemaLoader
         ByteBuffer val = ByteBufferUtil.bytes("sample value");
         ByteBuffer val2 = ByteBufferUtil.bytes("x value ");
 
-        // exercise addColumn(QueryPath, ...)
-        cf_new.addColumn(QueryPath.column(ByteBufferUtil.bytes("col1")), val, 3);
-        cf_new.addColumn(QueryPath.column(ByteBufferUtil.bytes("col2")), val, 4);
+        cf_new.addColumn(ByteBufferUtil.bytes("col1"), val, 3);
+        cf_new.addColumn(ByteBufferUtil.bytes("col2"), val, 4);
 
-        cf_old.addColumn(QueryPath.column(ByteBufferUtil.bytes("col2")), val2, 1);
-        cf_old.addColumn(QueryPath.column(ByteBufferUtil.bytes("col3")), val2, 2);
+        cf_old.addColumn(ByteBufferUtil.bytes("col2"), val2, 1);
+        cf_old.addColumn(ByteBufferUtil.bytes("col3"), val2, 2);
 
         cf_result.addAll(cf_new, HeapAllocator.instance);
         cf_result.addAll(cf_old, HeapAllocator.instance);
@@ -143,46 +141,16 @@ public class ColumnFamilyTest extends SchemaLoader
         // check that tombstone wins timestamp ties
         cf_result.addTombstone(ByteBufferUtil.bytes("col1"), 0, 3);
         assert cf_result.getColumn(ByteBufferUtil.bytes("col1")).isMarkedForDelete();
-        cf_result.addColumn(QueryPath.column(ByteBufferUtil.bytes("col1")), val2, 3);
+        cf_result.addColumn(ByteBufferUtil.bytes("col1"), val2, 3);
         assert cf_result.getColumn(ByteBufferUtil.bytes("col1")).isMarkedForDelete();
 
         // check that column value wins timestamp ties in absence of tombstone
-        cf_result.addColumn(QueryPath.column(ByteBufferUtil.bytes("col3")), val, 2);
+        cf_result.addColumn(ByteBufferUtil.bytes("col3"), val, 2);
         assert cf_result.getColumn(ByteBufferUtil.bytes("col3")).value().equals(val2);
-        cf_result.addColumn(QueryPath.column(ByteBufferUtil.bytes("col3")), ByteBufferUtil.bytes("z"), 2);
+        cf_result.addColumn(ByteBufferUtil.bytes("col3"), ByteBufferUtil.bytes("z"), 2);
         assert cf_result.getColumn(ByteBufferUtil.bytes("col3")).value().equals(ByteBufferUtil.bytes("z"));
     }
 
-    private void testSuperColumnResolution(ISortedColumns.Factory factory)
-    {
-        ColumnFamilyStore cfs = Table.open("Keyspace1").getColumnFamilyStore("Super1");
-        ColumnFamily cf = ColumnFamily.create(cfs.metadata, factory);
-        ByteBuffer superColumnName = ByteBufferUtil.bytes("sc");
-        ByteBuffer subColumnName = ByteBufferUtil.bytes(1L);
-
-        Column first = new Column(subColumnName, ByteBufferUtil.bytes("one"), 1L);
-        Column second = new Column(subColumnName, ByteBufferUtil.bytes("two"), 2L);
-
-        cf.addColumn(superColumnName, first);
-
-        // resolve older + new
-        cf.addColumn(superColumnName, second);
-        assertEquals(second, cf.getColumn(superColumnName).getSubColumn(subColumnName));
-
-        // resolve new + older
-        cf.addColumn(superColumnName, first);
-        assertEquals(second, cf.getColumn(superColumnName).getSubColumn(subColumnName));
-    }
-
-    @Test
-    public void testSuperColumnResolution()
-    {
-        testSuperColumnResolution(TreeMapBackedSortedColumns.factory());
-        testSuperColumnResolution(AtomicSortedColumns.factory());
-        // array-sorted does allow conflict resolution IF it is the last column.  Bit of an edge case.
-        testSuperColumnResolution(ArrayBackedSortedColumns.factory());
-    }
-    
     @Test
     public void testColumnStatsRecordsRowDeletesCorrectly() throws IOException
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a005df3/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 ef58aed..ee2e663 100644
--- a/test/unit/org/apache/cassandra/db/CommitLogTest.java
+++ b/test/unit/org/apache/cassandra/db/CommitLogTest.java
@@ -33,7 +33,6 @@ import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.commitlog.CommitLog;
 import org.apache.cassandra.db.commitlog.CommitLogDescriptor;
-import org.apache.cassandra.db.filter.QueryPath;
 import org.apache.cassandra.net.MessagingService;
 
 import static org.apache.cassandra.utils.ByteBufferUtil.bytes;
@@ -101,7 +100,7 @@ public class CommitLogTest extends SchemaLoader
         CommitLog.instance.resetUnsafe();
         // Roughly 32 MB mutation
         RowMutation rm = new RowMutation("Keyspace1", bytes("k"));
-        rm.add(new QueryPath("Standard1", null, bytes("c1")), ByteBuffer.allocate(DatabaseDescriptor.getCommitLogSegmentSize()/4), 0);
+        rm.add("Standard1", bytes("c1"), ByteBuffer.allocate(DatabaseDescriptor.getCommitLogSegmentSize()/4), 0);
 
         // Adding it 5 times
         CommitLog.instance.add(rm);
@@ -112,7 +111,7 @@ public class CommitLogTest extends SchemaLoader
 
         // Adding new mutation on another CF
         RowMutation rm2 = new RowMutation("Keyspace1", bytes("k"));
-        rm2.add(new QueryPath("Standard2", null, bytes("c1")), ByteBuffer.allocate(4), 0);
+        rm2.add("Standard2", bytes("c1"), ByteBuffer.allocate(4), 0);
         CommitLog.instance.add(rm2);
 
         assert CommitLog.instance.activeSegments() == 2 : "Expecting 2 segments, got " + CommitLog.instance.activeSegments();
@@ -130,7 +129,7 @@ public class CommitLogTest extends SchemaLoader
         CommitLog.instance.resetUnsafe();
         // Roughly 32 MB mutation
         RowMutation rm = new RowMutation("Keyspace1", bytes("k"));
-        rm.add(new QueryPath("Standard1", null, bytes("c1")), ByteBuffer.allocate(DatabaseDescriptor.getCommitLogSegmentSize()/4), 0);
+        rm.add("Standard1", bytes("c1"), ByteBuffer.allocate(DatabaseDescriptor.getCommitLogSegmentSize()/4), 0);
 
         // Adding it twice (won't change segment)
         CommitLog.instance.add(rm);
@@ -146,7 +145,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(new QueryPath("Standard2", null, bytes("c1")), ByteBuffer.allocate(DatabaseDescriptor.getCommitLogSegmentSize()/2), 0);
+        rm2.add("Standard2", bytes("c1"), ByteBuffer.allocate(DatabaseDescriptor.getCommitLogSegmentSize()/2), 0);
         CommitLog.instance.add(rm2);
         // also forces a new segment, since each entry-with-overhead is just over half the CL size
         CommitLog.instance.add(rm2);
@@ -171,7 +170,7 @@ public class CommitLogTest extends SchemaLoader
         CommitLog.instance.resetUnsafe();
 
         RowMutation rm = new RowMutation("Keyspace1", bytes("k"));
-        rm.add(new QueryPath("Standard1", null, bytes("c1")), ByteBuffer.allocate((DatabaseDescriptor.getCommitLogSegmentSize()) - 83), 0);
+        rm.add("Standard1", bytes("c1"), ByteBuffer.allocate((DatabaseDescriptor.getCommitLogSegmentSize()) - 83), 0);
         CommitLog.instance.add(rm);
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a005df3/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 d627139..c192623 100644
--- a/test/unit/org/apache/cassandra/db/CounterColumnTest.java
+++ b/test/unit/org/apache/cassandra/db/CounterColumnTest.java
@@ -33,7 +33,6 @@ import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
 import org.apache.cassandra.db.context.CounterContext;
 import static org.apache.cassandra.db.context.CounterContext.ContextState;
-import org.apache.cassandra.io.IColumnSerializer;
 import org.apache.cassandra.io.util.DataOutputBuffer;
 import org.apache.cassandra.utils.*;
 
@@ -74,9 +73,9 @@ public class CounterColumnTest extends SchemaLoader
     @Test
     public void testReconcile() throws UnknownHostException
     {
-        IColumn left;
-        IColumn right;
-        IColumn reconciled;
+        Column left;
+        Column right;
+        Column reconciled;
 
         ByteBuffer context;
 
@@ -286,7 +285,7 @@ public class CounterColumnTest extends SchemaLoader
         assert original.equals(deserialized);
 
         bufIn = new ByteArrayInputStream(serialized, 0, serialized.length);
-        CounterColumn deserializedOnRemote = (CounterColumn)Column.serializer().deserialize(new DataInputStream(bufIn), IColumnSerializer.Flag.FROM_REMOTE);
+        CounterColumn deserializedOnRemote = (CounterColumn)Column.serializer().deserialize(new DataInputStream(bufIn), ColumnSerializer.Flag.FROM_REMOTE);
         assert deserializedOnRemote.name().equals(original.name());
         assert deserializedOnRemote.total() == original.total();
         assert deserializedOnRemote.value().equals(cc.clearAllDelta(original.value()));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a005df3/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 10fafa5..e65fd85 100644
--- a/test/unit/org/apache/cassandra/db/CounterMutationTest.java
+++ b/test/unit/org/apache/cassandra/db/CounterMutationTest.java
@@ -44,7 +44,7 @@ public class CounterMutationTest extends SchemaLoader
         CounterId id1 = CounterId.getLocalId();
 
         rm = new RowMutation("Keyspace1", ByteBufferUtil.bytes("key1"));
-        rm.addCounter(new QueryPath("Counter1", null, ByteBufferUtil.bytes("Column1")), 3);
+        rm.addCounter("Counter1", ByteBufferUtil.bytes("Column1"), 3);
         cm = new CounterMutation(rm, ConsistencyLevel.ONE);
         cm.apply();
 
@@ -52,7 +52,7 @@ public class CounterMutationTest extends SchemaLoader
         CounterId id2 = CounterId.getLocalId();
 
         rm = new RowMutation("Keyspace1", ByteBufferUtil.bytes("key1"));
-        rm.addCounter(new QueryPath("Counter1", null, ByteBufferUtil.bytes("Column1")), 4);
+        rm.addCounter("Counter1", ByteBufferUtil.bytes("Column1"), 4);
         cm = new CounterMutation(rm, ConsistencyLevel.ONE);
         cm.apply();
 
@@ -60,8 +60,8 @@ public class CounterMutationTest extends SchemaLoader
         CounterId id3 = CounterId.getLocalId();
 
         rm = new RowMutation("Keyspace1", ByteBufferUtil.bytes("key1"));
-        rm.addCounter(new QueryPath("Counter1", null, ByteBufferUtil.bytes("Column1")), 5);
-        rm.addCounter(new QueryPath("Counter1", null, ByteBufferUtil.bytes("Column2")), 1);
+        rm.addCounter("Counter1", ByteBufferUtil.bytes("Column1"), 5);
+        rm.addCounter("Counter1", ByteBufferUtil.bytes("Column2"), 1);
         cm = new CounterMutation(rm, ConsistencyLevel.ONE);
         cm.apply();
 
@@ -71,7 +71,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();
-        IColumn c = cf.getColumn(ByteBufferUtil.bytes("Column1"));
+        Column c = cf.getColumn(ByteBufferUtil.bytes("Column1"));
         assert c != null;
         assert c instanceof CounterColumn;
         assert ((CounterColumn)c).total() == 12L;
@@ -101,40 +101,6 @@ public class CounterMutationTest extends SchemaLoader
     }
 
     @Test
-    public void testMutateSuperColumns() throws IOException
-    {
-        RowMutation rm;
-        CounterMutation cm;
-
-        rm = new RowMutation("Keyspace1", bytes("key1"));
-        rm.addCounter(new QueryPath("SuperCounter1", bytes("sc1"), bytes("Column1")), 1);
-        rm.addCounter(new QueryPath("SuperCounter1", bytes("sc2"), bytes("Column1")), 1);
-        cm = new CounterMutation(rm, ConsistencyLevel.ONE);
-        cm.apply();
-
-        rm = new RowMutation("Keyspace1", bytes("key1"));
-        rm.addCounter(new QueryPath("SuperCounter1", bytes("sc1"), bytes("Column2")), 1);
-        rm.addCounter(new QueryPath("SuperCounter1", bytes("sc2"), bytes("Column2")), 1);
-        cm = new CounterMutation(rm, ConsistencyLevel.ONE);
-        cm.apply();
-
-        RowMutation reprm = cm.makeReplicationMutation();
-        ColumnFamily cf = reprm.getColumnFamilies().iterator().next();
-
-        assert cf.getColumnCount() == 2;
-
-        IColumn sc1 = cf.getColumn(bytes("sc1"));
-        assert sc1 != null && sc1 instanceof SuperColumn;
-        assert sc1.getSubColumns().size() == 1;
-        assert sc1.getSubColumn(bytes("Column2")) != null;
-
-        IColumn sc2 = cf.getColumn(bytes("sc2"));
-        assert sc2 != null && sc2 instanceof SuperColumn;
-        assert sc2.getSubColumns().size() == 1;
-        assert sc2.getSubColumn(bytes("Column2")) != null;
-    }
-
-    @Test
     public void testGetOldShardFromSystemTable() throws IOException
     {
         // Renewing a bunch of times and checking we get the same thing from

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a005df3/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 0b04250..7fdfd3f 100644
--- a/test/unit/org/apache/cassandra/db/HintedHandOffTest.java
+++ b/test/unit/org/apache/cassandra/db/HintedHandOffTest.java
@@ -29,7 +29,6 @@ import org.junit.Test;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy;
-import org.apache.cassandra.db.filter.QueryPath;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 
@@ -56,11 +55,7 @@ public class HintedHandOffTest extends SchemaLoader
 
         // insert 1 hint
         RowMutation rm = new RowMutation(TABLE4, ByteBufferUtil.bytes(1));
-        rm.add(new QueryPath(STANDARD1_CF,
-                             null,
-                             ByteBufferUtil.bytes(String.valueOf(COLUMN1))),
-               ByteBufferUtil.EMPTY_BYTE_BUFFER,
-               System.currentTimeMillis());
+        rm.add(STANDARD1_CF, ByteBufferUtil.bytes(String.valueOf(COLUMN1)), ByteBufferUtil.EMPTY_BYTE_BUFFER, System.currentTimeMillis());
 
         RowMutation.hintFor(rm, UUID.randomUUID()).apply();
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a005df3/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 93f1fea..cc442f2 100644
--- a/test/unit/org/apache/cassandra/db/KeyCacheTest.java
+++ b/test/unit/org/apache/cassandra/db/KeyCacheTest.java
@@ -35,7 +35,6 @@ import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
-import org.apache.cassandra.db.filter.QueryPath;
 import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import static junit.framework.Assert.assertEquals;
@@ -106,10 +105,10 @@ public class KeyCacheTest extends SchemaLoader
 
         // inserts
         rm = new RowMutation(TABLE1, key1.key);
-        rm.add(new QueryPath(COLUMN_FAMILY1, null, ByteBufferUtil.bytes("1")), ByteBufferUtil.EMPTY_BYTE_BUFFER, 0);
+        rm.add(COLUMN_FAMILY1, ByteBufferUtil.bytes("1"), ByteBufferUtil.EMPTY_BYTE_BUFFER, 0);
         rm.apply();
         rm = new RowMutation(TABLE1, key2.key);
-        rm.add(new QueryPath(COLUMN_FAMILY1, null, ByteBufferUtil.bytes("2")), ByteBufferUtil.EMPTY_BYTE_BUFFER, 0);
+        rm.add(COLUMN_FAMILY1, ByteBufferUtil.bytes("2"), ByteBufferUtil.EMPTY_BYTE_BUFFER, 0);
         rm.apply();
 
         // to make sure we have SSTable
@@ -117,14 +116,14 @@ public class KeyCacheTest extends SchemaLoader
 
         // reads to cache key position
         cfs.getColumnFamily(QueryFilter.getSliceFilter(key1,
-                                                       new QueryPath(new ColumnParent(COLUMN_FAMILY1)),
+                                                       COLUMN_FAMILY1,
                                                        ByteBufferUtil.EMPTY_BYTE_BUFFER,
                                                        ByteBufferUtil.EMPTY_BYTE_BUFFER,
                                                        false,
                                                        10));
 
         cfs.getColumnFamily(QueryFilter.getSliceFilter(key2,
-                                                       new QueryPath(new ColumnParent(COLUMN_FAMILY1)),
+                                                       COLUMN_FAMILY1,
                                                        ByteBufferUtil.EMPTY_BYTE_BUFFER,
                                                        ByteBufferUtil.EMPTY_BYTE_BUFFER,
                                                        false,
@@ -140,14 +139,14 @@ public class KeyCacheTest extends SchemaLoader
 
         // re-read same keys to verify that key cache didn't grow further
         cfs.getColumnFamily(QueryFilter.getSliceFilter(key1,
-                                                       new QueryPath(new ColumnParent(COLUMN_FAMILY1)),
+                                                       COLUMN_FAMILY1,
                                                        ByteBufferUtil.EMPTY_BYTE_BUFFER,
                                                        ByteBufferUtil.EMPTY_BYTE_BUFFER,
                                                        false,
                                                        10));
 
         cfs.getColumnFamily(QueryFilter.getSliceFilter(key2,
-                                                       new QueryPath(new ColumnParent(COLUMN_FAMILY1)),
+                                                       COLUMN_FAMILY1,
                                                        ByteBufferUtil.EMPTY_BYTE_BUFFER,
                                                        ByteBufferUtil.EMPTY_BYTE_BUFFER,
                                                        false,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a005df3/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 e152a5d..ab792d5 100644
--- a/test/unit/org/apache/cassandra/db/KeyCollisionTest.java
+++ b/test/unit/org/apache/cassandra/db/KeyCollisionTest.java
@@ -27,7 +27,6 @@ import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.db.columniterator.IdentityQueryFilter;
-import org.apache.cassandra.db.filter.QueryPath;
 import org.apache.cassandra.dht.*;
 import org.apache.cassandra.config.*;
 import org.apache.cassandra.service.StorageService;
@@ -70,7 +69,7 @@ public class KeyCollisionTest extends SchemaLoader
         insert("key1", "key2", "key3"); // token = 4
         insert("longKey1", "longKey2"); // token = 8
 
-        List<Row> rows = cfs.getRangeSlice(null, new Bounds<RowPosition>(dk("k2"), dk("key2")), 10000, new IdentityQueryFilter(), null);
+        List<Row> rows = cfs.getRangeSlice(new Bounds<RowPosition>(dk("k2"), dk("key2")), 10000, new IdentityQueryFilter(), null);
         assert rows.size() == 4 : "Expecting 4 keys, got " + rows.size();
         assert rows.get(0).key.key.equals(ByteBufferUtil.bytes("k2"));
         assert rows.get(1).key.key.equals(ByteBufferUtil.bytes("k3"));
@@ -88,7 +87,7 @@ public class KeyCollisionTest extends SchemaLoader
     {
         RowMutation rm;
         rm = new RowMutation(KEYSPACE, ByteBufferUtil.bytes(key));
-        rm.add(new QueryPath(CF, null, ByteBufferUtil.bytes("column")), ByteBufferUtil.bytes("asdf"), 0);
+        rm.add(CF, ByteBufferUtil.bytes("column"), ByteBufferUtil.bytes("asdf"), 0);
         rm.apply();
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a005df3/test/unit/org/apache/cassandra/db/NameSortTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/NameSortTest.java b/test/unit/org/apache/cassandra/db/NameSortTest.java
index 668a2e2..97a4a8a 100644
--- a/test/unit/org/apache/cassandra/db/NameSortTest.java
+++ b/test/unit/org/apache/cassandra/db/NameSortTest.java
@@ -27,7 +27,6 @@ import java.util.concurrent.ExecutionException;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
-import org.apache.cassandra.db.filter.QueryPath;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 import org.junit.Test;
@@ -69,7 +68,7 @@ public class NameSortTest extends SchemaLoader
             {
                 ByteBuffer bytes = j % 2 == 0 ? ByteBufferUtil.bytes("a") : ByteBufferUtil.bytes("b");
                 rm = new RowMutation("Keyspace1", key);
-                rm.add(new QueryPath("Standard1", null, ByteBufferUtil.bytes(("Column-" + j))), bytes, j);
+                rm.add("Standard1", ByteBufferUtil.bytes(("Column-" + j)), bytes, j);
                 rm.applyUnsafe();
             }
 
@@ -101,32 +100,14 @@ public class NameSortTest extends SchemaLoader
             ColumnFamily cf;
 
             cf = Util.getColumnFamily(table, key, "Standard1");
-            Collection<IColumn> columns = cf.getSortedColumns();
-            for (IColumn column : columns)
+            Collection<Column> columns = cf.getSortedColumns();
+            for (Column column : columns)
             {
                 String name = ByteBufferUtil.string(column.name());
                 int j = Integer.valueOf(name.substring(name.length() - 1));
                 byte[] bytes = j % 2 == 0 ? "a".getBytes() : "b".getBytes();
                 assertEquals(new String(bytes), ByteBufferUtil.string(column.value()));
             }
-
-            cf = Util.getColumnFamily(table, key, "Super1");
-            assert cf != null : "key " + key + " is missing!";
-            Collection<IColumn> superColumns = cf.getSortedColumns();
-            assert superColumns.size() == 8 : cf;
-            for (IColumn superColumn : superColumns)
-            {
-                int j = Integer.valueOf(ByteBufferUtil.string(superColumn.name()).split("-")[1]);
-                Collection<IColumn> subColumns = superColumn.getSubColumns();
-                assert subColumns.size() == 4;
-                for (IColumn subColumn : subColumns)
-                {
-                    long k = subColumn.name().getLong(subColumn.name().position());
-                    byte[] bytes = (j + k) % 2 == 0 ? "a".getBytes() : "b".getBytes();
-                    assertEquals(new String(bytes), ByteBufferUtil.string(subColumn.value()));
-                }
-            }
         }
     }
-
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a005df3/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java b/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java
index 1bc846b..ae147d4 100644
--- a/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java
+++ b/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java
@@ -70,8 +70,6 @@ public class RangeTombstoneTest extends SchemaLoader
         rm.apply();
         // We don't flush to test with both a range tomsbtone in memtable and in sstable
 
-        QueryPath path = new QueryPath(CFNAME);
-
         // Queries by name
         int[] live = new int[]{ 4, 9, 11, 17, 28 };
         int[] dead = new int[]{ 12, 19, 21, 24, 27 };
@@ -80,7 +78,7 @@ public class RangeTombstoneTest extends SchemaLoader
             columns.add(b(i));
         for (int i : dead)
             columns.add(b(i));
-        cf = cfs.getColumnFamily(QueryFilter.getNamesFilter(dk(key), path, columns));
+        cf = cfs.getColumnFamily(QueryFilter.getNamesFilter(dk(key), CFNAME, columns));
 
         for (int i : live)
             assert isLive(cf, cf.getColumn(b(i))) : "Column " + i + " should be live";
@@ -88,7 +86,7 @@ public class RangeTombstoneTest extends SchemaLoader
             assert !isLive(cf, cf.getColumn(b(i))) : "Column " + i + " shouldn't be live";
 
         // Queries by slices
-        cf = cfs.getColumnFamily(QueryFilter.getSliceFilter(dk(key), path, b(7), b(30), false, Integer.MAX_VALUE));
+        cf = cfs.getColumnFamily(QueryFilter.getSliceFilter(dk(key), CFNAME, b(7), b(30), false, Integer.MAX_VALUE));
 
         for (int i : new int[]{ 7, 8, 9, 11, 13, 15, 17, 28, 29, 30 })
             assert isLive(cf, cf.getColumn(b(i))) : "Column " + i + " should be live";
@@ -132,8 +130,7 @@ public class RangeTombstoneTest extends SchemaLoader
         rm.apply();
         cfs.forceBlockingFlush();
 
-        QueryPath path = new QueryPath(CFNAME);
-        cf = cfs.getColumnFamily(QueryFilter.getIdentityFilter(dk(key), path));
+        cf = cfs.getColumnFamily(QueryFilter.getIdentityFilter(dk(key), CFNAME));
 
         for (int i = 0; i < 5; i++)
             assert isLive(cf, cf.getColumn(b(i))) : "Column " + i + " should be live";
@@ -144,7 +141,7 @@ public class RangeTombstoneTest extends SchemaLoader
 
         // Compact everything and re-test
         CompactionManager.instance.performMaximal(cfs);
-        cf = cfs.getColumnFamily(QueryFilter.getIdentityFilter(dk(key), path));
+        cf = cfs.getColumnFamily(QueryFilter.getIdentityFilter(dk(key), CFNAME));
 
         for (int i = 0; i < 5; i++)
             assert isLive(cf, cf.getColumn(b(i))) : "Column " + i + " should be live";
@@ -154,7 +151,7 @@ public class RangeTombstoneTest extends SchemaLoader
             assert !isLive(cf, cf.getColumn(b(i))) : "Column " + i + " shouldn't be live";
     }
 
-    private static boolean isLive(ColumnFamily cf, IColumn c)
+    private static boolean isLive(ColumnFamily cf, Column c)
     {
         return c != null && !c.isMarkedForDelete() && !cf.deletionInfo().isDeleted(c);
     }
@@ -170,7 +167,7 @@ public class RangeTombstoneTest extends SchemaLoader
 
     private static void add(RowMutation rm, int value, long timestamp)
     {
-        rm.add(new QueryPath(CFNAME, null, b(value)), b(value), timestamp);
+        rm.add(CFNAME, b(value), b(value), timestamp);
     }
 
     private static void delete(ColumnFamily cf, int from, int to, long timestamp)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a005df3/test/unit/org/apache/cassandra/db/ReadMessageTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/ReadMessageTest.java b/test/unit/org/apache/cassandra/db/ReadMessageTest.java
index 4b56b96..23aea67 100644
--- a/test/unit/org/apache/cassandra/db/ReadMessageTest.java
+++ b/test/unit/org/apache/cassandra/db/ReadMessageTest.java
@@ -24,13 +24,16 @@ import java.io.*;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.SortedSet;
+import java.util.TreeSet;
 
 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.filter.QueryPath;
+import org.apache.cassandra.db.filter.NamesQueryFilter;
+import org.apache.cassandra.db.filter.SliceQueryFilter;
 import org.apache.cassandra.io.util.DataOutputBuffer;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.utils.ByteBufferUtil;
@@ -41,30 +44,30 @@ public class ReadMessageTest extends SchemaLoader
     @Test
     public void testMakeReadMessage() throws IOException
     {
-        ArrayList<ByteBuffer> colList = new ArrayList<ByteBuffer>();
+        SortedSet<ByteBuffer> colList = new TreeSet<ByteBuffer>();
         colList.add(ByteBufferUtil.bytes("col1"));
         colList.add(ByteBufferUtil.bytes("col2"));
 
         ReadCommand rm, rm2;
         DecoratedKey dk = Util.dk("row1");
 
-        rm = new SliceByNamesReadCommand("Keyspace1", dk.key, new QueryPath("Standard1"), colList);
+        rm = new SliceByNamesReadCommand("Keyspace1", dk.key, "Standard1", new NamesQueryFilter(colList));
         rm2 = serializeAndDeserializeReadMessage(rm);
         assert rm2.toString().equals(rm.toString());
 
-        rm = new SliceFromReadCommand("Keyspace1", dk.key, new QueryPath("Standard1"), ByteBufferUtil.EMPTY_BYTE_BUFFER, ByteBufferUtil.EMPTY_BYTE_BUFFER, true, 2);
+        rm = new SliceFromReadCommand("Keyspace1", dk.key, "Standard1", new SliceQueryFilter(ByteBufferUtil.EMPTY_BYTE_BUFFER, ByteBufferUtil.EMPTY_BYTE_BUFFER, true, 2));
         rm2 = serializeAndDeserializeReadMessage(rm);
         assert rm2.toString().equals(rm.toString());
 
-        rm = new SliceFromReadCommand("Keyspace1", dk.key, new QueryPath("Standard1"), ByteBufferUtil.bytes("a"), ByteBufferUtil.bytes("z"), true, 5);
+        rm = new SliceFromReadCommand("Keyspace1", dk.key, "Standard1", new SliceQueryFilter(ByteBufferUtil.bytes("a"), ByteBufferUtil.bytes("z"), true, 5));
         rm2 = serializeAndDeserializeReadMessage(rm);
         assertEquals(rm2.toString(), rm.toString());
 
-        rm = new SliceFromReadCommand("Keyspace1", dk.key, new QueryPath("Standard1"), ByteBufferUtil.EMPTY_BYTE_BUFFER, ByteBufferUtil.EMPTY_BYTE_BUFFER, true, 2);
+        rm = new SliceFromReadCommand("Keyspace1", dk.key, "Standard1", new SliceQueryFilter(ByteBufferUtil.EMPTY_BYTE_BUFFER, ByteBufferUtil.EMPTY_BYTE_BUFFER, true, 2));
         rm2 = serializeAndDeserializeReadMessage(rm);
         assert rm2.toString().equals(rm.toString());
 
-        rm = new SliceFromReadCommand("Keyspace1", dk.key, new QueryPath("Standard1"), ByteBufferUtil.bytes("a"), ByteBufferUtil.bytes("z"), true, 5);
+        rm = new SliceFromReadCommand("Keyspace1", dk.key, "Standard1", new SliceQueryFilter(ByteBufferUtil.bytes("a"), ByteBufferUtil.bytes("z"), true, 5));
         rm2 = serializeAndDeserializeReadMessage(rm);
         assertEquals(rm2.toString(), rm.toString());
     }
@@ -89,12 +92,12 @@ public class ReadMessageTest extends SchemaLoader
 
         // add data
         rm = new RowMutation("Keyspace1", dk.key);
-        rm.add(new QueryPath("Standard1", null, ByteBufferUtil.bytes("Column1")), ByteBufferUtil.bytes("abcd"), 0);
+        rm.add("Standard1", ByteBufferUtil.bytes("Column1"), ByteBufferUtil.bytes("abcd"), 0);
         rm.apply();
 
-        ReadCommand command = new SliceByNamesReadCommand("Keyspace1", dk.key, new QueryPath("Standard1"), Arrays.asList(ByteBufferUtil.bytes("Column1")));
+        ReadCommand command = new SliceByNamesReadCommand("Keyspace1", dk.key, "Standard1", new NamesQueryFilter(ByteBufferUtil.bytes("Column1")));
         Row row = command.getRow(table);
-        IColumn col = row.cf.getColumn(ByteBufferUtil.bytes("Column1"));
+        Column col = row.cf.getColumn(ByteBufferUtil.bytes("Column1"));
         assertEquals(col.value(), ByteBuffer.wrap("abcd".getBytes()));
     }
 
@@ -103,11 +106,11 @@ public class ReadMessageTest extends SchemaLoader
     {
 
         RowMutation rm = new RowMutation("Keyspace1", ByteBufferUtil.bytes("row"));
-        rm.add(new QueryPath("Standard1", null, ByteBufferUtil.bytes("commit1")), ByteBufferUtil.bytes("abcd"), 0);
+        rm.add("Standard1", ByteBufferUtil.bytes("commit1"), ByteBufferUtil.bytes("abcd"), 0);
         rm.apply();
 
         rm = new RowMutation("NoCommitlogSpace", ByteBufferUtil.bytes("row"));
-        rm.add(new QueryPath("Standard1", null, ByteBufferUtil.bytes("commit2")), ByteBufferUtil.bytes("abcd"), 0);
+        rm.add("Standard1", ByteBufferUtil.bytes("commit2"), ByteBufferUtil.bytes("abcd"), 0);
         rm.apply();
 
         boolean commitLogMessageFound = false;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a005df3/test/unit/org/apache/cassandra/db/RecoveryManagerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RecoveryManagerTest.java b/test/unit/org/apache/cassandra/db/RecoveryManagerTest.java
index 68c0b37..96889b8 100644
--- a/test/unit/org/apache/cassandra/db/RecoveryManagerTest.java
+++ b/test/unit/org/apache/cassandra/db/RecoveryManagerTest.java
@@ -96,7 +96,7 @@ public class RecoveryManagerTest extends SchemaLoader
         cf = Util.getColumnFamily(table1, dk, "Counter1");
 
         assert cf.getColumnCount() == 1;
-        IColumn c = cf.getColumn(ByteBufferUtil.bytes("col"));
+        Column c = cf.getColumn(ByteBufferUtil.bytes("col"));
 
         assert c != null;
         assert ((CounterColumn)c).total() == 10L;