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

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

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a991b648/test/unit/org/apache/cassandra/cql3/validation/operations/CreateTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/CreateTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/CreateTest.java
index 398b851..18f2db4 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/CreateTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/CreateTest.java
@@ -30,8 +30,8 @@ import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.config.TriggerDefinition;
 import org.apache.cassandra.cql3.CQLTester;
-import org.apache.cassandra.db.ColumnFamily;
 import org.apache.cassandra.db.Mutation;
+import org.apache.cassandra.db.partitions.Partition;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.SyntaxException;
 import org.apache.cassandra.triggers.ITrigger;
@@ -466,10 +466,10 @@ public class CreateTest extends CQLTester
     {
         createTable("CREATE TABLE %s (a int, b int , c int, PRIMARY KEY (a, b)) WITH COMPACT STORAGE;");
 
-        assertInvalidMessage("Secondary indexes are not supported on PRIMARY KEY columns in COMPACT STORAGE tables",
+        assertInvalidMessage("Secondary indexes are not supported on COMPACT STORAGE tables that have clustering columns",
                              "CREATE INDEX ON %s (a);");
 
-        assertInvalidMessage("Secondary indexes are not supported on PRIMARY KEY columns in COMPACT STORAGE tables",
+        assertInvalidMessage("Secondary indexes are not supported on COMPACT STORAGE tables that have clustering columns",
                              "CREATE INDEX ON %s (b);");
 
         assertInvalidMessage("Secondary indexes are not supported on COMPACT STORAGE tables that have clustering columns",
@@ -522,7 +522,7 @@ public class CreateTest extends CQLTester
     public static class TestTrigger implements ITrigger
     {
         public TestTrigger() { }
-        public Collection<Mutation> augment(ByteBuffer key, ColumnFamily update)
+        public Collection<Mutation> augment(Partition update)
         {
             return Collections.emptyList();
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a991b648/test/unit/org/apache/cassandra/cql3/validation/operations/SelectLimitTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectLimitTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectLimitTest.java
index 275ff04..f1e2f55 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectLimitTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectLimitTest.java
@@ -105,8 +105,12 @@ public class SelectLimitTest extends CQLTester
                    row(1, 1),
                    row(1, 2),
                    row(1, 3));
-
-        // strict bound (v > 1) over a range of partitions is not supported for compact storage if limit is provided
-        assertInvalidThrow(InvalidRequestException.class, "SELECT * FROM %s WHERE v > 1 AND v <= 3 LIMIT 6 ALLOW FILTERING");
+        assertRows(execute("SELECT * FROM %s WHERE v > 1 AND v <= 3 LIMIT 6 ALLOW FILTERING"),
+                   row(0, 2),
+                   row(0, 3),
+                   row(1, 2),
+                   row(1, 3),
+                   row(2, 2),
+                   row(2, 3));
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a991b648/test/unit/org/apache/cassandra/cql3/validation/operations/SelectSingleColumnRelationTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectSingleColumnRelationTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectSingleColumnRelationTest.java
index 57b8a86..ff73a95 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectSingleColumnRelationTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectSingleColumnRelationTest.java
@@ -142,7 +142,7 @@ public class SelectSingleColumnRelationTest extends CQLTester
 
         assertRows(execute("select * from %s where a = ? and c < ? and b in (?, ?)", "first", 7, 3, 2),
                    row("first", 2, 6, 2));
-//---
+
         assertRows(execute("select * from %s where a = ? and c >= ? and c <= ? and b in (?, ?)", "first", 6, 7, 3, 2),
                    row("first", 2, 6, 2),
                    row("first", 3, 7, 3));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a991b648/test/unit/org/apache/cassandra/cql3/validation/operations/SelectTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectTest.java
index 506bdaf..8622c98 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/SelectTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/SelectTest.java
@@ -605,7 +605,7 @@ public class SelectTest extends CQLTester
         execute("INSERT INTO %s (account, id , categories) VALUES (?, ?, ?)", "test", 5, map("lmn", "foo"));
         execute("INSERT INTO %s (account, id , categories) VALUES (?, ?, ?)", "test", 6, map("lmn", "foo2"));
 
-        assertInvalidMessage("No secondary indexes on the restricted columns support the provided operators: 'categories CONTAINS <value>'",
+        assertInvalidMessage("No supported secondary index found for the non primary key columns restrictions",
                              "SELECT * FROM %s WHERE account = ? AND categories CONTAINS ?", "test", "foo");
 
         assertRows(execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS KEY ?", "test", "lmn"),
@@ -628,7 +628,7 @@ public class SelectTest extends CQLTester
         execute("INSERT INTO %s (account, id , categories) VALUES (?, ?, ?)", "test", 5, map("lmn", "foo"));
         execute("INSERT INTO %s (account, id , categories) VALUES (?, ?, ?)", "test", 6, map("lmn2", "foo"));
 
-        assertInvalidMessage("No secondary indexes on the restricted columns support the provided operators: 'categories CONTAINS KEY <value>'",
+        assertInvalidMessage("No supported secondary index found for the non primary key columns restrictions",
                              "SELECT * FROM %s WHERE account = ? AND categories CONTAINS KEY ?", "test", "lmn");
 
         assertRows(execute("SELECT * FROM %s WHERE account = ? AND categories CONTAINS ?", "test", "foo"),

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a991b648/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
deleted file mode 100644
index 0fdabe9..0000000
--- a/test/unit/org/apache/cassandra/db/ArrayBackedSortedColumnsTest.java
+++ /dev/null
@@ -1,426 +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.util.*;
-
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import static org.junit.Assert.*;
-
-import com.google.common.collect.Sets;
-
-import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.KSMetaData;
-import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.db.composites.*;
-import org.apache.cassandra.db.filter.ColumnSlice;
-import org.apache.cassandra.db.marshal.Int32Type;
-import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.locator.SimpleStrategy;
-import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.SearchIterator;
-import org.apache.cassandra.utils.BatchRemoveIterator;
-
-public class ArrayBackedSortedColumnsTest
-{
-    private static final String KEYSPACE1 = "ArrayBackedSortedColumnsTest";
-    private static final String CF_STANDARD1 = "Standard1";
-
-    @BeforeClass
-    public static void defineSchema() throws ConfigurationException
-    {
-        SchemaLoader.prepareServer();
-        SchemaLoader.createKeyspace(KEYSPACE1,
-                                    SimpleStrategy.class,
-                                    KSMetaData.optsWithRF(1),
-                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1));
-    }
-
-    @Test
-    public void testAdd()
-    {
-        testAddInternal(false);
-        testAddInternal(true);
-    }
-
-    private CFMetaData metadata()
-    {
-        return Schema.instance.getCFMetaData(KEYSPACE1, CF_STANDARD1);
-    }
-
-    private void testAddInternal(boolean reversed)
-    {
-        CellNameType type = new SimpleDenseCellNameType(Int32Type.instance);
-        ColumnFamily map = ArrayBackedSortedColumns.factory.create(metadata(), reversed);
-        int[] values = new int[]{ 1, 2, 2, 3 };
-
-        for (int i = 0; i < values.length; ++i)
-            map.addColumn(new BufferCell(type.makeCellName(values[reversed ? values.length - 1 - i : i])));
-
-        Iterator<Cell> iter = map.iterator();
-        assertEquals("1st column", 1, iter.next().name().toByteBuffer().getInt(0));
-        assertEquals("2nd column", 2, iter.next().name().toByteBuffer().getInt(0));
-        assertEquals("3rd column", 3, iter.next().name().toByteBuffer().getInt(0));
-    }
-
-    @Test
-    public void testOutOfOrder()
-    {
-        testAddOutOfOrder(false);
-        testAddOutOfOrder(false);
-    }
-
-    private void testAddOutOfOrder(boolean reversed)
-    {
-        CellNameType type = new SimpleDenseCellNameType(Int32Type.instance);
-        ColumnFamily cells = ArrayBackedSortedColumns.factory.create(metadata(), reversed);
-
-        int[] values = new int[]{ 1, 2, 1, 3, 4, 4, 5, 5, 1, 2, 6, 6, 6, 1, 2, 3 };
-        for (int i = 0; i < values.length; ++i)
-            cells.addColumn(new BufferCell(type.makeCellName(values[reversed ? values.length - 1 - i : i])));
-
-        assertEquals(6, cells.getColumnCount());
-
-        Iterator<Cell> iter = cells.iterator();
-        assertEquals(1, iter.next().name().toByteBuffer().getInt(0));
-        assertEquals(2, iter.next().name().toByteBuffer().getInt(0));
-        assertEquals(3, iter.next().name().toByteBuffer().getInt(0));
-        assertEquals(4, iter.next().name().toByteBuffer().getInt(0));
-        assertEquals(5, iter.next().name().toByteBuffer().getInt(0));
-        assertEquals(6, iter.next().name().toByteBuffer().getInt(0));
-
-        // Add more values
-        values = new int[]{ 11, 15, 12, 12, 12, 16, 10, 8, 8, 7, 4, 4, 5 };
-        for (int i = 0; i < values.length; ++i)
-            cells.addColumn(new BufferCell(type.makeCellName(values[reversed ? values.length - 1 - i : i])));
-
-        assertEquals(13, cells.getColumnCount());
-
-        iter = cells.reverseIterator();
-        assertEquals(16, iter.next().name().toByteBuffer().getInt(0));
-        assertEquals(15, iter.next().name().toByteBuffer().getInt(0));
-        assertEquals(12, iter.next().name().toByteBuffer().getInt(0));
-        assertEquals(11, iter.next().name().toByteBuffer().getInt(0));
-        assertEquals(10, iter.next().name().toByteBuffer().getInt(0));
-        assertEquals(8,  iter.next().name().toByteBuffer().getInt(0));
-        assertEquals(7, iter.next().name().toByteBuffer().getInt(0));
-        assertEquals(6, iter.next().name().toByteBuffer().getInt(0));
-        assertEquals(5, iter.next().name().toByteBuffer().getInt(0));
-        assertEquals(4, iter.next().name().toByteBuffer().getInt(0));
-        assertEquals(3, iter.next().name().toByteBuffer().getInt(0));
-        assertEquals(2, iter.next().name().toByteBuffer().getInt(0));
-        assertEquals(1, iter.next().name().toByteBuffer().getInt(0));
-    }
-
-    @Test
-    public void testGetColumn()
-    {
-        testGetColumnInternal(true);
-        testGetColumnInternal(false);
-    }
-
-    private void testGetColumnInternal(boolean reversed)
-    {
-        CellNameType type = new SimpleDenseCellNameType(Int32Type.instance);
-        ColumnFamily cells = ArrayBackedSortedColumns.factory.create(metadata(), reversed);
-
-        int[] values = new int[]{ -1, 20, 44, 55, 27, 27, 17, 1, 9, 89, 33, 44, 0, 9 };
-        for (int i = 0; i < values.length; ++i)
-            cells.addColumn(new BufferCell(type.makeCellName(values[reversed ? values.length - 1 - i : i])));
-
-        for (int i : values)
-            assertEquals(i, cells.getColumn(type.makeCellName(i)).name().toByteBuffer().getInt(0));
-    }
-
-    @Test
-    public void testAddAll()
-    {
-        testAddAllInternal(false);
-        testAddAllInternal(true);
-    }
-
-    private void testAddAllInternal(boolean reversed)
-    {
-        CellNameType type = new SimpleDenseCellNameType(Int32Type.instance);
-        ColumnFamily map = ArrayBackedSortedColumns.factory.create(metadata(), reversed);
-        ColumnFamily map2 = ArrayBackedSortedColumns.factory.create(metadata(), reversed);
-
-        int[] values1 = new int[]{ 1, 3, 5, 6 };
-        int[] values2 = new int[]{ 2, 4, 5, 6 };
-
-        for (int i = 0; i < values1.length; ++i)
-            map.addColumn(new BufferCell(type.makeCellName(values1[reversed ? values1.length - 1 - i : i])));
-
-        for (int i = 0; i < values2.length; ++i)
-            map2.addColumn(new BufferCell(type.makeCellName(values2[reversed ? values2.length - 1 - i : i])));
-
-        map2.addAll(map);
-
-        Iterator<Cell> iter = map2.iterator();
-        assertEquals("1st column", 1, iter.next().name().toByteBuffer().getInt(0));
-        assertEquals("2nd column", 2, iter.next().name().toByteBuffer().getInt(0));
-        assertEquals("3rd column", 3, iter.next().name().toByteBuffer().getInt(0));
-        assertEquals("4st column", 4, iter.next().name().toByteBuffer().getInt(0));
-        assertEquals("5st column", 5, iter.next().name().toByteBuffer().getInt(0));
-        assertEquals("6st column", 6, iter.next().name().toByteBuffer().getInt(0));
-    }
-
-    @Test
-    public void testGetCollection()
-    {
-        testGetCollectionInternal(false);
-        testGetCollectionInternal(true);
-    }
-
-    private void testGetCollectionInternal(boolean reversed)
-    {
-        CellNameType type = new SimpleDenseCellNameType(Int32Type.instance);
-        ColumnFamily map = ArrayBackedSortedColumns.factory.create(metadata(), reversed);
-        int[] values = new int[]{ 1, 2, 3, 5, 9 };
-
-        List<Cell> sorted = new ArrayList<>();
-        for (int v : values)
-            sorted.add(new BufferCell(type.makeCellName(v)));
-        List<Cell> reverseSorted = new ArrayList<>(sorted);
-        Collections.reverse(reverseSorted);
-
-        for (int i = 0; i < values.length; ++i)
-            map.addColumn(new BufferCell(type.makeCellName(values[reversed ? values.length - 1 - i : i])));
-
-        assertSame(sorted, map.getSortedColumns());
-        assertSame(reverseSorted, map.getReverseSortedColumns());
-    }
-
-    @Test
-    public void testIterator()
-    {
-        testIteratorInternal(false);
-        //testIteratorInternal(true);
-    }
-
-    private void testIteratorInternal(boolean reversed)
-    {
-        CellNameType type = new SimpleDenseCellNameType(Int32Type.instance);
-        ColumnFamily map = ArrayBackedSortedColumns.factory.create(metadata(), reversed);
-
-        int[] values = new int[]{ 1, 2, 3, 5, 9 };
-
-        for (int i = 0; i < values.length; ++i)
-            map.addColumn(new BufferCell(type.makeCellName(values[reversed ? values.length - 1 - i : i])));
-
-        assertSame(new int[]{ 3, 2, 1 }, map.reverseIterator(new ColumnSlice[]{ new ColumnSlice(type.make(3), Composites.EMPTY) }));
-        assertSame(new int[]{ 3, 2, 1 }, map.reverseIterator(new ColumnSlice[]{ new ColumnSlice(type.make(4), Composites.EMPTY) }));
-
-        assertSame(map.iterator(), map.iterator(ColumnSlice.ALL_COLUMNS_ARRAY));
-    }
-
-    @Test
-    public void testSearchIterator()
-    {
-        CellNameType type = new SimpleDenseCellNameType(Int32Type.instance);
-        ColumnFamily map = ArrayBackedSortedColumns.factory.create(metadata(), false);
-
-        int[] values = new int[]{ 1, 2, 3, 5, 9, 15, 21, 22 };
-
-        for (int i = 0; i < values.length; ++i)
-            map.addColumn(new BufferCell(type.makeCellName(values[i])));
-
-        SearchIterator<CellName, Cell> iter = map.searchIterator();
-        for (int i = 0 ; i < values.length ; i++)
-            assertSame(values[i], iter.next(type.makeCellName(values[i])));
-
-        iter = map.searchIterator();
-        for (int i = 0 ; i < values.length ; i+=2)
-            assertSame(values[i], iter.next(type.makeCellName(values[i])));
-
-        iter = map.searchIterator();
-        for (int i = 0 ; i < values.length ; i+=4)
-            assertSame(values[i], iter.next(type.makeCellName(values[i])));
-
-        iter = map.searchIterator();
-        for (int i = 0 ; i < values.length ; i+=1)
-        {
-            if (i % 2 == 0)
-            {
-                Cell cell = iter.next(type.makeCellName(values[i] - 1));
-                if (i > 0 && values[i - 1] == values[i] - 1)
-                    assertSame(values[i - 1], cell);
-                else
-                    assertNull(cell);
-            }
-        }
-    }
-
-    private <T> void assertSame(Iterable<T> c1, Iterable<T> c2)
-    {
-        assertSame(c1.iterator(), c2.iterator());
-    }
-
-    private <T> void assertSame(Iterator<T> iter1, Iterator<T> iter2)
-    {
-        while (iter1.hasNext() && iter2.hasNext())
-            assertEquals(iter1.next(), iter2.next());
-        if (iter1.hasNext() || iter2.hasNext())
-            fail("The collection don't have the same size");
-    }
-
-    private void assertSame(int name, Cell cell)
-    {
-        int value = ByteBufferUtil.toInt(cell.name().toByteBuffer());
-        assert name == value : "Expected " + name + " but got " + value;
-    }
-    private void assertSame(int[] names, Iterator<Cell> iter)
-    {
-        for (int name : names)
-        {
-            assert iter.hasNext() : "Expected " + name + " but no more result";
-            int value = ByteBufferUtil.toInt(iter.next().name().toByteBuffer());
-            assert name == value : "Expected " + name + " but got " + value;
-        }
-    }
-
-    @Test
-    public void testRemove()
-    {
-        testRemoveInternal(false);
-        testRemoveInternal(true);
-    }
-
-    private void testRemoveInternal(boolean reversed)
-    {
-        CellNameType type = new SimpleDenseCellNameType(Int32Type.instance);
-        ColumnFamily map = ArrayBackedSortedColumns.factory.create(metadata(), reversed);
-
-        int[] values = new int[]{ 1, 2, 2, 3 };
-
-        for (int i = 0; i < values.length; ++i)
-            map.addColumn(new BufferCell(type.makeCellName(values[reversed ? values.length - 1 - i : i])));
-
-        Iterator<Cell> iter = map.getReverseSortedColumns().iterator();
-        assertTrue(iter.hasNext());
-        iter.next();
-        iter.remove();
-        assertTrue(iter.hasNext());
-        iter.next();
-        iter.remove();
-        assertTrue(iter.hasNext());
-        iter.next();
-        iter.remove();
-        assertTrue(!iter.hasNext());
-    }
-
-    @Test(expected = IllegalStateException.class)
-    public void testBatchRemoveTwice()
-    {
-        CellNameType type = new SimpleDenseCellNameType(Int32Type.instance);
-        ColumnFamily map = ArrayBackedSortedColumns.factory.create(metadata(), false);
-        map.addColumn(new BufferCell(type.makeCellName(1)));
-        map.addColumn(new BufferCell(type.makeCellName(2)));
-
-        BatchRemoveIterator<Cell> batchIter = map.batchRemoveIterator();
-        batchIter.next();
-        batchIter.remove();
-        batchIter.remove();
-    }
-
-    @Test(expected = IllegalStateException.class)
-    public void testBatchCommitTwice()
-    {
-        CellNameType type = new SimpleDenseCellNameType(Int32Type.instance);
-        ColumnFamily map = ArrayBackedSortedColumns.factory.create(metadata(), false);
-        map.addColumn(new BufferCell(type.makeCellName(1)));
-        map.addColumn(new BufferCell(type.makeCellName(2)));
-
-        BatchRemoveIterator<Cell> batchIter = map.batchRemoveIterator();
-        batchIter.next();
-        batchIter.remove();
-        batchIter.commit();
-        batchIter.commit();
-    }
-
-    @Test
-    public void testBatchRemove()
-    {
-        testBatchRemoveInternal(false);
-        testBatchRemoveInternal(true);
-    }
-
-    public void testBatchRemoveInternal(boolean reversed)
-    {
-        CellNameType type = new SimpleDenseCellNameType(Int32Type.instance);
-        ColumnFamily map = ArrayBackedSortedColumns.factory.create(metadata(), reversed);
-        int[] values = new int[]{ 1, 2, 3, 5 };
-
-        for (int i = 0; i < values.length; ++i)
-            map.addColumn(new BufferCell(type.makeCellName(values[reversed ? values.length - 1 - i : i])));
-
-        BatchRemoveIterator<Cell> batchIter = map.batchRemoveIterator();
-        batchIter.next();
-        batchIter.remove();
-        batchIter.next();
-        batchIter.remove();
-
-        assertEquals("1st column before commit", 1, map.iterator().next().name().toByteBuffer().getInt(0));
-
-        batchIter.commit();
-
-        assertEquals("1st column after commit", 3, map.iterator().next().name().toByteBuffer().getInt(0));
-    }
-
-    @Test
-    public void testBatchRemoveCopy()
-    {
-        // Test delete some random columns and check the result
-        CellNameType type = new SimpleDenseCellNameType(Int32Type.instance);
-        ColumnFamily map = ArrayBackedSortedColumns.factory.create(metadata(), false);
-        int n = 127;
-        int[] values = new int[n];
-        for (int i = 0; i < n; i++)
-            values[i] = i;
-        Set<Integer> toRemove = Sets.newHashSet(3, 12, 13, 15, 58, 103, 112);
-
-        for (int value : values)
-            map.addColumn(new BufferCell(type.makeCellName(value)));
-
-        BatchRemoveIterator<Cell> batchIter = map.batchRemoveIterator();
-        while (batchIter.hasNext())
-            if (toRemove.contains(batchIter.next().name().toByteBuffer().getInt(0)))
-                batchIter.remove();
-
-        batchIter.commit();
-
-        int expected = 0;
-        while (toRemove.contains(expected))
-            expected++;
-
-        for (Cell column : map)
-        {
-            assertEquals(expected, column.name().toByteBuffer().getInt(0));
-            expected++;
-            while (toRemove.contains(expected))
-                expected++;
-        }
-        assertEquals(expected, n);
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a991b648/test/unit/org/apache/cassandra/db/BatchlogManagerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/BatchlogManagerTest.java b/test/unit/org/apache/cassandra/db/BatchlogManagerTest.java
index 70d1d0c..c81fc4f 100644
--- a/test/unit/org/apache/cassandra/db/BatchlogManagerTest.java
+++ b/test/unit/org/apache/cassandra/db/BatchlogManagerTest.java
@@ -19,10 +19,15 @@ package org.apache.cassandra.db;
 
 import java.net.InetAddress;
 import java.util.Collections;
-import java.util.List;
-import java.util.concurrent.ExecutionException;
+import java.util.Iterator;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.db.rows.Row;
+import org.apache.cassandra.db.partitions.ArrayBackedPartition;
+import org.apache.cassandra.db.partitions.PartitionUpdate;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
 
-import com.google.common.collect.Lists;
 import org.junit.BeforeClass;
 import org.junit.Before;
 import org.junit.Test;
@@ -33,8 +38,7 @@ import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.cql3.QueryProcessor;
 import org.apache.cassandra.cql3.UntypedResultSet;
-import org.apache.cassandra.db.composites.CellNameType;
-import org.apache.cassandra.db.commitlog.ReplayPosition;
+import org.apache.cassandra.db.marshal.BytesType;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.locator.TokenMetadata;
 import org.apache.cassandra.locator.SimpleStrategy;
@@ -61,7 +65,7 @@ public class BatchlogManagerTest
         SchemaLoader.createKeyspace(KEYSPACE1,
                 SimpleStrategy.class,
                 KSMetaData.optsWithRF(1),
-                SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1),
+                SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1, 1, BytesType.instance),
                 SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD2),
                 SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD3));
     }
@@ -76,28 +80,58 @@ public class BatchlogManagerTest
     }
 
     @Test
+    public void testDelete()
+    {
+        ColumnFamilyStore cfs = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF_STANDARD1);
+        CFMetaData cfm = cfs.metadata;
+        new RowUpdateBuilder(cfm, FBUtilities.timestampMicros(), ByteBufferUtil.bytes("1234"))
+                .clustering("c")
+                .add("val", "val" + 1234)
+                .build()
+                .applyUnsafe();
+
+        DecoratedKey dk = StorageService.getPartitioner().decorateKey(ByteBufferUtil.bytes("1234"));
+        ArrayBackedPartition results = Util.getOnlyPartitionUnfiltered(Util.cmd(cfs, dk).build());
+        Iterator<Row> iter = results.iterator();
+        assert iter.hasNext();
+
+        Mutation mutation = new Mutation(KEYSPACE1, dk);
+        mutation.add(PartitionUpdate.fullPartitionDelete(cfm,
+                                                         mutation.key(),
+                                                         FBUtilities.timestampMicros(),
+                                                         FBUtilities.nowInSeconds()));
+        mutation.applyUnsafe();
+
+        Util.assertEmpty(Util.cmd(cfs, dk).build());
+    }
+
+    // TODO: Fix. Currently endlessly looping on BatchLogManager.replayAllFailedBatches
+    @Test
     public void testReplay() throws Exception
     {
         long initialAllBatches = BatchlogManager.instance.countAllBatches();
         long initialReplayedBatches = BatchlogManager.instance.getTotalBatchesReplayed();
 
+        CFMetaData cfm = Keyspace.open(KEYSPACE1).getColumnFamilyStore(CF_STANDARD1).metadata;
+
         // Generate 1000 mutations and put them all into the batchlog.
         // Half (500) ready to be replayed, half not.
-        CellNameType comparator = Keyspace.open(KEYSPACE1).getColumnFamilyStore("Standard1").metadata.comparator;
         for (int i = 0; i < 1000; i++)
         {
-            Mutation mutation = new Mutation(KEYSPACE1, bytes(i));
-            mutation.add("Standard1", comparator.makeCellName(bytes(i)), bytes(i), System.currentTimeMillis());
+            Mutation m = new RowUpdateBuilder(cfm, FBUtilities.timestampMicros(), bytes(i))
+                    .clustering("name" + i)
+                    .add("val", "val" + i)
+                    .build();
 
             long timestamp = i < 500
                            ? (System.currentTimeMillis() - DatabaseDescriptor.getWriteRpcTimeout() * 2) * 1000
                            : Long.MAX_VALUE;
 
-            BatchlogManager.getBatchlogMutationFor(Collections.singleton(mutation),
+            Mutation m2 = BatchlogManager.getBatchlogMutationFor(Collections.singleton(m),
                                                    UUIDGen.getTimeUUID(),
                                                    MessagingService.current_version,
-                                                   timestamp)
-                           .applyUnsafe();
+                                                   timestamp);
+            m2.applyUnsafe();
         }
 
         // Flush the batchlog to disk (see CASSANDRA-6822).
@@ -119,8 +153,8 @@ public class BatchlogManagerTest
             if (i < 500)
             {
                 assertEquals(bytes(i), result.one().getBytes("key"));
-                assertEquals(bytes(i), result.one().getBytes("column1"));
-                assertEquals(bytes(i), result.one().getBytes("value"));
+                assertEquals("name" + i, result.one().getString("name"));
+                assertEquals("val" + i, result.one().getString("val"));
             }
             else
             {
@@ -133,6 +167,7 @@ public class BatchlogManagerTest
         assertEquals(500, result.one().getLong("count"));
     }
 
+    /*
     @Test
     public void testTruncatedReplay() throws InterruptedException, ExecutionException
     {
@@ -200,4 +235,5 @@ public class BatchlogManagerTest
             assertEquals(bytes(i), result.one().getBytes("value"));
         }
     }
+    */
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a991b648/test/unit/org/apache/cassandra/db/CellTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/CellTest.java b/test/unit/org/apache/cassandra/db/CellTest.java
index 493dbbf..a7e4bf7 100644
--- a/test/unit/org/apache/cassandra/db/CellTest.java
+++ b/test/unit/org/apache/cassandra/db/CellTest.java
@@ -23,20 +23,36 @@ package org.apache.cassandra.db;
 
 import java.nio.ByteBuffer;
 
+import junit.framework.Assert;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
-import junit.framework.Assert;
-import org.apache.cassandra.Util;
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.config.KSMetaData;
+import org.apache.cassandra.db.rows.*;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.locator.SimpleStrategy;
+import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.concurrent.OpOrder;
-import org.apache.cassandra.utils.memory.NativeAllocator;
-import org.apache.cassandra.utils.memory.NativePool;
+import org.apache.cassandra.utils.FBUtilities;
 
 public class CellTest
 {
+    private static final String KEYSPACE1 = "CellTest";
+    private static final String CF_STANDARD1 = "Standard1";
 
-    private static final OpOrder order = new OpOrder();
-    private static NativeAllocator allocator = new NativePool(Integer.MAX_VALUE, Integer.MAX_VALUE, 1f, null).newAllocator();
+    private CFMetaData cfm = SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1);
+
+    @BeforeClass
+    public static void defineSchema() throws ConfigurationException
+    {
+        SchemaLoader.prepareServer();
+        SchemaLoader.createKeyspace(KEYSPACE1,
+                SimpleStrategy.class,
+                KSMetaData.optsWithRF(1),
+                SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1));
+    }
 
     @Test
     public void testConflictingTypeEquality()
@@ -49,12 +65,12 @@ public class CellTest
                 // don't test equality for both sides native, as this is based on CellName resolution
                 if (lhs && rhs)
                     continue;
-                Cell a = expiring("a", "a", 1, 1, lhs);
-                Cell b = regular("a", "a", 1, rhs);
+                Cell a = buildCell(cfm, "a", "a", 1, 1);
+                Cell b = buildCell(cfm, "a", "a", 1, 0);
                 Assert.assertNotSame(a, b);
                 Assert.assertNotSame(b, a);
-                a = deleted("a", 1, 1, lhs);
-                b = regular("a", ByteBufferUtil.bytes(1), 1, rhs);
+
+                a = deleted(cfm, "a", "a", 1, 1);
                 Assert.assertNotSame(a, b);
                 Assert.assertNotSame(b, a);
             }
@@ -71,7 +87,6 @@ public class CellTest
         Assert.assertEquals(-1, testExpiring("a", "a", 2, 1, null, null, 1L, null));
         Assert.assertEquals(-1, testExpiring("a", "a", 2, 1, null, "b", 1L, 2));
 
-        // newer TTL
         Assert.assertEquals(-1, testExpiring("a", "a", 1, 2, null, null, null, 1));
         Assert.assertEquals(1, testExpiring("a", "a", 1, 2, null, "b", null, 1));
 
@@ -90,52 +105,65 @@ public class CellTest
             t2 = t1;
         if (et2 == null)
             et2 = et1;
-        int result = testExpiring(n1, v1, t1, et1, false, n2, v2, t2, et2, false);
-        Assert.assertEquals(result, testExpiring(n1, v1, t1, et1, false, n2, v2, t2, et2, true));
-        Assert.assertEquals(result, testExpiring(n1, v1, t1, et1, true, n2, v2, t2, et2, false));
-        Assert.assertEquals(result, testExpiring(n1, v1, t1, et1, true, n2, v2, t2, et2, true));
-        return result;
-    }
+        Cell c1 = buildCell(cfm, n1, v1, t1, et1);
+        Cell c2 = buildCell(cfm, n2, v2, t2, et2);
 
-    private int testExpiring(String n1, String v1, long t1, int et1, boolean native1, String n2, String v2, long t2, int et2, boolean native2)
-    {
-        Cell c1 = expiring(n1, v1, t1, et1, native1);
-        Cell c2 = expiring(n2, v2, t2, et2, native2);
-        return reconcile(c1, c2);
+        int now = FBUtilities.nowInSeconds();
+        if (Cells.reconcile(c1, c2, now) == c1)
+            return Cells.reconcile(c2, c1, now) == c1 ? -1 : 0;
+        return Cells.reconcile(c2, c1, now) == c2 ? 1 : 0;
     }
 
-    int reconcile(Cell c1, Cell c2)
+    private Cell buildCell(CFMetaData cfm, String columnName, String value, long timestamp, int ttl)
     {
-        if (c1.reconcile(c2) == c1)
-            return c2.reconcile(c1) == c1 ? -1 : 0;
-        return c2.reconcile(c1) == c2 ? 1 : 0;
+        ColumnDefinition cdef = cfm.getColumnDefinition(ByteBufferUtil.bytes(columnName));
+        LivenessInfo info = SimpleLivenessInfo.forUpdate(timestamp, ttl, FBUtilities.nowInSeconds(), cfm);
+        return new TestCell(cdef, ByteBufferUtil.bytes(value), info);
     }
 
-    private Cell expiring(String name, String value, long timestamp, int expirationTime, boolean nativeCell)
+    private Cell deleted(CFMetaData cfm, String columnName, String value, int localDeletionTime, long timestamp)
     {
-        ExpiringCell cell = new BufferExpiringCell(Util.cellname(name), ByteBufferUtil.bytes(value), timestamp, 1, expirationTime);
-        if (nativeCell)
-            cell = new NativeExpiringCell(allocator, order.getCurrent(), cell);
-        return cell;
+        ColumnDefinition cdef = cfm.getColumnDefinition(ByteBufferUtil.bytes(columnName));
+        LivenessInfo info = SimpleLivenessInfo.forDeletion(timestamp, localDeletionTime);
+        return new TestCell(cdef, ByteBufferUtil.bytes(value), info);
     }
 
-    private Cell regular(String name, ByteBuffer value, long timestamp, boolean nativeCell)
+    public static class TestCell extends AbstractCell
     {
-        Cell cell = new BufferCell(Util.cellname(name), value, timestamp);
-        if (nativeCell)
-            cell = new NativeCell(allocator, order.getCurrent(), cell);
-        return cell;
-    }
-    private Cell regular(String name, String value, long timestamp, boolean nativeCell)
-    {
-        return regular(name, ByteBufferUtil.bytes(value), timestamp, nativeCell);
-    }
+        private final ColumnDefinition column;
+        private final ByteBuffer value;
+        private final LivenessInfo info;
 
-    private Cell deleted(String name, int localDeletionTime, long timestamp, boolean nativeCell)
-    {
-        DeletedCell cell = new BufferDeletedCell(Util.cellname(name), localDeletionTime, timestamp);
-        if (nativeCell)
-            cell = new NativeDeletedCell(allocator, order.getCurrent(), cell);
-        return cell;
+        public TestCell(ColumnDefinition column, ByteBuffer value, LivenessInfo info)
+        {
+            this.column = column;
+            this.value = value;
+            this.info = info.takeAlias();
+        }
+
+        public ColumnDefinition column()
+        {
+            return column;
+        }
+
+        public boolean isCounterCell()
+        {
+            return false;
+        }
+
+        public ByteBuffer value()
+        {
+            return value;
+        }
+
+        public LivenessInfo livenessInfo()
+        {
+            return info;
+        }
+
+        public CellPath path()
+        {
+            return null;
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a991b648/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 81f7d41..d1214cb 100644
--- a/test/unit/org/apache/cassandra/db/CleanupTest.java
+++ b/test/unit/org/apache/cassandra/db/CleanupTest.java
@@ -22,12 +22,14 @@ import java.io.IOException;
 import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.nio.ByteBuffer;
-import java.util.Arrays;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.db.filter.RowFilter;
+
 import org.junit.BeforeClass;
 import org.junit.Test;
 
@@ -35,12 +37,9 @@ import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
 import org.apache.cassandra.config.KSMetaData;
 import org.apache.cassandra.cql3.Operator;
-import org.apache.cassandra.db.filter.IDiskAtomFilter;
-import org.apache.cassandra.db.columniterator.IdentityQueryFilter;
 import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.db.index.SecondaryIndex;
 import org.apache.cassandra.dht.ByteOrderedPartitioner.BytesToken;
-import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.locator.SimpleStrategy;
@@ -54,8 +53,8 @@ public class CleanupTest
 {
     public static final int LOOPS = 200;
     public static final String KEYSPACE1 = "CleanupTest1";
-    public static final String CF1 = "Indexed1";
-    public static final String CF2 = "Standard1";
+    public static final String CF_INDEXED1 = "Indexed1";
+    public static final String CF_STANDARD1 = "Standard1";
     public static final ByteBuffer COLUMN = ByteBufferUtil.bytes("birthdate");
     public static final ByteBuffer VALUE = ByteBuffer.allocate(8);
     static
@@ -71,28 +70,29 @@ public class CleanupTest
         SchemaLoader.createKeyspace(KEYSPACE1,
                                     SimpleStrategy.class,
                                     KSMetaData.optsWithRF(1),
-                                    SchemaLoader.standardCFMD(KEYSPACE1, CF2),
-                                    SchemaLoader.indexCFMD(KEYSPACE1, CF1, true));
+                                    SchemaLoader.standardCFMD(KEYSPACE1, CF_STANDARD1),
+                                    SchemaLoader.compositeIndexCFMD(KEYSPACE1, CF_INDEXED1, true));
     }
 
+    /*
     @Test
     public void testCleanup() throws ExecutionException, InterruptedException
     {
         StorageService.instance.getTokenMetadata().clearUnsafe();
 
         Keyspace keyspace = Keyspace.open(KEYSPACE1);
-        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF2);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF_STANDARD1);
 
-        List<Row> rows;
+        UnfilteredPartitionIterator iter;
 
         // insert data and verify we get it back w/ range query
-        fillCF(cfs, LOOPS);
+        fillCF(cfs, "val", LOOPS);
 
         // record max timestamps of the sstables pre-cleanup
         List<Long> expectedMaxTimestamps = getMaxTimestampList(cfs);
 
-        rows = Util.getRangeSlice(cfs);
-        assertEquals(LOOPS, rows.size());
+        iter = Util.getRangeSlice(cfs);
+        assertEquals(LOOPS, Iterators.size(iter));
 
         // with one token in the ring, owned by the local node, cleanup should be a no-op
         CompactionManager.instance.performCleanup(cfs);
@@ -101,35 +101,31 @@ public class CleanupTest
         assert expectedMaxTimestamps.equals(getMaxTimestampList(cfs));
 
         // check data is still there
-        rows = Util.getRangeSlice(cfs);
-        assertEquals(LOOPS, rows.size());
+        iter = Util.getRangeSlice(cfs);
+        assertEquals(LOOPS, Iterators.size(iter));
     }
+    */
 
     @Test
     public void testCleanupWithIndexes() throws IOException, ExecutionException, InterruptedException
     {
         Keyspace keyspace = Keyspace.open(KEYSPACE1);
-        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF1);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF_INDEXED1);
 
-        List<Row> rows;
 
         // insert data and verify we get it back w/ range query
-        fillCF(cfs, LOOPS);
-        rows = Util.getRangeSlice(cfs);
-        assertEquals(LOOPS, rows.size());
+        fillCF(cfs, "birthdate", LOOPS);
+        assertEquals(LOOPS, Util.getAll(Util.cmd(cfs).build()).size());
 
-        SecondaryIndex index = cfs.indexManager.getIndexForColumn(COLUMN);
+        SecondaryIndex index = cfs.indexManager.getIndexForColumn(cfs.metadata.getColumnDefinition(COLUMN));
         long start = System.nanoTime();
         while (!index.isIndexBuilt(COLUMN) && System.nanoTime() - start < TimeUnit.SECONDS.toNanos(10))
             Thread.sleep(10);
 
-        // verify we get it back w/ index query too
-        IndexExpression expr = new IndexExpression(COLUMN, Operator.EQ, VALUE);
-        List<IndexExpression> clause = Arrays.asList(expr);
-        IDiskAtomFilter filter = new IdentityQueryFilter();
-        Range<RowPosition> range = Util.range("", "");
-        rows = keyspace.getColumnFamilyStore(CF1).search(range, clause, filter, Integer.MAX_VALUE);
-        assertEquals(LOOPS, rows.size());
+        ColumnDefinition cdef = cfs.metadata.getColumnDefinition(COLUMN);
+        RowFilter cf = RowFilter.create();
+        cf.add(cdef, Operator.EQ, VALUE);
+        assertEquals(LOOPS, Util.getAll(Util.cmd(cfs).filterOn("birthdate", Operator.EQ, VALUE).build()).size());
 
         // we don't allow cleanup when the local host has no range to avoid wipping up all data when a node has not join the ring.
         // So to make sure cleanup erase everything here, we give the localhost the tiniest possible range.
@@ -143,15 +139,13 @@ public class CleanupTest
         CompactionManager.instance.performCleanup(cfs);
 
         // row data should be gone
-        rows = Util.getRangeSlice(cfs);
-        assertEquals(0, rows.size());
+        assertEquals(0, Util.getAll(Util.cmd(cfs).build()).size());
 
         // not only should it be gone but there should be no data on disk, not even tombstones
         assert cfs.getSSTables().isEmpty();
 
         // 2ary indexes should result in no results, too (although tombstones won't be gone until compacted)
-        rows = cfs.search(range, clause, filter, Integer.MAX_VALUE);
-        assertEquals(0, rows.size());
+        assertEquals(0, Util.getAll(Util.cmd(cfs).filterOn("birthdate", Operator.EQ, VALUE).build()).size());
     }
 
     @Test
@@ -160,16 +154,12 @@ public class CleanupTest
         StorageService.instance.getTokenMetadata().clearUnsafe();
 
         Keyspace keyspace = Keyspace.open(KEYSPACE1);
-        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF2);
-
-        List<Row> rows;
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF_STANDARD1);
 
         // insert data and verify we get it back w/ range query
-        fillCF(cfs, LOOPS);
-
-        rows = Util.getRangeSlice(cfs);
+        fillCF(cfs, "val", LOOPS);
 
-        assertEquals(LOOPS, rows.size());
+        assertEquals(LOOPS, Util.getAll(Util.cmd(cfs).build()).size());
         TokenMetadata tmd = StorageService.instance.getTokenMetadata();
 
         byte[] tk1 = new byte[1], tk2 = new byte[1];
@@ -179,11 +169,10 @@ public class CleanupTest
         tmd.updateNormalToken(new BytesToken(tk2), InetAddress.getByName("127.0.0.2"));
         CompactionManager.instance.performCleanup(cfs);
 
-        rows = Util.getRangeSlice(cfs);
-        assertEquals(0, rows.size());
+        assertEquals(0, Util.getAll(Util.cmd(cfs).build()).size());
     }
 
-    protected void fillCF(ColumnFamilyStore cfs, int rowsPerSSTable)
+    protected void fillCF(ColumnFamilyStore cfs, String colName, int rowsPerSSTable)
     {
         CompactionManager.instance.disableAutoCompaction();
 
@@ -191,10 +180,11 @@ public class CleanupTest
         {
             String key = String.valueOf(i);
             // create a row and update the birthdate value, test that the index query fetches the new version
-            Mutation rm;
-            rm = new Mutation(KEYSPACE1, ByteBufferUtil.bytes(key));
-            rm.add(cfs.name, Util.cellname(COLUMN), VALUE, System.currentTimeMillis());
-            rm.applyUnsafe();
+            new RowUpdateBuilder(cfs.metadata, System.currentTimeMillis(), ByteBufferUtil.bytes(key))
+                    .clustering(COLUMN)
+                    .add(colName, VALUE)
+                    .build()
+                    .applyUnsafe();
         }
 
         cfs.forceBlockingFlush();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a991b648/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
deleted file mode 100644
index c227816..0000000
--- a/test/unit/org/apache/cassandra/db/CollationControllerTest.java
+++ /dev/null
@@ -1,138 +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.BeforeClass;
-import org.junit.Test;
-
-import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.Util;
-import org.apache.cassandra.config.KSMetaData;
-import org.apache.cassandra.db.composites.CellName;
-import org.apache.cassandra.db.filter.QueryFilter;
-import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.locator.SimpleStrategy;
-import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.FBUtilities;
-
-import static org.junit.Assert.assertEquals;
-
-public class CollationControllerTest
-{
-    private static final String KEYSPACE1 = "CollationControllerTest";
-    private static final String CF = "Standard1";
-    private static final String CFGCGRACE = "StandardGCGS0";
-
-    @BeforeClass
-    public static void defineSchema() throws ConfigurationException
-    {
-        SchemaLoader.prepareServer();
-        SchemaLoader.createKeyspace(KEYSPACE1,
-                                    SimpleStrategy.class,
-                                    KSMetaData.optsWithRF(1),
-                                    SchemaLoader.standardCFMD(KEYSPACE1, CF),
-                                    SchemaLoader.standardCFMD(KEYSPACE1, CFGCGRACE).gcGraceSeconds(0));
-    }
-
-    @Test
-    public void getTopLevelColumnsSkipsSSTablesModifiedBeforeRowDelete() 
-    {
-        Keyspace keyspace = Keyspace.open(KEYSPACE1);
-        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF);
-        Mutation rm;
-        DecoratedKey dk = Util.dk("key1");
-        
-        // add data
-        rm = new Mutation(keyspace.getName(), dk.getKey());
-        rm.add(cfs.name, Util.cellname("Column1"), ByteBufferUtil.bytes("asdf"), 0);
-        rm.applyUnsafe();
-        cfs.forceBlockingFlush();
-        
-        // remove
-        rm = new Mutation(keyspace.getName(), dk.getKey());
-        rm.delete(cfs.name, 10);
-        rm.applyUnsafe();
-        
-        // add another mutation because sstable maxtimestamp isn't set
-        // correctly during flush if the most recent mutation is a row delete
-        rm = new Mutation(keyspace.getName(), Util.dk("key2").getKey());
-        rm.add(cfs.name, Util.cellname("Column1"), ByteBufferUtil.bytes("zxcv"), 20);
-        rm.applyUnsafe();
-        
-        cfs.forceBlockingFlush();
-
-        // add yet one more mutation
-        rm = new Mutation(keyspace.getName(), dk.getKey());
-        rm.add(cfs.name, Util.cellname("Column1"), ByteBufferUtil.bytes("foobar"), 30);
-        rm.applyUnsafe();
-        cfs.forceBlockingFlush();
-
-        // A NamesQueryFilter goes down one code path (through collectTimeOrderedData())
-        // It should only iterate the last flushed sstable, since it probably contains the most recent value for Column1
-        QueryFilter filter = Util.namesQueryFilter(cfs, dk, "Column1");
-        CollationController controller = new CollationController(cfs, filter, Integer.MIN_VALUE);
-        controller.getTopLevelColumns(true);
-        assertEquals(1, controller.getSstablesIterated());
-
-        // 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, cfs.name, System.currentTimeMillis());
-        controller = new CollationController(cfs, filter, Integer.MIN_VALUE);
-        controller.getTopLevelColumns(true);
-        assertEquals(2, controller.getSstablesIterated());
-    }
-
-    @Test
-    public void ensureTombstonesAppliedAfterGCGS()
-    {
-        Keyspace keyspace = Keyspace.open(KEYSPACE1);
-        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CFGCGRACE);
-        cfs.disableAutoCompaction();
-
-        Mutation rm;
-        DecoratedKey dk = Util.dk("key1");
-        CellName cellName = Util.cellname("Column1");
-
-        // add data
-        rm = new Mutation(keyspace.getName(), dk.getKey());
-        rm.add(cfs.name, cellName, ByteBufferUtil.bytes("asdf"), 0);
-        rm.applyUnsafe();
-        cfs.forceBlockingFlush();
-
-        // remove
-        rm = new Mutation(keyspace.getName(), dk.getKey());
-        rm.delete(cfs.name, cellName, 0);
-        rm.applyUnsafe();
-        cfs.forceBlockingFlush();
-
-        // use "realistic" query times since we'll compare these numbers to the local deletion time of the tombstone
-        QueryFilter filter;
-        long queryAt = System.currentTimeMillis() + 1000;
-        int gcBefore = cfs.gcBefore(queryAt);
-
-        filter = QueryFilter.getNamesFilter(dk, cfs.name, FBUtilities.singleton(cellName, cfs.getComparator()), queryAt);
-        CollationController controller = new CollationController(cfs, filter, gcBefore);
-        assert ColumnFamilyStore.removeDeleted(controller.getTopLevelColumns(true), gcBefore) == null;
-
-        filter = QueryFilter.getIdentityFilter(dk, cfs.name, queryAt);
-        controller = new CollationController(cfs, filter, gcBefore);
-        assert ColumnFamilyStore.removeDeleted(controller.getTopLevelColumns(true), gcBefore) == null;
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a991b648/test/unit/org/apache/cassandra/db/ColumnFamilyMetricTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/ColumnFamilyMetricTest.java b/test/unit/org/apache/cassandra/db/ColumnFamilyMetricTest.java
index b644264..9c642c7 100644
--- a/test/unit/org/apache/cassandra/db/ColumnFamilyMetricTest.java
+++ b/test/unit/org/apache/cassandra/db/ColumnFamilyMetricTest.java
@@ -21,6 +21,7 @@ import java.nio.ByteBuffer;
 import java.util.Collection;
 import java.util.function.Supplier;
 
+import org.apache.cassandra.utils.FBUtilities;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.apache.cassandra.SchemaLoader;
@@ -31,7 +32,6 @@ import org.apache.cassandra.locator.SimpleStrategy;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 import static org.junit.Assert.assertEquals;
-import static org.apache.cassandra.Util.cellname;
 
 public class ColumnFamilyMetricTest
 {
@@ -49,23 +49,24 @@ public class ColumnFamilyMetricTest
     public void testSizeMetric()
     {
         Keyspace keyspace = Keyspace.open("Keyspace1");
-        ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard2");
-        store.disableAutoCompaction();
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("Standard2");
+        cfs.disableAutoCompaction();
 
-        store.truncateBlocking();
+        cfs.truncateBlocking();
 
-        assertEquals(0, store.metric.liveDiskSpaceUsed.getCount());
-        assertEquals(0, store.metric.totalDiskSpaceUsed.getCount());
+        assertEquals(0, cfs.metric.liveDiskSpaceUsed.getCount());
+        assertEquals(0, cfs.metric.totalDiskSpaceUsed.getCount());
 
         for (int j = 0; j < 10; j++)
         {
-            ByteBuffer key = ByteBufferUtil.bytes(String.valueOf(j));
-            Mutation rm = new Mutation("Keyspace1", key);
-            rm.add("Standard2", cellname("0"), ByteBufferUtil.EMPTY_BYTE_BUFFER, j);
-            rm.apply();
+            new RowUpdateBuilder(cfs.metadata, FBUtilities.timestampMicros(), String.valueOf(j))
+                    .clustering("0")
+                    .add("val", ByteBufferUtil.EMPTY_BYTE_BUFFER)
+                    .build()
+                    .applyUnsafe();
         }
-        store.forceBlockingFlush();
-        Collection<SSTableReader> sstables = store.getSSTables();
+        cfs.forceBlockingFlush();
+        Collection<SSTableReader> sstables = cfs.getSSTables();
         long size = 0;
         for (SSTableReader reader : sstables)
         {
@@ -73,15 +74,15 @@ public class ColumnFamilyMetricTest
         }
 
         // size metrics should show the sum of all SSTable sizes
-        assertEquals(size, store.metric.liveDiskSpaceUsed.getCount());
-        assertEquals(size, store.metric.totalDiskSpaceUsed.getCount());
+        assertEquals(size, cfs.metric.liveDiskSpaceUsed.getCount());
+        assertEquals(size, cfs.metric.totalDiskSpaceUsed.getCount());
 
-        store.truncateBlocking();
+        cfs.truncateBlocking();
 
         // after truncate, size metrics should be down to 0
-        Util.spinAssertEquals(0L, () -> store.metric.liveDiskSpaceUsed.getCount(), 30);
-        Util.spinAssertEquals(0L, () -> store.metric.totalDiskSpaceUsed.getCount(), 30);
+        Util.spinAssertEquals(0L, () -> cfs.metric.liveDiskSpaceUsed.getCount(), 30);
+        Util.spinAssertEquals(0L, () -> cfs.metric.totalDiskSpaceUsed.getCount(), 30);
 
-        store.enableAutoCompaction();
+        cfs.enableAutoCompaction();
     }
 }