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

[2/9] Replace supercolumns internally by composites

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a005df3/test/unit/org/apache/cassandra/db/RecoveryManagerTruncateTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RecoveryManagerTruncateTest.java b/test/unit/org/apache/cassandra/db/RecoveryManagerTruncateTest.java
index 843bbef..6776e65 100644
--- a/test/unit/org/apache/cassandra/db/RecoveryManagerTruncateTest.java
+++ b/test/unit/org/apache/cassandra/db/RecoveryManagerTruncateTest.java
@@ -29,7 +29,6 @@ import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
 import org.apache.cassandra.db.commitlog.CommitLog;
 import org.apache.cassandra.db.filter.QueryFilter;
-import org.apache.cassandra.db.filter.QueryPath;
 import org.junit.Test;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
@@ -66,7 +65,7 @@ public class RecoveryManagerTruncateTest extends SchemaLoader
 		assertNull(getFromTable(table, "Standard1", "keymulti", "col1"));
 	}
 
-	private IColumn getFromTable(Table table, String cfName, String keyName, String columnName)
+	private Column getFromTable(Table table, String cfName, String keyName, String columnName)
 	{
 		ColumnFamily cf;
 		ColumnFamilyStore cfStore = table.getColumnFamilyStore(cfName);
@@ -75,7 +74,7 @@ public class RecoveryManagerTruncateTest extends SchemaLoader
 			return null;
 		}
 		cf = cfStore.getColumnFamily(QueryFilter.getNamesFilter(
-		        Util.dk(keyName), new QueryPath(cfName), ByteBufferUtil.bytes(columnName)));
+		        Util.dk(keyName), cfName, ByteBufferUtil.bytes(columnName)));
 		if (cf == null)
 		{
 			return null;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a005df3/test/unit/org/apache/cassandra/db/RemoveColumnFamilyTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RemoveColumnFamilyTest.java b/test/unit/org/apache/cassandra/db/RemoveColumnFamilyTest.java
index 15369d6..fb42361 100644
--- a/test/unit/org/apache/cassandra/db/RemoveColumnFamilyTest.java
+++ b/test/unit/org/apache/cassandra/db/RemoveColumnFamilyTest.java
@@ -25,7 +25,6 @@ import org.junit.Test;
 
 import static junit.framework.Assert.assertNull;
 import org.apache.cassandra.db.filter.QueryFilter;
-import org.apache.cassandra.db.filter.QueryPath;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
@@ -44,15 +43,15 @@ public class RemoveColumnFamilyTest extends SchemaLoader
 
         // add data
         rm = new RowMutation("Keyspace1", dk.key);
-        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();
 
         // remove
         rm = new RowMutation("Keyspace1", dk.key);
-        rm.delete(new QueryPath("Standard1"), 1);
+        rm.delete("Standard1", 1);
         rm.apply();
 
-        ColumnFamily retrieved = store.getColumnFamily(QueryFilter.getIdentityFilter(dk, new QueryPath("Standard1", null, ByteBufferUtil.bytes("Column1"))));
+        ColumnFamily retrieved = store.getColumnFamily(QueryFilter.getIdentityFilter(dk, "Standard1"));
         assert retrieved.isMarkedForDelete();
         assertNull(retrieved.getColumn(ByteBufferUtil.bytes("Column1")));
         assertNull(Util.cloneAndRemoveDeleted(retrieved, Integer.MAX_VALUE));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a005df3/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush1Test.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush1Test.java b/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush1Test.java
index f724cb5..b32b3a9 100644
--- a/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush1Test.java
+++ b/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush1Test.java
@@ -25,7 +25,6 @@ import org.junit.Test;
 
 import static junit.framework.Assert.assertNull;
 import org.apache.cassandra.db.filter.QueryFilter;
-import org.apache.cassandra.db.filter.QueryPath;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
@@ -44,17 +43,17 @@ public class RemoveColumnFamilyWithFlush1Test extends SchemaLoader
 
         // add data
         rm = new RowMutation("Keyspace1", dk.key);
-        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);
         rm.apply();
         store.forceBlockingFlush();
 
         // remove
         rm = new RowMutation("Keyspace1", dk.key);
-        rm.delete(new QueryPath("Standard1"), 1);
+        rm.delete("Standard1", 1);
         rm.apply();
 
-        ColumnFamily retrieved = store.getColumnFamily(QueryFilter.getIdentityFilter(dk, new QueryPath("Standard1")));
+        ColumnFamily retrieved = store.getColumnFamily(QueryFilter.getIdentityFilter(dk, "Standard1"));
         assert retrieved.isMarkedForDelete();
         assertNull(retrieved.getColumn(ByteBufferUtil.bytes("Column1")));
         assertNull(Util.cloneAndRemoveDeleted(retrieved, Integer.MAX_VALUE));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a005df3/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush2Test.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush2Test.java b/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush2Test.java
index 49585dc..d59d09b 100644
--- a/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush2Test.java
+++ b/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush2Test.java
@@ -25,7 +25,6 @@ import org.junit.Test;
 
 import static junit.framework.Assert.assertNull;
 import org.apache.cassandra.db.filter.QueryFilter;
-import org.apache.cassandra.db.filter.QueryPath;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
@@ -44,15 +43,15 @@ public class RemoveColumnFamilyWithFlush2Test extends SchemaLoader
 
         // add data
         rm = new RowMutation("Keyspace1", dk.key);
-        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();
         // remove
         rm = new RowMutation("Keyspace1", dk.key);
-        rm.delete(new QueryPath("Standard1"), 1);
+        rm.delete("Standard1", 1);
         rm.apply();
         store.forceBlockingFlush();
 
-        ColumnFamily retrieved = store.getColumnFamily(QueryFilter.getIdentityFilter(dk, new QueryPath("Standard1", null, ByteBufferUtil.bytes("Column1"))));
+        ColumnFamily retrieved = store.getColumnFamily(QueryFilter.getIdentityFilter(dk, "Standard1"));
         assert retrieved.isMarkedForDelete();
         assertNull(retrieved.getColumn(ByteBufferUtil.bytes("Column1")));
         assertNull(Util.cloneAndRemoveDeleted(retrieved, Integer.MAX_VALUE));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a005df3/test/unit/org/apache/cassandra/db/RemoveColumnTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RemoveColumnTest.java b/test/unit/org/apache/cassandra/db/RemoveColumnTest.java
index 06ce57d..4c3f3aa 100644
--- a/test/unit/org/apache/cassandra/db/RemoveColumnTest.java
+++ b/test/unit/org/apache/cassandra/db/RemoveColumnTest.java
@@ -27,7 +27,6 @@ import static junit.framework.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 
 import org.apache.cassandra.db.filter.QueryFilter;
-import org.apache.cassandra.db.filter.QueryPath;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
@@ -46,19 +45,19 @@ public class RemoveColumnTest extends SchemaLoader
 
         // add data
         rm = new RowMutation("Keyspace1", dk.key);
-        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();
         store.forceBlockingFlush();
 
         // remove
         rm = new RowMutation("Keyspace1", dk.key);
-        rm.delete(new QueryPath("Standard1", null, ByteBufferUtil.bytes("Column1")), 1);
+        rm.delete("Standard1", ByteBufferUtil.bytes("Column1"), 1);
         rm.apply();
 
-        ColumnFamily retrieved = store.getColumnFamily(QueryFilter.getNamesFilter(dk, new QueryPath("Standard1"), ByteBufferUtil.bytes("Column1")));
+        ColumnFamily retrieved = store.getColumnFamily(QueryFilter.getNamesFilter(dk, "Standard1", ByteBufferUtil.bytes("Column1")));
         assert retrieved.getColumn(ByteBufferUtil.bytes("Column1")).isMarkedForDelete();
         assertNull(Util.cloneAndRemoveDeleted(retrieved, Integer.MAX_VALUE));
-        assertNull(Util.cloneAndRemoveDeleted(store.getColumnFamily(QueryFilter.getIdentityFilter(dk, new QueryPath("Standard1"))), Integer.MAX_VALUE));
+        assertNull(Util.cloneAndRemoveDeleted(store.getColumnFamily(QueryFilter.getIdentityFilter(dk, "Standard1")), Integer.MAX_VALUE));
     }
 
     @Test

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a005df3/test/unit/org/apache/cassandra/db/RemoveSubColumnTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RemoveSubColumnTest.java b/test/unit/org/apache/cassandra/db/RemoveSubColumnTest.java
index 70cbaa2..47fbe80 100644
--- a/test/unit/org/apache/cassandra/db/RemoveSubColumnTest.java
+++ b/test/unit/org/apache/cassandra/db/RemoveSubColumnTest.java
@@ -18,6 +18,7 @@
 */
 package org.apache.cassandra.db;
 
+import java.nio.ByteBuffer;
 import java.io.IOException;
 import java.util.concurrent.ExecutionException;
 
@@ -25,7 +26,7 @@ import org.junit.Test;
 
 import static junit.framework.Assert.assertNull;
 import org.apache.cassandra.db.filter.QueryFilter;
-import org.apache.cassandra.db.filter.QueryPath;
+import org.apache.cassandra.db.marshal.CompositeType;
 import static org.apache.cassandra.Util.getBytes;
 import org.apache.cassandra.Util;
 import org.apache.cassandra.SchemaLoader;
@@ -48,13 +49,14 @@ public class RemoveSubColumnTest extends SchemaLoader
         rm.apply();
         store.forceBlockingFlush();
 
+        ByteBuffer cname = CompositeType.build(ByteBufferUtil.bytes("SC1"), getBytes(1L));
         // remove
         rm = new RowMutation("Keyspace1", dk.key);
-        rm.delete(new QueryPath("Super1", ByteBufferUtil.bytes("SC1"), getBytes(1L)), 1);
+        rm.delete("Super1", cname, 1);
         rm.apply();
 
-        ColumnFamily retrieved = store.getColumnFamily(QueryFilter.getIdentityFilter(dk, new QueryPath("Super1", ByteBufferUtil.bytes("SC1"))));
-        assert retrieved.getColumn(ByteBufferUtil.bytes("SC1")).getSubColumn(getBytes(1L)).isMarkedForDelete();
+        ColumnFamily retrieved = store.getColumnFamily(QueryFilter.getIdentityFilter(dk, "Super1"));
+        assert retrieved.getColumn(cname).isMarkedForDelete();
         assertNull(Util.cloneAndRemoveDeleted(retrieved, Integer.MAX_VALUE));
     }
 
@@ -73,8 +75,10 @@ public class RemoveSubColumnTest extends SchemaLoader
         store.forceBlockingFlush();
 
         // remove the SC
+        ByteBuffer scName = ByteBufferUtil.bytes("SC1");
+        ByteBuffer cname = CompositeType.build(scName, getBytes(1L));
         rm = new RowMutation("Keyspace1", dk.key);
-        rm.delete(new QueryPath("Super1", ByteBufferUtil.bytes("SC1"), null), 1);
+        rm.deleteRange("Super1", SuperColumns.startOf(scName), SuperColumns.endOf(scName), 1);
         rm.apply();
 
         // Mark current time and make sure the next insert happens at least
@@ -84,11 +88,11 @@ public class RemoveSubColumnTest extends SchemaLoader
 
         // remove the column itself
         rm = new RowMutation("Keyspace1", dk.key);
-        rm.delete(new QueryPath("Super1", ByteBufferUtil.bytes("SC1"), getBytes(1)), 2);
+        rm.delete("Super1", cname, 2);
         rm.apply();
 
-        ColumnFamily retrieved = store.getColumnFamily(QueryFilter.getIdentityFilter(dk, new QueryPath("Super1", ByteBufferUtil.bytes("SC1"))), gcbefore);
-        assert retrieved.getColumn(ByteBufferUtil.bytes("SC1")).getSubColumn(getBytes(1)).isMarkedForDelete();
+        ColumnFamily retrieved = store.getColumnFamily(QueryFilter.getIdentityFilter(dk, "Super1"), gcbefore);
+        assert retrieved.getColumn(cname).isMarkedForDelete();
         assertNull(Util.cloneAndRemoveDeleted(retrieved, Integer.MAX_VALUE));
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a005df3/test/unit/org/apache/cassandra/db/RemoveSuperColumnTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RemoveSuperColumnTest.java b/test/unit/org/apache/cassandra/db/RemoveSuperColumnTest.java
deleted file mode 100644
index b15b8c6..0000000
--- a/test/unit/org/apache/cassandra/db/RemoveSuperColumnTest.java
+++ /dev/null
@@ -1,200 +0,0 @@
-/*
-* Licensed to the Apache Software Foundation (ASF) under one
-* or more contributor license agreements.  See the NOTICE file
-* distributed with this work for additional information
-* regarding copyright ownership.  The ASF licenses this file
-* to you under the Apache License, Version 2.0 (the
-* "License"); you may not use this file except in compliance
-* with the License.  You may obtain a copy of the License at
-*
-*    http://www.apache.org/licenses/LICENSE-2.0
-*
-* Unless required by applicable law or agreed to in writing,
-* software distributed under the License is distributed on an
-* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-* KIND, either express or implied.  See the License for the
-* specific language governing permissions and limitations
-* under the License.
-*/
-package org.apache.cassandra.db;
-
-import java.io.IOException;
-import java.util.concurrent.ExecutionException;
-import java.util.Collection;
-
-import org.junit.Test;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertEquals;
-
-import org.apache.cassandra.Util;
-import org.apache.cassandra.db.compaction.CompactionManager;
-import org.apache.cassandra.db.filter.QueryFilter;
-import org.apache.cassandra.db.filter.QueryPath;
-import static org.apache.cassandra.Util.addMutation;
-import static org.apache.cassandra.Util.getBytes;
-
-import org.apache.cassandra.SchemaLoader;
-import static junit.framework.Assert.assertNotNull;
-import org.apache.cassandra.utils.ByteBufferUtil;
-
-
-public class RemoveSuperColumnTest extends SchemaLoader
-{
-    @Test
-    public void testRemoveSuperColumn() throws IOException, ExecutionException, InterruptedException
-    {
-        ColumnFamilyStore store = Table.open("Keyspace1").getColumnFamilyStore("Super1");
-        RowMutation rm;
-        DecoratedKey dk = Util.dk("key1");
-
-        // add data
-        rm = new RowMutation("Keyspace1", dk.key);
-        addMutation(rm, "Super1", "SC1", 1, "val1", 0);
-        rm.apply();
-        store.forceBlockingFlush();
-
-        // remove
-        rm = new RowMutation("Keyspace1", dk.key);
-        rm.delete(new QueryPath("Super1", ByteBufferUtil.bytes("SC1")), 1);
-        rm.apply();
-
-        validateRemoveTwoSources(dk);
-
-        store.forceBlockingFlush();
-        validateRemoveTwoSources(dk);
-
-        CompactionManager.instance.performMaximal(store);
-        assertEquals(1, store.getSSTables().size());
-        validateRemoveCompacted(dk);
-    }
-
-    @Test
-    public void testRemoveDeletedSubColumn() throws IOException, ExecutionException, InterruptedException
-    {
-        ColumnFamilyStore store = Table.open("Keyspace1").getColumnFamilyStore("Super3");
-        RowMutation rm;
-        DecoratedKey dk = Util.dk("key1");
-
-        // add data
-        rm = new RowMutation("Keyspace1", dk.key);
-        addMutation(rm, "Super3", "SC1", 1, "val1", 0);
-        addMutation(rm, "Super3", "SC1", 2, "val1", 0);
-        rm.apply();
-        store.forceBlockingFlush();
-
-        // remove
-        rm = new RowMutation("Keyspace1", dk.key);
-        rm.delete(new QueryPath("Super3", ByteBufferUtil.bytes("SC1"), Util.getBytes(1L)), 1);
-        rm.apply();
-
-        validateRemoveSubColumn(dk);
-
-        store.forceBlockingFlush();
-        validateRemoveSubColumn(dk);
-    }
-
-    private void validateRemoveSubColumn(DecoratedKey dk) throws IOException
-    {
-        ColumnFamilyStore store = Table.open("Keyspace1").getColumnFamilyStore("Super3");
-        ColumnFamily cf = store.getColumnFamily(QueryFilter.getNamesFilter(dk, new QueryPath("Super3", ByteBufferUtil.bytes("SC1")), Util.getBytes(1L)));
-        assertNull(Util.cloneAndRemoveDeleted(cf, Integer.MAX_VALUE));
-        cf = store.getColumnFamily(QueryFilter.getNamesFilter(dk, new QueryPath("Super3", ByteBufferUtil.bytes("SC1")), Util.getBytes(2L)));
-        assertNotNull(Util.cloneAndRemoveDeleted(cf, Integer.MAX_VALUE));
-    }
-
-    private void validateRemoveTwoSources(DecoratedKey dk) throws IOException
-    {
-        ColumnFamilyStore store = Table.open("Keyspace1").getColumnFamilyStore("Super1");
-        ColumnFamily cf = store.getColumnFamily(QueryFilter.getNamesFilter(dk, new QueryPath("Super1"), ByteBufferUtil.bytes("SC1")));
-        assert cf.getSortedColumns().iterator().next().getMarkedForDeleteAt() == 1 : cf;
-        assert cf.getSortedColumns().iterator().next().getSubColumns().size() == 0 : cf;
-        assertNull(Util.cloneAndRemoveDeleted(cf, Integer.MAX_VALUE));
-        cf = store.getColumnFamily(QueryFilter.getNamesFilter(dk, new QueryPath("Super1"), ByteBufferUtil.bytes("SC1")));
-        assertNull(Util.cloneAndRemoveDeleted(cf, Integer.MAX_VALUE));
-        cf = store.getColumnFamily(QueryFilter.getIdentityFilter(dk, new QueryPath("Super1")));
-        assertNull(Util.cloneAndRemoveDeleted(cf, Integer.MAX_VALUE));
-        assertNull(Util.cloneAndRemoveDeleted(store.getColumnFamily(QueryFilter.getIdentityFilter(dk, new QueryPath("Super1"))), Integer.MAX_VALUE));
-    }
-
-    private void validateRemoveCompacted(DecoratedKey dk) throws IOException
-    {
-        ColumnFamilyStore store = Table.open("Keyspace1").getColumnFamilyStore("Super1");
-        ColumnFamily resolved = store.getColumnFamily(QueryFilter.getNamesFilter(dk, new QueryPath("Super1"), ByteBufferUtil.bytes("SC1")));
-        assert resolved.getSortedColumns().iterator().next().getMarkedForDeleteAt() == 1;
-        Collection<IColumn> subColumns = resolved.getSortedColumns().iterator().next().getSubColumns();
-        assert subColumns.size() == 0;
-    }
-
-    @Test
-    public void testRemoveSuperColumnWithNewData() throws IOException, ExecutionException, InterruptedException
-    {
-        ColumnFamilyStore store = Table.open("Keyspace1").getColumnFamilyStore("Super2");
-        RowMutation rm;
-        DecoratedKey dk = Util.dk("key1");
-
-        // add data
-        rm = new RowMutation("Keyspace1", dk.key);
-        addMutation(rm, "Super2", "SC1", 1, "val1", 0);
-        rm.apply();
-        store.forceBlockingFlush();
-
-        // remove
-        rm = new RowMutation("Keyspace1", dk.key);
-        rm.delete(new QueryPath("Super2", ByteBufferUtil.bytes("SC1")), 1);
-        rm.apply();
-
-        // new data
-        rm = new RowMutation("Keyspace1", dk.key);
-        addMutation(rm, "Super2", "SC1", 2, "val2", 2);
-        rm.apply();
-
-        validateRemoveWithNewData(dk);
-
-        store.forceBlockingFlush();
-        validateRemoveWithNewData(dk);
-
-        CompactionManager.instance.performMaximal(store);
-        assertEquals(1, store.getSSTables().size());
-        validateRemoveWithNewData(dk);
-    }
-
-    private void validateRemoveWithNewData(DecoratedKey dk) throws IOException
-    {
-        ColumnFamilyStore store = Table.open("Keyspace1").getColumnFamilyStore("Super2");
-        ColumnFamily cf = store.getColumnFamily(QueryFilter.getNamesFilter(dk, new QueryPath("Super2", ByteBufferUtil.bytes("SC1")), getBytes(2L)));
-        Collection<IColumn> subColumns = cf.getSortedColumns().iterator().next().getSubColumns();
-        assert subColumns.size() == 1;
-        assert subColumns.iterator().next().timestamp() == 2;
-    }
-
-    @Test
-    public void testRemoveSuperColumnResurrection() throws IOException, ExecutionException, InterruptedException
-    {
-        ColumnFamilyStore store = Table.open("Keyspace1").getColumnFamilyStore("Super2");
-        RowMutation rm;
-        DecoratedKey key = Util.dk("keyC");
-
-        // add data
-        rm = new RowMutation("Keyspace1", key.key);
-        addMutation(rm, "Super2", "SC1", 1, "val1", 0);
-        rm.apply();
-
-        // remove
-        rm = new RowMutation("Keyspace1", key.key);
-        rm.delete(new QueryPath("Super2", ByteBufferUtil.bytes("SC1")), 1);
-        rm.apply();
-        assertNull(Util.cloneAndRemoveDeleted(store.getColumnFamily(QueryFilter.getNamesFilter(key, new QueryPath("Super2"), ByteBufferUtil.bytes("SC1"))), Integer.MAX_VALUE));
-
-        // resurrect
-        rm = new RowMutation("Keyspace1", key.key);
-        addMutation(rm, "Super2", "SC1", 1, "val2", 2);
-        rm.apply();
-
-        // validate
-        ColumnFamily cf = store.getColumnFamily(QueryFilter.getNamesFilter(key, new QueryPath("Super2"), ByteBufferUtil.bytes("SC1")));
-        cf = Util.cloneAndRemoveDeleted(cf, Integer.MAX_VALUE);
-        Collection<IColumn> subColumns = cf.getSortedColumns().iterator().next().getSubColumns();
-        assert subColumns.size() == 1;
-        assert subColumns.iterator().next().timestamp() == 2;
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a005df3/test/unit/org/apache/cassandra/db/RowCacheTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RowCacheTest.java b/test/unit/org/apache/cassandra/db/RowCacheTest.java
index 99b8cbc..c75cbc7 100644
--- a/test/unit/org/apache/cassandra/db/RowCacheTest.java
+++ b/test/unit/org/apache/cassandra/db/RowCacheTest.java
@@ -28,7 +28,6 @@ import org.apache.cassandra.Util;
 import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.service.CacheService;
 import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.db.filter.QueryPath;
 
 public class RowCacheTest extends SchemaLoader
 {
@@ -62,17 +61,16 @@ public class RowCacheTest extends SchemaLoader
         for (int i = 0; i < 100; i++)
         {
             DecoratedKey key = Util.dk("key" + i);
-            QueryPath path = new QueryPath(COLUMN_FAMILY, null, ByteBufferUtil.bytes("col" + i));
 
-            cachedStore.getColumnFamily(key, path, ByteBufferUtil.EMPTY_BYTE_BUFFER, ByteBufferUtil.EMPTY_BYTE_BUFFER, false, 1);
+            cachedStore.getColumnFamily(key, ByteBufferUtil.EMPTY_BYTE_BUFFER, ByteBufferUtil.EMPTY_BYTE_BUFFER, false, 1);
             assert CacheService.instance.rowCache.size() == i + 1;
             assert cachedStore.containsCachedRow(key); // current key should be stored in the cache
 
             // checking if column is read correctly after cache
-            ColumnFamily cf = cachedStore.getColumnFamily(key, path, ByteBufferUtil.EMPTY_BYTE_BUFFER, ByteBufferUtil.EMPTY_BYTE_BUFFER, false, 1);
-            Collection<IColumn> columns = cf.getSortedColumns();
+            ColumnFamily cf = cachedStore.getColumnFamily(key, ByteBufferUtil.EMPTY_BYTE_BUFFER, ByteBufferUtil.EMPTY_BYTE_BUFFER, false, 1);
+            Collection<Column> columns = cf.getSortedColumns();
 
-            IColumn column = columns.iterator().next();
+            Column column = columns.iterator().next();
 
             assert columns.size() == 1;
             assert column.name().equals(ByteBufferUtil.bytes("col" + i));
@@ -85,16 +83,15 @@ public class RowCacheTest extends SchemaLoader
         for (int i = 100; i < 110; i++)
         {
             DecoratedKey key = Util.dk("key" + i);
-            QueryPath path = new QueryPath(COLUMN_FAMILY, null, ByteBufferUtil.bytes("col" + i));
 
-            cachedStore.getColumnFamily(key, path, ByteBufferUtil.EMPTY_BYTE_BUFFER, ByteBufferUtil.EMPTY_BYTE_BUFFER, false, 1);
+            cachedStore.getColumnFamily(key, ByteBufferUtil.EMPTY_BYTE_BUFFER, ByteBufferUtil.EMPTY_BYTE_BUFFER, false, 1);
             assert cachedStore.containsCachedRow(key); // cache should be populated with the latest rows read (old ones should be popped)
 
             // checking if column is read correctly after cache
-            ColumnFamily cf = cachedStore.getColumnFamily(key, path, ByteBufferUtil.EMPTY_BYTE_BUFFER, ByteBufferUtil.EMPTY_BYTE_BUFFER, false, 1);
-            Collection<IColumn> columns = cf.getSortedColumns();
+            ColumnFamily cf = cachedStore.getColumnFamily(key, ByteBufferUtil.EMPTY_BYTE_BUFFER, ByteBufferUtil.EMPTY_BYTE_BUFFER, false, 1);
+            Collection<Column> columns = cf.getSortedColumns();
 
-            IColumn column = columns.iterator().next();
+            Column column = columns.iterator().next();
 
             assert columns.size() == 1;
             assert column.name().equals(ByteBufferUtil.bytes("col" + i));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a005df3/test/unit/org/apache/cassandra/db/RowIterationTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RowIterationTest.java b/test/unit/org/apache/cassandra/db/RowIterationTest.java
index 26f11c8..a6c0495 100644
--- a/test/unit/org/apache/cassandra/db/RowIterationTest.java
+++ b/test/unit/org/apache/cassandra/db/RowIterationTest.java
@@ -30,7 +30,7 @@ import org.apache.cassandra.Util;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.db.filter.QueryPath;
+import org.apache.cassandra.db.marshal.CompositeType;
 import org.apache.cassandra.utils.FBUtilities;
 import static junit.framework.Assert.assertEquals;
 import org.apache.cassandra.utils.ByteBufferUtil;
@@ -52,7 +52,7 @@ public class RowIterationTest extends SchemaLoader
         for (int i = 0; i < ROWS_PER_SSTABLE; i++) {
             DecoratedKey key = Util.dk(String.valueOf(i));
             RowMutation rm = new RowMutation(TABLE1, key.key);
-            rm.add(new QueryPath("Super3", ByteBufferUtil.bytes("sc"), ByteBufferUtil.bytes(String.valueOf(i))), ByteBuffer.wrap(new byte[ROWS_PER_SSTABLE * 10 - i * 2]), i);
+            rm.add("Super3", CompositeType.build(ByteBufferUtil.bytes("sc"), ByteBufferUtil.bytes(String.valueOf(i))), ByteBuffer.wrap(new byte[ROWS_PER_SSTABLE * 10 - i * 2]), i);
             rm.apply();
             inserted.add(key);
         }
@@ -70,16 +70,16 @@ public class RowIterationTest extends SchemaLoader
 
         // Delete row in first sstable
         RowMutation rm = new RowMutation(TABLE1, key.key);
-        rm.delete(new QueryPath(CF_NAME, null, null), 0);
-        rm.add(new QueryPath(CF_NAME, null, ByteBufferUtil.bytes("c")), ByteBufferUtil.bytes("values"), 0L);
+        rm.delete(CF_NAME, 0);
+        rm.add(CF_NAME, ByteBufferUtil.bytes("c"), ByteBufferUtil.bytes("values"), 0L);
         DeletionInfo delInfo1 = rm.getColumnFamilies().iterator().next().deletionInfo();
         rm.apply();
         store.forceBlockingFlush();
 
         // Delete row in second sstable with higher timestamp
         rm = new RowMutation(TABLE1, key.key);
-        rm.delete(new QueryPath(CF_NAME, null, null), 1);
-        rm.add(new QueryPath(CF_NAME, null, ByteBufferUtil.bytes("c")), ByteBufferUtil.bytes("values"), 1L);
+        rm.delete(CF_NAME, 1);
+        rm.add(CF_NAME, ByteBufferUtil.bytes("c"), ByteBufferUtil.bytes("values"), 1L);
         DeletionInfo delInfo2 = rm.getColumnFamilies().iterator().next().deletionInfo();
         assert delInfo2.getTopLevelDeletion().markedForDeleteAt == 1L;
         rm.apply();
@@ -99,7 +99,7 @@ public class RowIterationTest extends SchemaLoader
 
         // Delete a row in first sstable
         RowMutation rm = new RowMutation(TABLE1, key.key);
-        rm.delete(new QueryPath(CF_NAME, null, null), 0);
+        rm.delete(CF_NAME, 0);
         rm.apply();
         store.forceBlockingFlush();
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a005df3/test/unit/org/apache/cassandra/db/RowTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RowTest.java b/test/unit/org/apache/cassandra/db/RowTest.java
index c176abf..a36fb9e 100644
--- a/test/unit/org/apache/cassandra/db/RowTest.java
+++ b/test/unit/org/apache/cassandra/db/RowTest.java
@@ -48,21 +48,6 @@ public class RowTest extends SchemaLoader
     }
 
     @Test
-    public void testDiffSuperColumn()
-    {
-        SuperColumn sc1 = new SuperColumn(ByteBufferUtil.bytes("one"), AsciiType.instance);
-        sc1.addColumn(column("subcolumn", "A", 0));
-
-        SuperColumn sc2 = new SuperColumn(ByteBufferUtil.bytes("one"), AsciiType.instance);
-        DeletionInfo delInfo = new DeletionInfo(0, 0);
-        sc2.delete(delInfo);
-
-        SuperColumn scDiff = (SuperColumn)sc1.diff(sc2);
-        assertEquals(scDiff.getSubColumns().size(), 0);
-        assertEquals(scDiff.deletionInfo(), delInfo);
-    }
-
-    @Test
     public void testResolve()
     {
         ColumnFamily cf1 = ColumnFamily.create("Keyspace1", "Standard1");

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a005df3/test/unit/org/apache/cassandra/db/ScrubTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/ScrubTest.java b/test/unit/org/apache/cassandra/db/ScrubTest.java
index fbde908..a652f7f 100644
--- a/test/unit/org/apache/cassandra/db/ScrubTest.java
+++ b/test/unit/org/apache/cassandra/db/ScrubTest.java
@@ -33,6 +33,8 @@ import org.apache.cassandra.Util;
 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.NamesQueryFilter;
+import org.apache.cassandra.db.marshal.CompositeType;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.CLibrary;
@@ -93,7 +95,7 @@ public class ScrubTest extends SchemaLoader
         boolean caught = false;
         try
         {
-             rows = cfs.getRangeSlice(ByteBufferUtil.bytes("1"), Util.range("", ""), 1000, new IdentityQueryFilter(), null);
+             rows = cfs.getRangeSlice(Util.range("", ""), 1000, new NamesQueryFilter(CompositeType.build(ByteBufferUtil.bytes("1"))), null);
              fail("This slice should fail");
         }
         catch (NegativeArraySizeException e)
@@ -103,7 +105,7 @@ public class ScrubTest extends SchemaLoader
         assert caught : "'corrupt' test file actually was not";
 
         CompactionManager.instance.performScrub(cfs);
-        rows = cfs.getRangeSlice(ByteBufferUtil.bytes("1"), Util.range("", ""), 1000, new IdentityQueryFilter(), null);
+        rows = cfs.getRangeSlice(Util.range("", ""), 1000, new IdentityQueryFilter(), null);
         assertEquals(100, rows.size());
     }
 
@@ -118,13 +120,13 @@ public class ScrubTest extends SchemaLoader
 
         // insert data and verify we get it back w/ range query
         fillCF(cfs, 1);
-        rows = cfs.getRangeSlice(null, Util.range("", ""), 1000, new IdentityQueryFilter(), null);
+        rows = cfs.getRangeSlice(Util.range("", ""), 1000, new IdentityQueryFilter(), null);
         assertEquals(1, rows.size());
 
         CompactionManager.instance.performScrub(cfs);
 
         // check data is still there
-        rows = cfs.getRangeSlice(null, Util.range("", ""), 1000, new IdentityQueryFilter(), null);
+        rows = cfs.getRangeSlice(Util.range("", ""), 1000, new IdentityQueryFilter(), null);
         assertEquals(1, rows.size());
     }
 
@@ -158,13 +160,13 @@ public class ScrubTest extends SchemaLoader
 
         // insert data and verify we get it back w/ range query
         fillCF(cfs, 10);
-        rows = cfs.getRangeSlice(null, Util.range("", ""), 1000, new IdentityQueryFilter(), null);
+        rows = cfs.getRangeSlice(Util.range("", ""), 1000, new IdentityQueryFilter(), null);
         assertEquals(10, rows.size());
 
         CompactionManager.instance.performScrub(cfs);
 
         // check data is still there
-        rows = cfs.getRangeSlice(null, Util.range("", ""), 1000, new IdentityQueryFilter(), null);
+        rows = cfs.getRangeSlice(Util.range("", ""), 1000, new IdentityQueryFilter(), null);
         assertEquals(10, rows.size());
     }
 
@@ -198,11 +200,11 @@ public class ScrubTest extends SchemaLoader
         assert cfs.getSSTables().size() > 0;
 
         List<Row> rows;
-        rows = cfs.getRangeSlice(null, Util.range("", ""), 1000, new IdentityQueryFilter(), null);
+        rows = cfs.getRangeSlice(Util.range("", ""), 1000, new IdentityQueryFilter(), null);
         assert !isRowOrdered(rows) : "'corrupt' test file actually was not";
 
         CompactionManager.instance.performScrub(cfs);
-        rows = cfs.getRangeSlice(null, Util.range("", ""), 1000, new IdentityQueryFilter(), null);
+        rows = cfs.getRangeSlice(Util.range("", ""), 1000, new IdentityQueryFilter(), null);
         assert isRowOrdered(rows) : "Scrub failed: " + rows;
         assert rows.size() == 6: "Got " + rows.size();
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a005df3/test/unit/org/apache/cassandra/db/SecondaryIndexColumnSizeTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/SecondaryIndexColumnSizeTest.java b/test/unit/org/apache/cassandra/db/SecondaryIndexColumnSizeTest.java
index 8872d65..708c04f 100644
--- a/test/unit/org/apache/cassandra/db/SecondaryIndexColumnSizeTest.java
+++ b/test/unit/org/apache/cassandra/db/SecondaryIndexColumnSizeTest.java
@@ -29,7 +29,6 @@ import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.db.index.PerColumnSecondaryIndex;
 import org.apache.cassandra.db.index.PerRowSecondaryIndex;
 import org.apache.cassandra.db.index.SecondaryIndexSearcher;
-import org.apache.cassandra.thrift.Column;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 import static org.junit.Assert.assertFalse;
@@ -40,9 +39,6 @@ public class SecondaryIndexColumnSizeTest
     @Test
     public void test64kColumn()
     {
-        Column column = new Column();
-        column.name = ByteBufferUtil.bytes("test");
-
         // a byte buffer more than 64k
         ByteBuffer buffer = ByteBuffer.allocate(1024 * 65);
         buffer.clear();
@@ -53,7 +49,7 @@ public class SecondaryIndexColumnSizeTest
 
         // for read
         buffer.flip();
-        column.value = buffer;
+        Column column = new Column(ByteBufferUtil.bytes("test"), buffer, 0);
 
         MockRowIndex mockRowIndex = new MockRowIndex();
         MockColumnIndex mockColumnIndex = new MockColumnIndex();
@@ -203,17 +199,17 @@ public class SecondaryIndexColumnSizeTest
         }
 
         @Override
-        public void delete(ByteBuffer rowKey, IColumn col)
+        public void delete(ByteBuffer rowKey, Column col)
         {
         }
 
         @Override
-        public void insert(ByteBuffer rowKey, IColumn col)
+        public void insert(ByteBuffer rowKey, Column col)
         {
         }
 
         @Override
-        public void update(ByteBuffer rowKey, IColumn col)
+        public void update(ByteBuffer rowKey, Column col)
         {
         }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a005df3/test/unit/org/apache/cassandra/db/SerializationsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/SerializationsTest.java b/test/unit/org/apache/cassandra/db/SerializationsTest.java
index acfdf4b..5be7096 100644
--- a/test/unit/org/apache/cassandra/db/SerializationsTest.java
+++ b/test/unit/org/apache/cassandra/db/SerializationsTest.java
@@ -23,6 +23,7 @@ import org.apache.cassandra.Util;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.filter.*;
 import org.apache.cassandra.db.marshal.BytesType;
+import org.apache.cassandra.db.marshal.CompositeType;
 import org.apache.cassandra.dht.AbstractBounds;
 import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.dht.Range;
@@ -45,34 +46,39 @@ import java.util.*;
 
 public class SerializationsTest extends AbstractSerializationsTester
 {
+    Statics statics = new Statics();
+
     @BeforeClass
     public static void loadSchema() throws IOException
     {
         loadSchema(true);
     }
 
+    private ByteBuffer startCol = ByteBufferUtil.bytes("Start");
+    private ByteBuffer stopCol = ByteBufferUtil.bytes("Stop");
+    private ByteBuffer emptyCol = ByteBufferUtil.bytes("");
+    public NamesQueryFilter namesPred = new NamesQueryFilter(statics.NamedCols);
+    public NamesQueryFilter namesSCPred = new NamesQueryFilter(statics.NamedSCCols);
+    public SliceQueryFilter emptyRangePred = new SliceQueryFilter(emptyCol, emptyCol, false, 100);
+    public SliceQueryFilter nonEmptyRangePred = new SliceQueryFilter(startCol, stopCol, true, 100);
+    public SliceQueryFilter nonEmptyRangeSCPred = new SliceQueryFilter(CompositeType.build(statics.SC, startCol), CompositeType.build(statics.SC, stopCol), true, 100);
+
     private void testRangeSliceCommandWrite() throws IOException
     {
-        ByteBuffer startCol = ByteBufferUtil.bytes("Start");
-        ByteBuffer stopCol = ByteBufferUtil.bytes("Stop");
-        ByteBuffer emptyCol = ByteBufferUtil.bytes("");
-        NamesQueryFilter namesPred = new NamesQueryFilter(Statics.NamedCols);
-        SliceQueryFilter emptyRangePred = new SliceQueryFilter(emptyCol, emptyCol, false, 100);
-        SliceQueryFilter nonEmptyRangePred = new SliceQueryFilter(startCol, stopCol, true, 100);
         IPartitioner part = StorageService.getPartitioner();
         AbstractBounds<RowPosition> bounds = new Range<Token>(part.getRandomToken(), part.getRandomToken()).toRowBounds();
 
-        RangeSliceCommand namesCmd = new RangeSliceCommand(Statics.KS, "Standard1", null, namesPred, bounds, 100);
+        RangeSliceCommand namesCmd = new RangeSliceCommand(statics.KS, "Standard1", namesPred, bounds, 100);
         MessageOut<RangeSliceCommand> namesCmdMsg = namesCmd.createMessage();
-        RangeSliceCommand emptyRangeCmd = new RangeSliceCommand(Statics.KS, "Standard1", null, emptyRangePred, bounds, 100);
+        RangeSliceCommand emptyRangeCmd = new RangeSliceCommand(statics.KS, "Standard1", emptyRangePred, bounds, 100);
         MessageOut<RangeSliceCommand> emptyRangeCmdMsg = emptyRangeCmd.createMessage();
-        RangeSliceCommand regRangeCmd = new RangeSliceCommand(Statics.KS, "Standard1", null,  nonEmptyRangePred, bounds, 100);
+        RangeSliceCommand regRangeCmd = new RangeSliceCommand(statics.KS, "Standard1", nonEmptyRangePred, bounds, 100);
         MessageOut<RangeSliceCommand> regRangeCmdMsg = regRangeCmd.createMessage();
-        RangeSliceCommand namesCmdSup = new RangeSliceCommand(Statics.KS, "Super1", Statics.SC, namesPred, bounds, 100);
+        RangeSliceCommand namesCmdSup = new RangeSliceCommand(statics.KS, "Super1", namesSCPred, bounds, 100);
         MessageOut<RangeSliceCommand> namesCmdSupMsg = namesCmdSup.createMessage();
-        RangeSliceCommand emptyRangeCmdSup = new RangeSliceCommand(Statics.KS, "Super1", Statics.SC, emptyRangePred, bounds, 100);
+        RangeSliceCommand emptyRangeCmdSup = new RangeSliceCommand(statics.KS, "Super1", emptyRangePred, bounds, 100);
         MessageOut<RangeSliceCommand> emptyRangeCmdSupMsg = emptyRangeCmdSup.createMessage();
-        RangeSliceCommand regRangeCmdSup = new RangeSliceCommand(Statics.KS, "Super1", Statics.SC,  nonEmptyRangePred, bounds, 100);
+        RangeSliceCommand regRangeCmdSup = new RangeSliceCommand(statics.KS, "Super1", nonEmptyRangeSCPred, bounds, 100);
         MessageOut<RangeSliceCommand> regRangeCmdSupMsg = regRangeCmdSup.createMessage();
 
         DataOutputStream out = getOutput("db.RangeSliceCommand.bin");
@@ -107,8 +113,8 @@ public class SerializationsTest extends AbstractSerializationsTester
 
     private void testSliceByNamesReadCommandWrite() throws IOException
     {
-        SliceByNamesReadCommand standardCmd = new SliceByNamesReadCommand(Statics.KS, Statics.Key, Statics.StandardPath, Statics.NamedCols);
-        SliceByNamesReadCommand superCmd = new SliceByNamesReadCommand(Statics.KS, Statics.Key, Statics.SuperPath, Statics.NamedCols);
+        SliceByNamesReadCommand standardCmd = new SliceByNamesReadCommand(statics.KS, statics.Key, statics.StandardCF, namesPred);
+        SliceByNamesReadCommand superCmd = new SliceByNamesReadCommand(statics.KS, statics.Key, statics.SuperCF, namesSCPred);
 
         DataOutputStream out = getOutput("db.SliceByNamesReadCommand.bin");
         SliceByNamesReadCommand.serializer.serialize(standardCmd, out, getVersion());
@@ -142,8 +148,9 @@ public class SerializationsTest extends AbstractSerializationsTester
 
     private void testSliceFromReadCommandWrite() throws IOException
     {
-        SliceFromReadCommand standardCmd = new SliceFromReadCommand(Statics.KS, Statics.Key, Statics.StandardPath, Statics.Start, Statics.Stop, true, 100);
-        SliceFromReadCommand superCmd = new SliceFromReadCommand(Statics.KS, Statics.Key, Statics.SuperPath, Statics.Start, Statics.Stop, true, 100);
+        SliceFromReadCommand standardCmd = new SliceFromReadCommand(statics.KS, statics.Key, statics.StandardCF, nonEmptyRangePred);
+        SliceFromReadCommand superCmd = new SliceFromReadCommand(statics.KS, statics.Key, statics.SuperCF, nonEmptyRangeSCPred);
+        
         DataOutputStream out = getOutput("db.SliceFromReadCommand.bin");
         SliceFromReadCommand.serializer.serialize(standardCmd, out, getVersion());
         SliceFromReadCommand.serializer.serialize(superCmd, out, getVersion());
@@ -151,6 +158,7 @@ public class SerializationsTest extends AbstractSerializationsTester
         ReadCommand.serializer.serialize(superCmd, out, getVersion());
         standardCmd.createMessage().serialize(out, getVersion());
         superCmd.createMessage().serialize(out, getVersion());
+
         out.close();
 
         // test serializedSize
@@ -177,15 +185,15 @@ public class SerializationsTest extends AbstractSerializationsTester
     private void testRowWrite() throws IOException
     {
         DataOutputStream out = getOutput("db.Row.bin");
-        Row.serializer.serialize(Statics.StandardRow, out, getVersion());
-        Row.serializer.serialize(Statics.SuperRow, out, getVersion());
-        Row.serializer.serialize(Statics.NullRow, out, getVersion());
+        Row.serializer.serialize(statics.StandardRow, out, getVersion());
+        Row.serializer.serialize(statics.SuperRow, out, getVersion());
+        Row.serializer.serialize(statics.NullRow, out, getVersion());
         out.close();
 
         // test serializedSize
-        testSerializedSize(Statics.StandardRow, Row.serializer);
-        testSerializedSize(Statics.SuperRow, Row.serializer);
-        testSerializedSize(Statics.NullRow, Row.serializer);
+        testSerializedSize(statics.StandardRow, Row.serializer);
+        testSerializedSize(statics.SuperRow, Row.serializer);
+        testSerializedSize(statics.NullRow, Row.serializer);
     }
 
     @Test
@@ -203,17 +211,17 @@ public class SerializationsTest extends AbstractSerializationsTester
 
     private void testRowMutationWrite() throws IOException
     {
-        RowMutation emptyRm = new RowMutation(Statics.KS, Statics.Key);
-        RowMutation standardRowRm = new RowMutation(Statics.KS, Statics.StandardRow);
-        RowMutation superRowRm = new RowMutation(Statics.KS, Statics.SuperRow);
-        RowMutation standardRm = new RowMutation(Statics.KS, Statics.Key);
-        standardRm.add(Statics.StandardCf);
-        RowMutation superRm = new RowMutation(Statics.KS, Statics.Key);
-        superRm.add(Statics.SuperCf);
+        RowMutation emptyRm = new RowMutation(statics.KS, statics.Key);
+        RowMutation standardRowRm = new RowMutation(statics.KS, statics.StandardRow);
+        RowMutation superRowRm = new RowMutation(statics.KS, statics.SuperRow);
+        RowMutation standardRm = new RowMutation(statics.KS, statics.Key);
+        standardRm.add(statics.StandardCf);
+        RowMutation superRm = new RowMutation(statics.KS, statics.Key);
+        superRm.add(statics.SuperCf);
         Map<UUID, ColumnFamily> mods = new HashMap<UUID, ColumnFamily>();
-        mods.put(Statics.StandardCf.metadata().cfId, Statics.StandardCf);
-        mods.put(Statics.SuperCf.metadata().cfId, Statics.SuperCf);
-        RowMutation mixedRm = new RowMutation(Statics.KS, Statics.Key, mods);
+        mods.put(statics.StandardCf.metadata().cfId, statics.StandardCf);
+        mods.put(statics.SuperCf.metadata().cfId, statics.SuperCf);
+        RowMutation mixedRm = new RowMutation(statics.KS, statics.Key, mods);
 
         DataOutputStream out = getOutput("db.RowMutation.bin");
         RowMutation.serializer.serialize(emptyRm, out, getVersion());
@@ -265,9 +273,9 @@ public class SerializationsTest extends AbstractSerializationsTester
 
     private void testTruncateWrite() throws IOException
     {
-        Truncation tr = new Truncation(Statics.KS, "Doesn't Really Matter");
-        TruncateResponse aff = new TruncateResponse(Statics.KS, "Doesn't Matter Either", true);
-        TruncateResponse neg = new TruncateResponse(Statics.KS, "Still Doesn't Matter", false);
+        Truncation tr = new Truncation(statics.KS, "Doesn't Really Matter");
+        TruncateResponse aff = new TruncateResponse(statics.KS, "Doesn't Matter Either", true);
+        TruncateResponse neg = new TruncateResponse(statics.KS, "Still Doesn't Matter", false);
         DataOutputStream out = getOutput("db.Truncation.bin");
         Truncation.serializer.serialize(tr, out, getVersion());
         TruncateResponse.serializer.serialize(aff, out, getVersion());
@@ -338,39 +346,35 @@ public class SerializationsTest extends AbstractSerializationsTester
 
     private static class Statics
     {
-        private static final String KS = "Keyspace1";
-        private static final ByteBuffer Key = ByteBufferUtil.bytes("Key01");
-        private static final SortedSet<ByteBuffer> NamedCols = new TreeSet<ByteBuffer>(BytesType.instance)
+        private final String KS = "Keyspace1";
+        private final ByteBuffer Key = ByteBufferUtil.bytes("Key01");
+        private final SortedSet<ByteBuffer> NamedCols = new TreeSet<ByteBuffer>(BytesType.instance)
         {{
             add(ByteBufferUtil.bytes("AAA"));
             add(ByteBufferUtil.bytes("BBB"));
             add(ByteBufferUtil.bytes("CCC"));
         }};
-        private static final ByteBuffer SC = ByteBufferUtil.bytes("SCName");
-        private static final QueryPath StandardPath = new QueryPath("Standard1");
-        private static final QueryPath SuperPath = new QueryPath("Super1", SC);
-        private static final ByteBuffer Start = ByteBufferUtil.bytes("Start");
-        private static final ByteBuffer Stop = ByteBufferUtil.bytes("Stop");
-
-        private static final ColumnFamily StandardCf = ColumnFamily.create(Statics.KS, "Standard1");
-        private static final ColumnFamily SuperCf = ColumnFamily.create(Statics.KS, "Super1");
-
-        private static final SuperColumn SuperCol = new SuperColumn(Statics.SC, Schema.instance.getComparator(Statics.KS, "Super1"))
+        private final ByteBuffer SC = ByteBufferUtil.bytes("SCName");
+        private final SortedSet<ByteBuffer> NamedSCCols = new TreeSet<ByteBuffer>(BytesType.instance)
         {{
-            addColumn(new Column(bb("aaaa")));
-            addColumn(new Column(bb("bbbb"), bb("bbbbb-value")));
-            addColumn(new Column(bb("cccc"), bb("ccccc-value"), 1000L));
-            addColumn(new DeletedColumn(bb("dddd"), 500, 1000));
-            addColumn(new DeletedColumn(bb("eeee"), bb("eeee-value"), 1001));
-            addColumn(new ExpiringColumn(bb("ffff"), bb("ffff-value"), 2000, 1000));
-            addColumn(new ExpiringColumn(bb("gggg"), bb("gggg-value"), 2001, 1000, 2002));
+            add(CompositeType.build(SC, ByteBufferUtil.bytes("AAA")));
+            add(CompositeType.build(SC, ByteBufferUtil.bytes("BBB")));
+            add(CompositeType.build(SC, ByteBufferUtil.bytes("CCC")));
         }};
+        private final String StandardCF = "Standard1";
+        private final String SuperCF = "Super1";
+        private final ByteBuffer Start = ByteBufferUtil.bytes("Start");
+        private final ByteBuffer Stop = ByteBufferUtil.bytes("Stop");
+
+        private final ColumnFamily StandardCf = ColumnFamily.create(KS, StandardCF);
+        private final ColumnFamily SuperCf = ColumnFamily.create(KS, SuperCF);
 
-        private static final Row StandardRow = new Row(Util.dk("key0"), Statics.StandardCf);
-        private static final Row SuperRow = new Row(Util.dk("key1"), Statics.SuperCf);
-        private static final Row NullRow = new Row(Util.dk("key2"), null);
+        private final Row StandardRow = new Row(Util.dk("key0"), StandardCf);
+        private final Row SuperRow = new Row(Util.dk("key1"), SuperCf);
+        private final Row NullRow = new Row(Util.dk("key2"), null);
 
-        static {
+        private Statics()
+        {
             StandardCf.addColumn(new Column(bb("aaaa")));
             StandardCf.addColumn(new Column(bb("bbbb"), bb("bbbbb-value")));
             StandardCf.addColumn(new Column(bb("cccc"), bb("ccccc-value"), 1000L));
@@ -379,7 +383,13 @@ public class SerializationsTest extends AbstractSerializationsTester
             StandardCf.addColumn(new ExpiringColumn(bb("ffff"), bb("ffff-value"), 2000, 1000));
             StandardCf.addColumn(new ExpiringColumn(bb("gggg"), bb("gggg-value"), 2001, 1000, 2002));
 
-            SuperCf.addColumn(Statics.SuperCol);
+            SuperCf.addColumn(new Column(CompositeType.build(SC, bb("aaaa"))));
+            SuperCf.addColumn(new Column(CompositeType.build(SC, bb("bbbb")), bb("bbbbb-value")));
+            SuperCf.addColumn(new Column(CompositeType.build(SC, bb("cccc")), bb("ccccc-value"), 1000L));
+            SuperCf.addColumn(new DeletedColumn(CompositeType.build(SC, bb("dddd")), 500, 1000));
+            SuperCf.addColumn(new DeletedColumn(CompositeType.build(SC, bb("eeee")), bb("eeee-value"), 1001));
+            SuperCf.addColumn(new ExpiringColumn(CompositeType.build(SC, bb("ffff")), bb("ffff-value"), 2000, 1000));
+            SuperCf.addColumn(new ExpiringColumn(CompositeType.build(SC, bb("gggg")), bb("gggg-value"), 2001, 1000, 2002));
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a005df3/test/unit/org/apache/cassandra/db/SuperColumnTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/SuperColumnTest.java b/test/unit/org/apache/cassandra/db/SuperColumnTest.java
deleted file mode 100644
index 6c07d2d..0000000
--- a/test/unit/org/apache/cassandra/db/SuperColumnTest.java
+++ /dev/null
@@ -1,103 +0,0 @@
-/*
-* Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements.  See the NOTICE file
-* distributed with this work for additional information
-* regarding copyright ownership.  The ASF licenses this file
-* to you under the Apache License, Version 2.0 (the
-* "License"); you may not use this file except in compliance
-* with the License.  You may obtain a copy of the License at
-*
-*    http://www.apache.org/licenses/LICENSE-2.0
-*
-* Unless required by applicable law or agreed to in writing,
-* software distributed under the License is distributed on an
-* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-* KIND, either express or implied.  See the License for the
-* specific language governing permissions and limitations
-* under the License.
-*/
-package org.apache.cassandra.db;
-
-import org.junit.Test;
-
-import static junit.framework.Assert.assertNotNull;
-import static junit.framework.Assert.assertNull;
-import static org.apache.cassandra.Util.getBytes;
-import org.apache.cassandra.db.context.CounterContext;
-import static org.apache.cassandra.db.context.CounterContext.ContextState;
-import org.apache.cassandra.db.marshal.LongType;
-import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.CounterId;
-
-public class SuperColumnTest
-{
-    private static final CounterContext cc = new CounterContext();
-
-    @Test
-    public void testMissingSubcolumn() {
-    	SuperColumn sc = new SuperColumn(ByteBufferUtil.bytes("sc1"), LongType.instance);
-    	sc.addColumn(new Column(getBytes(1), ByteBufferUtil.bytes("value"), 1));
-    	assertNotNull(sc.getSubColumn(getBytes(1)));
-    	assertNull(sc.getSubColumn(getBytes(2)));
-    }
-
-    @Test
-    public void testAddColumnIncrementCounter()
-    {
-        ContextState state;
-
-    	SuperColumn sc = new SuperColumn(ByteBufferUtil.bytes("sc1"), LongType.instance);
-
-        state = ContextState.allocate(4, 1);
-        state.writeElement(CounterId.fromInt(1), 7L, 0L);
-        state.writeElement(CounterId.fromInt(2), 5L, 7L);
-        state.writeElement(CounterId.fromInt(4), 2L, 9L);
-        state.writeElement(CounterId.getLocalId(), 3L, 3L, true);
-        sc.addColumn(new CounterColumn(getBytes(1), state.context, 3L, 0L));
-
-        state = ContextState.allocate(4, 1);
-        state.writeElement(CounterId.fromInt(2), 3L, 4L);
-        state.writeElement(CounterId.fromInt(4), 4L, 1L);
-        state.writeElement(CounterId.fromInt(8), 9L, 0L);
-        state.writeElement(CounterId.getLocalId(), 9L, 5L, true);
-        sc.addColumn(new CounterColumn(getBytes(1), state.context, 10L, 0L));
-
-        state = ContextState.allocate(3, 0);
-        state.writeElement(CounterId.fromInt(2), 1L, 0L);
-        state.writeElement(CounterId.fromInt(3), 6L, 0L);
-        state.writeElement(CounterId.fromInt(7), 3L, 0L);
-        sc.addColumn(new CounterColumn(getBytes(2), state.context, 9L, 0L));
-
-    	assertNotNull(sc.getSubColumn(getBytes(1)));
-    	assertNull(sc.getSubColumn(getBytes(3)));
-
-        // column: 1
-        ContextState c1 = ContextState.allocate(5, 1);
-        c1.writeElement(CounterId.fromInt(1), 7L, 0L);
-        c1.writeElement(CounterId.fromInt(2), 5L, 7L);
-        c1.writeElement(CounterId.fromInt(4), 4L, 1L);
-        c1.writeElement(CounterId.fromInt(8), 9L, 0L);
-        c1.writeElement(CounterId.getLocalId(), 12L, 8L, true);
-        assert 0 == ByteBufferUtil.compareSubArrays(
-            ((CounterColumn)sc.getSubColumn(getBytes(1))).value(),
-            0,
-            c1.context,
-            0,
-            c1.context.remaining());
-
-        // column: 2
-        ContextState c2 = ContextState.allocate(3, 0);
-        c2.writeElement(CounterId.fromInt(2), 1L, 0L);
-        c2.writeElement(CounterId.fromInt(3), 6L, 0L);
-        c2.writeElement(CounterId.fromInt(7), 3L, 0L);
-        assert 0 == ByteBufferUtil.compareSubArrays(
-            ((CounterColumn)sc.getSubColumn(getBytes(2))).value(),
-            0,
-            c2.context,
-            0,
-            c2.context.remaining());
-
-    	assertNotNull(sc.getSubColumn(getBytes(1)));
-    	assertNotNull(sc.getSubColumn(getBytes(2)));
-    	assertNull(sc.getSubColumn(getBytes(3)));
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a005df3/test/unit/org/apache/cassandra/db/TableTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/TableTest.java b/test/unit/org/apache/cassandra/db/TableTest.java
index 20ab173..0b14ccb 100644
--- a/test/unit/org/apache/cassandra/db/TableTest.java
+++ b/test/unit/org/apache/cassandra/db/TableTest.java
@@ -40,7 +40,6 @@ import static org.apache.cassandra.Util.column;
 import static org.apache.cassandra.Util.expiringColumn;
 import static org.apache.cassandra.Util.getBytes;
 import org.apache.cassandra.Util;
-import org.apache.cassandra.db.filter.QueryPath;
 import org.apache.cassandra.db.marshal.Int32Type;
 import org.apache.cassandra.io.sstable.SSTableReader;
 import org.apache.cassandra.utils.ByteBufferUtil;
@@ -76,13 +75,13 @@ public class TableTest extends SchemaLoader
             {
                 ColumnFamily cf;
 
-                cf = cfStore.getColumnFamily(QueryFilter.getNamesFilter(TEST_KEY, new QueryPath("Standard3"), new TreeSet<ByteBuffer>()));
+                cf = cfStore.getColumnFamily(QueryFilter.getNamesFilter(TEST_KEY, "Standard3", new TreeSet<ByteBuffer>()));
                 assertColumns(cf);
 
-                cf = cfStore.getColumnFamily(QueryFilter.getSliceFilter(TEST_KEY, new QueryPath("Standard3"), ByteBufferUtil.EMPTY_BYTE_BUFFER, ByteBufferUtil.EMPTY_BYTE_BUFFER, false, 0));
+                cf = cfStore.getColumnFamily(QueryFilter.getSliceFilter(TEST_KEY, "Standard3", ByteBufferUtil.EMPTY_BYTE_BUFFER, ByteBufferUtil.EMPTY_BYTE_BUFFER, false, 0));
                 assertColumns(cf);
 
-                cf = cfStore.getColumnFamily(QueryFilter.getNamesFilter(TEST_KEY, new QueryPath("Standard3"), ByteBufferUtil.bytes("col99")));
+                cf = cfStore.getColumnFamily(QueryFilter.getNamesFilter(TEST_KEY, "Standard3", ByteBufferUtil.bytes("col99")));
                 assertColumns(cf);
             }
         };
@@ -109,10 +108,10 @@ public class TableTest extends SchemaLoader
             {
                 ColumnFamily cf;
 
-                cf = cfStore.getColumnFamily(QueryFilter.getNamesFilter(TEST_KEY, new QueryPath("Standard1"), ByteBufferUtil.bytes("col1")));
+                cf = cfStore.getColumnFamily(QueryFilter.getNamesFilter(TEST_KEY, "Standard1", ByteBufferUtil.bytes("col1")));
                 assertColumns(cf, "col1");
 
-                cf = cfStore.getColumnFamily(QueryFilter.getNamesFilter(TEST_KEY, new QueryPath("Standard1"), ByteBufferUtil.bytes("col3")));
+                cf = cfStore.getColumnFamily(QueryFilter.getNamesFilter(TEST_KEY, "Standard1", ByteBufferUtil.bytes("col3")));
                 assertColumns(cf, "col3");
             }
         };
@@ -134,13 +133,13 @@ public class TableTest extends SchemaLoader
         rm.add(cf);
         rm.apply();
 
-        cf = cfStore.getColumnFamily(key, new QueryPath("Standard1"), ByteBufferUtil.bytes("b"), ByteBufferUtil.bytes("c"), false, 100);
+        cf = cfStore.getColumnFamily(key, ByteBufferUtil.bytes("b"), ByteBufferUtil.bytes("c"), false, 100);
         assertEquals(2, cf.getColumnCount());
 
-        cf = cfStore.getColumnFamily(key, new QueryPath("Standard1"), ByteBufferUtil.bytes("b"), ByteBufferUtil.bytes("b"), false, 100);
+        cf = cfStore.getColumnFamily(key, ByteBufferUtil.bytes("b"), ByteBufferUtil.bytes("b"), false, 100);
         assertEquals(1, cf.getColumnCount());
 
-        cf = cfStore.getColumnFamily(key, new QueryPath("Standard1"), ByteBufferUtil.bytes("b"), ByteBufferUtil.bytes("c"), false, 1);
+        cf = cfStore.getColumnFamily(key, ByteBufferUtil.bytes("b"), ByteBufferUtil.bytes("c"), false, 1);
         assertEquals(1, cf.getColumnCount());
     }
 
@@ -192,30 +191,30 @@ public class TableTest extends SchemaLoader
                 assert DatabaseDescriptor.getColumnIndexSize() == 4096 : "Unexpected column index size, block boundaries won't be where tests expect them.";
 
                 // test forward, spanning a segment.
-                cf = cfStore.getColumnFamily(ROW, new QueryPath("Standard1"), ByteBufferUtil.bytes("col096"), ByteBufferUtil.bytes("col099"), false, 4);
+                cf = cfStore.getColumnFamily(ROW, ByteBufferUtil.bytes("col096"), ByteBufferUtil.bytes("col099"), false, 4);
                 assertColumns(cf, "col096", "col097", "col098", "col099");
 
                 // test reversed, spanning a segment.
-                cf = cfStore.getColumnFamily(ROW, new QueryPath("Standard1"), ByteBufferUtil.bytes("col099"), ByteBufferUtil.bytes("col096"), true, 4);
+                cf = cfStore.getColumnFamily(ROW, ByteBufferUtil.bytes("col099"), ByteBufferUtil.bytes("col096"), true, 4);
                 assertColumns(cf, "col096", "col097", "col098", "col099");
 
                 // test forward, within a segment.
-                cf = cfStore.getColumnFamily(ROW, new QueryPath("Standard1"), ByteBufferUtil.bytes("col100"), ByteBufferUtil.bytes("col103"), false, 4);
+                cf = cfStore.getColumnFamily(ROW, ByteBufferUtil.bytes("col100"), ByteBufferUtil.bytes("col103"), false, 4);
                 assertColumns(cf, "col100", "col101", "col102", "col103");
 
                 // test reversed, within a segment.
-                cf = cfStore.getColumnFamily(ROW, new QueryPath("Standard1"), ByteBufferUtil.bytes("col103"), ByteBufferUtil.bytes("col100"), true, 4);
+                cf = cfStore.getColumnFamily(ROW, ByteBufferUtil.bytes("col103"), ByteBufferUtil.bytes("col100"), true, 4);
                 assertColumns(cf, "col100", "col101", "col102", "col103");
 
                 // test forward from beginning, spanning a segment.
                 String[] strCols = new String[100]; // col000-col099
                 for (int i = 0; i < 100; i++)
                     strCols[i] = "col" + fmt.format(i);
-                cf = cfStore.getColumnFamily(ROW, new QueryPath("Standard1"), ByteBufferUtil.EMPTY_BYTE_BUFFER, ByteBufferUtil.bytes("col099"), false, 100);
+                cf = cfStore.getColumnFamily(ROW, ByteBufferUtil.EMPTY_BYTE_BUFFER, ByteBufferUtil.bytes("col099"), false, 100);
                 assertColumns(cf, strCols);
 
                 // test reversed, from end, spanning a segment.
-                cf = cfStore.getColumnFamily(ROW, new QueryPath("Standard1"), ByteBufferUtil.EMPTY_BYTE_BUFFER, ByteBufferUtil.bytes("col288"), true, 12);
+                cf = cfStore.getColumnFamily(ROW, ByteBufferUtil.EMPTY_BYTE_BUFFER, ByteBufferUtil.bytes("col288"), true, 12);
                 assertColumns(cf, "col288", "col289", "col290", "col291", "col292", "col293", "col294", "col295", "col296", "col297", "col298", "col299");
             }
         };
@@ -249,7 +248,7 @@ public class TableTest extends SchemaLoader
             rm.add(cf);
             rm.apply();
 
-            cf = cfs.getColumnFamily(ROW, new QueryPath("StandardLong1"), ByteBufferUtil.EMPTY_BYTE_BUFFER, ByteBufferUtil.EMPTY_BYTE_BUFFER, true, 1);
+            cf = cfs.getColumnFamily(ROW, ByteBufferUtil.EMPTY_BYTE_BUFFER, ByteBufferUtil.EMPTY_BYTE_BUFFER, true, 1);
             assertEquals(1, cf.getColumnNames().size());
             assertEquals(i, cf.getColumnNames().iterator().next().getLong());
         }
@@ -261,11 +260,11 @@ public class TableTest extends SchemaLoader
         ColumnFamily cf;
 
         // key before the rows that exists
-        cf = cfStore.getColumnFamily(Util.dk("a"), new QueryPath("Standard2"), ByteBufferUtil.EMPTY_BYTE_BUFFER, ByteBufferUtil.EMPTY_BYTE_BUFFER, false, 1);
+        cf = cfStore.getColumnFamily(Util.dk("a"), ByteBufferUtil.EMPTY_BYTE_BUFFER, ByteBufferUtil.EMPTY_BYTE_BUFFER, false, 1);
         assertColumns(cf);
 
         // key after the rows that exist
-        cf = cfStore.getColumnFamily(Util.dk("z"), new QueryPath("Standard2"), ByteBufferUtil.EMPTY_BYTE_BUFFER, ByteBufferUtil.EMPTY_BYTE_BUFFER, false, 1);
+        cf = cfStore.getColumnFamily(Util.dk("z"), ByteBufferUtil.EMPTY_BYTE_BUFFER, ByteBufferUtil.EMPTY_BYTE_BUFFER, false, 1);
         assertColumns(cf);
     }
 
@@ -289,7 +288,7 @@ public class TableTest extends SchemaLoader
         rm.apply();
 
         rm = new RowMutation("Keyspace1", ROW.key);
-        rm.delete(new QueryPath("Standard1", null, ByteBufferUtil.bytes("col4")), 2L);
+        rm.delete("Standard1", ByteBufferUtil.bytes("col4"), 2L);
         rm.apply();
 
         Runnable verify = new WrappedRunnable()
@@ -298,26 +297,26 @@ public class TableTest extends SchemaLoader
             {
                 ColumnFamily cf;
 
-                cf = cfStore.getColumnFamily(ROW, new QueryPath("Standard1"), ByteBufferUtil.bytes("col5"), ByteBufferUtil.EMPTY_BYTE_BUFFER, false, 2);
+                cf = cfStore.getColumnFamily(ROW, ByteBufferUtil.bytes("col5"), ByteBufferUtil.EMPTY_BYTE_BUFFER, false, 2);
                 assertColumns(cf, "col5", "col7");
 
-                cf = cfStore.getColumnFamily(ROW, new QueryPath("Standard1"), ByteBufferUtil.bytes("col4"), ByteBufferUtil.EMPTY_BYTE_BUFFER, false, 2);
+                cf = cfStore.getColumnFamily(ROW, ByteBufferUtil.bytes("col4"), ByteBufferUtil.EMPTY_BYTE_BUFFER, false, 2);
                 assertColumns(cf, "col4", "col5", "col7");
                 assertColumns(ColumnFamilyStore.removeDeleted(cf, Integer.MAX_VALUE), "col5", "col7");
 
-                cf = cfStore.getColumnFamily(ROW, new QueryPath("Standard1"), ByteBufferUtil.bytes("col5"), ByteBufferUtil.EMPTY_BYTE_BUFFER, true, 2);
+                cf = cfStore.getColumnFamily(ROW, ByteBufferUtil.bytes("col5"), ByteBufferUtil.EMPTY_BYTE_BUFFER, true, 2);
                 assertColumns(cf, "col3", "col4", "col5");
 
-                cf = cfStore.getColumnFamily(ROW, new QueryPath("Standard1"), ByteBufferUtil.bytes("col6"), ByteBufferUtil.EMPTY_BYTE_BUFFER, true, 2);
+                cf = cfStore.getColumnFamily(ROW, ByteBufferUtil.bytes("col6"), ByteBufferUtil.EMPTY_BYTE_BUFFER, true, 2);
                 assertColumns(cf, "col3", "col4", "col5");
 
-                cf = cfStore.getColumnFamily(ROW, new QueryPath("Standard1"), ByteBufferUtil.EMPTY_BYTE_BUFFER, ByteBufferUtil.EMPTY_BYTE_BUFFER, true, 2);
+                cf = cfStore.getColumnFamily(ROW, ByteBufferUtil.EMPTY_BYTE_BUFFER, ByteBufferUtil.EMPTY_BYTE_BUFFER, true, 2);
                 assertColumns(cf, "col7", "col9");
 
-                cf = cfStore.getColumnFamily(ROW, new QueryPath("Standard1"), ByteBufferUtil.bytes("col95"), ByteBufferUtil.EMPTY_BYTE_BUFFER, false, 2);
+                cf = cfStore.getColumnFamily(ROW, ByteBufferUtil.bytes("col95"), ByteBufferUtil.EMPTY_BYTE_BUFFER, false, 2);
                 assertColumns(cf);
 
-                cf = cfStore.getColumnFamily(ROW, new QueryPath("Standard1"), ByteBufferUtil.bytes("col0"), ByteBufferUtil.EMPTY_BYTE_BUFFER, true, 2);
+                cf = cfStore.getColumnFamily(ROW, ByteBufferUtil.bytes("col0"), ByteBufferUtil.EMPTY_BYTE_BUFFER, true, 2);
                 assertColumns(cf);
             }
         };
@@ -348,11 +347,11 @@ public class TableTest extends SchemaLoader
             {
                 ColumnFamily cf;
 
-                cf = cfStore.getColumnFamily(ROW, new QueryPath("Standard1"), ByteBufferUtil.EMPTY_BYTE_BUFFER, ByteBufferUtil.EMPTY_BYTE_BUFFER, false, 2);
+                cf = cfStore.getColumnFamily(ROW, ByteBufferUtil.EMPTY_BYTE_BUFFER, ByteBufferUtil.EMPTY_BYTE_BUFFER, false, 2);
                 assertColumns(cf, "col1", "col2");
                 assertColumns(ColumnFamilyStore.removeDeleted(cf, Integer.MAX_VALUE), "col1");
 
-                cf = cfStore.getColumnFamily(ROW, new QueryPath("Standard1"), ByteBufferUtil.bytes("col2"), ByteBufferUtil.EMPTY_BYTE_BUFFER, false, 1);
+                cf = cfStore.getColumnFamily(ROW, ByteBufferUtil.bytes("col2"), ByteBufferUtil.EMPTY_BYTE_BUFFER, false, 1);
                 assertColumns(cf, "col2");
                 assertColumns(ColumnFamilyStore.removeDeleted(cf, Integer.MAX_VALUE));
             }
@@ -395,7 +394,7 @@ public class TableTest extends SchemaLoader
             {
                 ColumnFamily cf;
 
-                cf = cfStore.getColumnFamily(ROW, new QueryPath("Standard1"), ByteBufferUtil.bytes("col2"), ByteBufferUtil.EMPTY_BYTE_BUFFER, false, 3);
+                cf = cfStore.getColumnFamily(ROW, ByteBufferUtil.bytes("col2"), ByteBufferUtil.EMPTY_BYTE_BUFFER, false, 3);
                 assertColumns(cf, "col2", "col3", "col4");
 
                 ByteBuffer col = cf.getColumn(ByteBufferUtil.bytes("col2")).value();
@@ -446,7 +445,7 @@ public class TableTest extends SchemaLoader
     {
         DecoratedKey key = Util.dk("row3");
         ColumnFamily cf;
-        cf = cfStore.getColumnFamily(key, new QueryPath("Standard1"), ByteBufferUtil.bytes("col1000"), ByteBufferUtil.EMPTY_BYTE_BUFFER, false, 3);
+        cf = cfStore.getColumnFamily(key, ByteBufferUtil.bytes("col1000"), ByteBufferUtil.EMPTY_BYTE_BUFFER, false, 3);
         assertColumns(cf, "col1000", "col1001", "col1002");
 
         ByteBuffer col;
@@ -457,7 +456,7 @@ public class TableTest extends SchemaLoader
         col = cf.getColumn(ByteBufferUtil.bytes("col1002")).value();
         assertEquals(ByteBufferUtil.string(col), "v1002");
 
-        cf = cfStore.getColumnFamily(key, new QueryPath("Standard1"), ByteBufferUtil.bytes("col1195"), ByteBufferUtil.EMPTY_BYTE_BUFFER, false, 3);
+        cf = cfStore.getColumnFamily(key, ByteBufferUtil.bytes("col1195"), ByteBufferUtil.EMPTY_BYTE_BUFFER, false, 3);
         assertColumns(cf, "col1195", "col1196", "col1197");
 
         col = cf.getColumn(ByteBufferUtil.bytes("col1195")).value();
@@ -468,17 +467,17 @@ public class TableTest extends SchemaLoader
         assertEquals(ByteBufferUtil.string(col), "v1197");
 
 
-        cf = cfStore.getColumnFamily(key, new QueryPath("Standard1"), ByteBufferUtil.bytes("col1996"), ByteBufferUtil.EMPTY_BYTE_BUFFER, true, 1000);
-        IColumn[] columns = cf.getSortedColumns().toArray(new IColumn[0]);
+        cf = cfStore.getColumnFamily(key, ByteBufferUtil.bytes("col1996"), ByteBufferUtil.EMPTY_BYTE_BUFFER, true, 1000);
+        Column[] columns = cf.getSortedColumns().toArray(new Column[0]);
         for (int i = 1000; i < 1996; i++)
         {
             String expectedName = "col" + i;
-            IColumn column = columns[i - 1000];
+            Column column = columns[i - 1000];
             assertEquals(ByteBufferUtil.string(column.name()), expectedName);
             assertEquals(ByteBufferUtil.string(column.value()), ("v" + i));
         }
 
-        cf = cfStore.getColumnFamily(key, new QueryPath("Standard1"), ByteBufferUtil.bytes("col1990"), ByteBufferUtil.EMPTY_BYTE_BUFFER, false, 3);
+        cf = cfStore.getColumnFamily(key, ByteBufferUtil.bytes("col1990"), ByteBufferUtil.EMPTY_BYTE_BUFFER, false, 3);
         assertColumns(cf, "col1990", "col1991", "col1992");
         col = cf.getColumn(ByteBufferUtil.bytes("col1990")).value();
         assertEquals(ByteBufferUtil.string(col), "v1990");
@@ -487,7 +486,7 @@ public class TableTest extends SchemaLoader
         col = cf.getColumn(ByteBufferUtil.bytes("col1992")).value();
         assertEquals(ByteBufferUtil.string(col), "v1992");
 
-        cf = cfStore.getColumnFamily(key, new QueryPath("Standard1"), ByteBufferUtil.EMPTY_BYTE_BUFFER, ByteBufferUtil.EMPTY_BYTE_BUFFER, true, 3);
+        cf = cfStore.getColumnFamily(key, ByteBufferUtil.EMPTY_BYTE_BUFFER, ByteBufferUtil.EMPTY_BYTE_BUFFER, true, 3);
         assertColumns(cf, "col1997", "col1998", "col1999");
         col = cf.getColumn(ByteBufferUtil.bytes("col1997")).value();
         assertEquals(ByteBufferUtil.string(col), "v1997");
@@ -496,61 +495,18 @@ public class TableTest extends SchemaLoader
         col = cf.getColumn(ByteBufferUtil.bytes("col1999")).value();
         assertEquals(ByteBufferUtil.string(col), "v1999");
 
-        cf = cfStore.getColumnFamily(key, new QueryPath("Standard1"), ByteBufferUtil.bytes("col9000"), ByteBufferUtil.EMPTY_BYTE_BUFFER, true, 3);
+        cf = cfStore.getColumnFamily(key, ByteBufferUtil.bytes("col9000"), ByteBufferUtil.EMPTY_BYTE_BUFFER, true, 3);
         assertColumns(cf, "col1997", "col1998", "col1999");
 
-        cf = cfStore.getColumnFamily(key, new QueryPath("Standard1"), ByteBufferUtil.bytes("col9000"), ByteBufferUtil.EMPTY_BYTE_BUFFER, false, 3);
+        cf = cfStore.getColumnFamily(key, ByteBufferUtil.bytes("col9000"), ByteBufferUtil.EMPTY_BYTE_BUFFER, false, 3);
         assertColumns(cf);
     }
 
-    @Test
-    public void testGetSliceFromSuperBasic() throws Throwable
+    public static void assertColumns(ColumnFamily container, String... columnNames)
     {
-        // tests slicing against data from one row spread across two sstables
-        final Table table = Table.open("Keyspace1");
-        final ColumnFamilyStore cfStore = table.getColumnFamilyStore("Super1");
-        final DecoratedKey ROW = Util.dk("row2");
-
-        RowMutation rm = new RowMutation("Keyspace1", ROW.key);
-        ColumnFamily cf = ColumnFamily.create("Keyspace1", "Super1");
-        SuperColumn sc = new SuperColumn(ByteBufferUtil.bytes("sc1"), Int32Type.instance);
-        sc.addColumn(new Column(getBytes(1), ByteBufferUtil.bytes("val1"), 1L));
-        cf.addColumn(sc);
-        rm.add(cf);
-        rm.apply();
-
-        Runnable verify = new WrappedRunnable()
-        {
-            public void runMayThrow() throws Exception
-            {
-                ColumnFamily cf = cfStore.getColumnFamily(ROW, new QueryPath("Super1"), ByteBufferUtil.EMPTY_BYTE_BUFFER, ByteBufferUtil.EMPTY_BYTE_BUFFER, false, 10);
-                assertColumns(cf, "sc1");
-
-                ByteBuffer val = cf.getColumn(ByteBufferUtil.bytes("sc1")).getSubColumn(getBytes(1)).value();
-
-                assertEquals(ByteBufferUtil.string(val), "val1");
-            }
-        };
-
-        reTest(table.getColumnFamilyStore("Standard1"), verify);
-    }
-
-    public static void assertColumns(ColumnFamily cf, String... columnNames)
-    {
-        assertColumns((IColumnContainer)cf, columnNames);
-    }
-
-    public static void assertSubColumns(ColumnFamily cf, String scName, String... columnNames)
-    {
-        IColumnContainer sc = cf == null ? null : ((IColumnContainer)cf.getColumn(ByteBufferUtil.bytes(scName)));
-        assertColumns(sc, columnNames);
-    }
-
-    public static void assertColumns(IColumnContainer container, String... columnNames)
-    {
-        Collection<IColumn> columns = container == null ? new TreeSet<IColumn>() : container.getSortedColumns();
+        Collection<Column> columns = container == null ? new TreeSet<Column>() : container.getSortedColumns();
         List<String> L = new ArrayList<String>();
-        for (IColumn column : columns)
+        for (Column column : columns)
         {
             try
             {
@@ -580,13 +536,7 @@ public class TableTest extends SchemaLoader
         assertColumn(cf.getColumn(ByteBufferUtil.bytes(name)), value, timestamp);
     }
 
-    public static void assertSubColumn(ColumnFamily cf, String scName, String name, String value, long timestamp)
-    {
-        SuperColumn sc = (SuperColumn)cf.getColumn(ByteBufferUtil.bytes(scName));
-        assertColumn(sc.getSubColumn(ByteBufferUtil.bytes(name)), value, timestamp);
-    }
-
-    public static void assertColumn(IColumn column, String value, long timestamp)
+    public static void assertColumn(Column column, String value, long timestamp)
     {
         assertNotNull(column);
         assertEquals(0, ByteBufferUtil.compareUnsigned(column.value(), ByteBufferUtil.bytes(value)));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3a005df3/test/unit/org/apache/cassandra/db/TimeSortTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/TimeSortTest.java b/test/unit/org/apache/cassandra/db/TimeSortTest.java
index 7ea91a5..87777ca 100644
--- a/test/unit/org/apache/cassandra/db/TimeSortTest.java
+++ b/test/unit/org/apache/cassandra/db/TimeSortTest.java
@@ -31,7 +31,6 @@ import static org.apache.cassandra.Util.getBytes;
 import org.apache.cassandra.Util;
 
 import org.apache.cassandra.db.filter.QueryFilter;
-import org.apache.cassandra.db.filter.QueryPath;
 import org.apache.cassandra.db.marshal.LongType;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
@@ -47,16 +46,16 @@ public class TimeSortTest extends SchemaLoader
         DecoratedKey key = Util.dk("key0");
 
         rm = new RowMutation("Keyspace1", key.key);
-        rm.add(new QueryPath("StandardLong1", null, getBytes(100)), ByteBufferUtil.bytes("a"), 100);
+        rm.add("StandardLong1", getBytes(100), ByteBufferUtil.bytes("a"), 100);
         rm.apply();
         cfStore.forceBlockingFlush();
 
         rm = new RowMutation("Keyspace1", key.key);
-        rm.add(new QueryPath("StandardLong1", null, getBytes(0)), ByteBufferUtil.bytes("b"), 0);
+        rm.add("StandardLong1", getBytes(0), ByteBufferUtil.bytes("b"), 0);
         rm.apply();
 
-        ColumnFamily cf = cfStore.getColumnFamily(key, new QueryPath("StandardLong1"), getBytes(10), ByteBufferUtil.EMPTY_BYTE_BUFFER, false, 1000);
-        Collection<IColumn> columns = cf.getSortedColumns();
+        ColumnFamily cf = cfStore.getColumnFamily(key, getBytes(10), ByteBufferUtil.EMPTY_BYTE_BUFFER, false, 1000);
+        Collection<Column> columns = cf.getSortedColumns();
         assert columns.size() == 1;
     }
 
@@ -71,7 +70,7 @@ public class TimeSortTest extends SchemaLoader
             RowMutation rm = new RowMutation("Keyspace1", ByteBufferUtil.bytes(Integer.toString(i)));
             for (int j = 0; j < 8; ++j)
             {
-                rm.add(new QueryPath("StandardLong1", null, getBytes(j * 2)), ByteBufferUtil.bytes("a"), j * 2);
+                rm.add("StandardLong1", getBytes(j * 2), ByteBufferUtil.bytes("a"), j * 2);
             }
             rm.apply();
         }
@@ -86,21 +85,21 @@ public class TimeSortTest extends SchemaLoader
         RowMutation rm = new RowMutation("Keyspace1", key.key);
         for (int j = 0; j < 4; ++j)
         {
-            rm.add(new QueryPath("StandardLong1", null, getBytes(j * 2 + 1)), ByteBufferUtil.bytes("b"), j * 2 + 1);
+            rm.add("StandardLong1", getBytes(j * 2 + 1), ByteBufferUtil.bytes("b"), j * 2 + 1);
         }
         rm.apply();
         // and some overwrites
         rm = new RowMutation("Keyspace1", key.key);
-        rm.add(new QueryPath("StandardLong1", null, getBytes(0)), ByteBufferUtil.bytes("c"), 100);
-        rm.add(new QueryPath("StandardLong1", null, getBytes(10)), ByteBufferUtil.bytes("c"), 100);
+        rm.add("StandardLong1", getBytes(0), ByteBufferUtil.bytes("c"), 100);
+        rm.add("StandardLong1", getBytes(10), ByteBufferUtil.bytes("c"), 100);
         rm.apply();
 
         // verify
-        ColumnFamily cf = cfStore.getColumnFamily(key, new QueryPath("StandardLong1"), getBytes(0), ByteBufferUtil.EMPTY_BYTE_BUFFER, false, 1000);
-        Collection<IColumn> columns = cf.getSortedColumns();
+        ColumnFamily cf = cfStore.getColumnFamily(key, getBytes(0), ByteBufferUtil.EMPTY_BYTE_BUFFER, false, 1000);
+        Collection<Column> columns = cf.getSortedColumns();
         assertEquals(12, columns.size());
-        Iterator<IColumn> iter = columns.iterator();
-        IColumn column;
+        Iterator<Column> iter = columns.iterator();
+        Column column;
         for (int j = 0; j < 8; j++)
         {
             column = iter.next();
@@ -109,7 +108,7 @@ public class TimeSortTest extends SchemaLoader
         TreeSet<ByteBuffer> columnNames = new TreeSet<ByteBuffer>(LongType.instance);
         columnNames.add(getBytes(10));
         columnNames.add(getBytes(0));
-        cf = cfStore.getColumnFamily(QueryFilter.getNamesFilter(Util.dk("900"), new QueryPath("StandardLong1"), columnNames));
+        cf = cfStore.getColumnFamily(QueryFilter.getNamesFilter(Util.dk("900"), "StandardLong1", columnNames));
         assert "c".equals(ByteBufferUtil.string(cf.getColumn(getBytes(0)).value()));
         assert "c".equals(ByteBufferUtil.string(cf.getColumn(getBytes(10)).value()));
     }
@@ -121,11 +120,11 @@ public class TimeSortTest extends SchemaLoader
             DecoratedKey key = Util.dk(Integer.toString(i));
             for (int j = 0; j < 8; j += 3)
             {
-                ColumnFamily cf = table.getColumnFamilyStore("StandardLong1").getColumnFamily(key, new QueryPath("StandardLong1"), getBytes(j * 2), ByteBufferUtil.EMPTY_BYTE_BUFFER, false, 1000);
-                Collection<IColumn> columns = cf.getSortedColumns();
+                ColumnFamily cf = table.getColumnFamilyStore("StandardLong1").getColumnFamily(key, getBytes(j * 2), ByteBufferUtil.EMPTY_BYTE_BUFFER, false, 1000);
+                Collection<Column> columns = cf.getSortedColumns();
                 assert columns.size() == 8 - j;
                 int k = j;
-                for (IColumn c : columns)
+                for (Column c : columns)
                 {
                     assertEquals((k++) * 2, c.timestamp());