You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by sl...@apache.org on 2013/12/17 17:53:10 UTC

[2/6] Rename Column to Cell

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/test/unit/org/apache/cassandra/Util.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/Util.java b/test/unit/org/apache/cassandra/Util.java
index 87d19d2..f6e5b51 100644
--- a/test/unit/org/apache/cassandra/Util.java
+++ b/test/unit/org/apache/cassandra/Util.java
@@ -25,7 +25,6 @@ import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
@@ -45,7 +44,6 @@ import org.apache.cassandra.db.filter.IDiskAtomFilter;
 import org.apache.cassandra.db.filter.QueryFilter;
 import org.apache.cassandra.db.filter.SliceQueryFilter;
 import org.apache.cassandra.db.filter.NamesQueryFilter;
-import org.apache.cassandra.db.marshal.CompositeType;
 import org.apache.cassandra.dht.*;
 import org.apache.cassandra.gms.ApplicationState;
 import org.apache.cassandra.gms.Gossiper;
@@ -108,19 +106,19 @@ public class Util
         return CellNames.simpleDense(ByteBufferUtil.bytes(l));
     }
 
-    public static Column column(String name, String value, long timestamp)
+    public static Cell column(String name, String value, long timestamp)
     {
-        return new Column(cellname(name), ByteBufferUtil.bytes(value), timestamp);
+        return new Cell(cellname(name), ByteBufferUtil.bytes(value), timestamp);
     }
 
-    public static Column expiringColumn(String name, String value, long timestamp, int ttl)
+    public static Cell expiringColumn(String name, String value, long timestamp, int ttl)
     {
-        return new ExpiringColumn(cellname(name), ByteBufferUtil.bytes(value), timestamp, ttl);
+        return new ExpiringCell(cellname(name), ByteBufferUtil.bytes(value), timestamp, ttl);
     }
 
-    public static Column counterColumn(String name, long value, long timestamp)
+    public static Cell counterColumn(String name, long value, long timestamp)
     {
-        return new CounterUpdateColumn(cellname(name), value, timestamp);
+        return new CounterUpdateCell(cellname(name), value, timestamp);
     }
 
     public static Token token(String key)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/test/unit/org/apache/cassandra/config/DefsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/config/DefsTest.java b/test/unit/org/apache/cassandra/config/DefsTest.java
index fd2cb82..edb2b8d 100644
--- a/test/unit/org/apache/cassandra/config/DefsTest.java
+++ b/test/unit/org/apache/cassandra/config/DefsTest.java
@@ -29,7 +29,6 @@ import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.composites.*;
-import org.apache.cassandra.db.filter.QueryFilter;
 import org.apache.cassandra.db.marshal.BytesType;
 import org.apache.cassandra.db.marshal.UTF8Type;
 import org.apache.cassandra.db.marshal.TimeUUIDType;
@@ -181,7 +180,7 @@ public class DefsTest extends SchemaLoader
 
         ColumnFamily cfam = store.getColumnFamily(Util.namesQueryFilter(store, dk, col0));
         assert cfam.getColumn(col0) != null;
-        Column col = cfam.getColumn(col0);
+        Cell col = cfam.getColumn(col0);
         assert ByteBufferUtil.bytes("value0").equals(col.value());
     }
 
@@ -255,7 +254,7 @@ public class DefsTest extends SchemaLoader
 
         ColumnFamily cfam = store.getColumnFamily(Util.namesQueryFilter(store, dk, col0));
         assert cfam.getColumn(col0) != null;
-        Column col = cfam.getColumn(col0);
+        Cell col = cfam.getColumn(col0);
         assert ByteBufferUtil.bytes("value0").equals(col.value());
     }
 
@@ -364,7 +363,7 @@ public class DefsTest extends SchemaLoader
 
         ColumnFamily cfam = store.getColumnFamily(Util.namesQueryFilter(store, dk, col0));
         assert cfam.getColumn(col0) != null;
-        Column col = cfam.getColumn(col0);
+        Cell col = cfam.getColumn(col0);
         assert ByteBufferUtil.bytes("value0").equals(col.value());
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/test/unit/org/apache/cassandra/db/ArrayBackedSortedColumnsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/ArrayBackedSortedColumnsTest.java b/test/unit/org/apache/cassandra/db/ArrayBackedSortedColumnsTest.java
index 9074cf1..2befc29 100644
--- a/test/unit/org/apache/cassandra/db/ArrayBackedSortedColumnsTest.java
+++ b/test/unit/org/apache/cassandra/db/ArrayBackedSortedColumnsTest.java
@@ -59,9 +59,9 @@ public class ArrayBackedSortedColumnsTest extends SchemaLoader
         int[] values = new int[]{ 1, 2, 2, 3 };
 
         for (int i = 0; i < values.length; ++i)
-            map.addColumn(new Column(type.makeCellName(values[reversed ? values.length - 1 - i : i])), HeapAllocator.instance);
+            map.addColumn(new Cell(type.makeCellName(values[reversed ? values.length - 1 - i : i])), HeapAllocator.instance);
 
-        Iterator<Column> iter = map.iterator();
+        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));
@@ -84,14 +84,14 @@ public class ArrayBackedSortedColumnsTest extends SchemaLoader
         int[] values2 = new int[]{ 2, 4, 5, 6 };
 
         for (int i = 0; i < values1.length; ++i)
-            map.addColumn(new Column(type.makeCellName(values1[reversed ? values1.length - 1 - i : i])), HeapAllocator.instance);
+            map.addColumn(new Cell(type.makeCellName(values1[reversed ? values1.length - 1 - i : i])), HeapAllocator.instance);
 
         for (int i = 0; i < values2.length; ++i)
-            map2.addColumn(new Column(type.makeCellName(values2[reversed ? values2.length - 1 - i : i])), HeapAllocator.instance);
+            map2.addColumn(new Cell(type.makeCellName(values2[reversed ? values2.length - 1 - i : i])), HeapAllocator.instance);
 
-        map2.addAll(map, HeapAllocator.instance, Functions.<Column>identity());
+        map2.addAll(map, HeapAllocator.instance, Functions.<Cell>identity());
 
-        Iterator<Column> iter = map2.iterator();
+        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));
@@ -113,14 +113,14 @@ public class ArrayBackedSortedColumnsTest extends SchemaLoader
         ColumnFamily map = ArrayBackedSortedColumns.factory.create(metadata(), reversed);
         int[] values = new int[]{ 1, 2, 3, 5, 9 };
 
-        List<Column> sorted = new ArrayList<Column>();
+        List<Cell> sorted = new ArrayList<Cell>();
         for (int v : values)
-            sorted.add(new Column(type.makeCellName(v)));
-        List<Column> reverseSorted = new ArrayList<Column>(sorted);
+            sorted.add(new Cell(type.makeCellName(v)));
+        List<Cell> reverseSorted = new ArrayList<Cell>(sorted);
         Collections.reverse(reverseSorted);
 
         for (int i = 0; i < values.length; ++i)
-            map.addColumn(new Column(type.makeCellName(values[reversed ? values.length - 1 - i : i])), HeapAllocator.instance);
+            map.addColumn(new Cell(type.makeCellName(values[reversed ? values.length - 1 - i : i])), HeapAllocator.instance);
 
         assertSame(sorted, map.getSortedColumns());
         assertSame(reverseSorted, map.getReverseSortedColumns());
@@ -141,7 +141,7 @@ public class ArrayBackedSortedColumnsTest extends SchemaLoader
         int[] values = new int[]{ 1, 2, 3, 5, 9 };
 
         for (int i = 0; i < values.length; ++i)
-            map.addColumn(new Column(type.makeCellName(values[reversed ? values.length - 1 - i : i])), HeapAllocator.instance);
+            map.addColumn(new Cell(type.makeCellName(values[reversed ? values.length - 1 - i : i])), HeapAllocator.instance);
 
         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) }));
@@ -162,7 +162,7 @@ public class ArrayBackedSortedColumnsTest extends SchemaLoader
             fail("The collection don't have the same size");
     }
 
-    private void assertSame(int[] names, Iterator<Column> iter)
+    private void assertSame(int[] names, Iterator<Cell> iter)
     {
         for (int name : names)
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java b/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
index e807e2e..d1a2af7 100644
--- a/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
+++ b/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
@@ -45,7 +45,6 @@ import org.apache.cassandra.db.composites.*;
 import org.apache.cassandra.db.columniterator.IdentityQueryFilter;
 import org.apache.cassandra.db.filter.*;
 import org.apache.cassandra.db.index.SecondaryIndex;
-import org.apache.cassandra.db.marshal.CompositeType;
 import org.apache.cassandra.db.marshal.LexicalUUIDType;
 import org.apache.cassandra.db.marshal.LongType;
 import org.apache.cassandra.dht.*;
@@ -60,7 +59,6 @@ import org.apache.cassandra.utils.WrappedRunnable;
 import static org.junit.Assert.*;
 import static org.apache.cassandra.Util.*;
 import static org.apache.cassandra.utils.ByteBufferUtil.bytes;
-import static org.apache.commons.lang3.ArrayUtils.EMPTY_BYTE_ARRAY;
 
 @RunWith(OrderedJUnit4ClassRunner.class)
 public class ColumnFamilyStoreTest extends SchemaLoader
@@ -306,7 +304,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         assert rows.isEmpty();
 
         // verify that it's not being indexed under the deletion column value either
-        Column deletion = rm.getColumnFamilies().iterator().next().iterator().next();
+        Cell deletion = rm.getColumnFamilies().iterator().next().iterator().next();
         ByteBuffer deletionLong = ByteBufferUtil.bytes((long) ByteBufferUtil.toInt(deletion.value()));
         IndexExpression expr0 = new IndexExpression(ByteBufferUtil.bytes("birthdate"), IndexExpression.Operator.EQ, deletionLong);
         List<IndexExpression> clause0 = Arrays.asList(expr0);
@@ -703,16 +701,16 @@ public class ColumnFamilyStoreTest extends SchemaLoader
 
         // create an isolated sstable.
         putColsSuper(cfs, key, scfName,
-                new Column(cellname(1L), ByteBufferUtil.bytes("val1"), 1),
-                new Column(cellname(2L), ByteBufferUtil.bytes("val2"), 1),
-                new Column(cellname(3L), ByteBufferUtil.bytes("val3"), 1));
+                new Cell(cellname(1L), ByteBufferUtil.bytes("val1"), 1),
+                new Cell(cellname(2L), ByteBufferUtil.bytes("val2"), 1),
+                new Cell(cellname(3L), ByteBufferUtil.bytes("val3"), 1));
         cfs.forceBlockingFlush();
 
         // insert, don't flush.
         putColsSuper(cfs, key, scfName,
-                new Column(cellname(4L), ByteBufferUtil.bytes("val4"), 1),
-                new Column(cellname(5L), ByteBufferUtil.bytes("val5"), 1),
-                new Column(cellname(6L), ByteBufferUtil.bytes("val6"), 1));
+                new Cell(cellname(4L), ByteBufferUtil.bytes("val4"), 1),
+                new Cell(cellname(5L), ByteBufferUtil.bytes("val5"), 1),
+                new Cell(cellname(6L), ByteBufferUtil.bytes("val6"), 1));
 
         // verify insert.
         final SlicePredicate sp = new SlicePredicate();
@@ -739,17 +737,17 @@ public class ColumnFamilyStoreTest extends SchemaLoader
 
         // late insert.
         putColsSuper(cfs, key, scfName,
-                new Column(cellname(4L), ByteBufferUtil.bytes("val4"), 1L),
-                new Column(cellname(7L), ByteBufferUtil.bytes("val7"), 1L));
+                new Cell(cellname(4L), ByteBufferUtil.bytes("val4"), 1L),
+                new Cell(cellname(7L), ByteBufferUtil.bytes("val7"), 1L));
 
         // re-verify delete.
         assertRowAndColCount(1, 0, false, cfs.getRangeSlice(Util.range("f", "g"), null, ThriftValidation.asIFilter(sp, cfs.metadata, scfName), 100));
 
         // make sure new writes are recognized.
         putColsSuper(cfs, key, scfName,
-                new Column(cellname(3L), ByteBufferUtil.bytes("val3"), 3),
-                new Column(cellname(8L), ByteBufferUtil.bytes("val8"), 3),
-                new Column(cellname(9L), ByteBufferUtil.bytes("val9"), 3));
+                new Cell(cellname(3L), ByteBufferUtil.bytes("val3"), 3),
+                new Cell(cellname(8L), ByteBufferUtil.bytes("val8"), 3),
+                new Cell(cellname(9L), ByteBufferUtil.bytes("val9"), 3));
         assertRowAndColCount(1, 3, false, cfs.getRangeSlice(Util.range("f", "g"), null, ThriftValidation.asIFilter(sp, cfs.metadata, scfName), 100));
     }
 
@@ -768,24 +766,24 @@ public class ColumnFamilyStoreTest extends SchemaLoader
     private static String str(ColumnFamily cf) throws CharacterCodingException
     {
         StringBuilder sb = new StringBuilder();
-        for (Column col : cf.getSortedColumns())
+        for (Cell col : cf.getSortedColumns())
             sb.append(String.format("(%s,%s,%d),", ByteBufferUtil.string(col.name().toByteBuffer()), ByteBufferUtil.string(col.value()), col.timestamp()));
         return sb.toString();
     }
 
-    private static void putColsSuper(ColumnFamilyStore cfs, DecoratedKey key, ByteBuffer scfName, Column... cols) throws Throwable
+    private static void putColsSuper(ColumnFamilyStore cfs, DecoratedKey key, ByteBuffer scfName, Cell... cols) throws Throwable
     {
         ColumnFamily cf = TreeMapBackedSortedColumns.factory.create(cfs.keyspace.getName(), cfs.name);
-        for (Column col : cols)
+        for (Cell col : cols)
             cf.addColumn(col.withUpdatedName(CellNames.compositeDense(scfName, col.name().toByteBuffer())));
         RowMutation rm = new RowMutation(cfs.keyspace.getName(), key.key, cf);
         rm.apply();
     }
 
-    private static void putColsStandard(ColumnFamilyStore cfs, DecoratedKey key, Column... cols) throws Throwable
+    private static void putColsStandard(ColumnFamilyStore cfs, DecoratedKey key, Cell... cols) throws Throwable
     {
         ColumnFamily cf = TreeMapBackedSortedColumns.factory.create(cfs.keyspace.getName(), cfs.name);
-        for (Column col : cols)
+        for (Cell col : cols)
             cf.addColumn(col);
         RowMutation rm = new RowMutation(cfs.keyspace.getName(), key.key, cf);
         rm.apply();
@@ -890,8 +888,8 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         DecoratedKey key = Util.dk("slice-get-uuid-type");
 
         // Insert a row with one supercolumn and multiple subcolumns
-        putColsSuper(cfs, key, superColName, new Column(cellname("a"), ByteBufferUtil.bytes("A"), 1),
-                                             new Column(cellname("b"), ByteBufferUtil.bytes("B"), 1));
+        putColsSuper(cfs, key, superColName, new Cell(cellname("a"), ByteBufferUtil.bytes("A"), 1),
+                                             new Cell(cellname("b"), ByteBufferUtil.bytes("B"), 1));
 
         // Get the entire supercolumn like normal
         ColumnFamily cfGet = cfs.getColumnFamily(QueryFilter.getIdentityFilter(key, cfName, System.currentTimeMillis()));
@@ -921,8 +919,8 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfName);
         cfs.clearUnsafe();
 
-        // Create a column a 'high timestamp'
-        putColsStandard(cfs, key, new Column(cname, ByteBufferUtil.bytes("a"), 2));
+        // Create a cell a 'high timestamp'
+        putColsStandard(cfs, key, new Cell(cname, ByteBufferUtil.bytes("a"), 2));
         cfs.forceBlockingFlush();
 
         // Nuke the metadata and reload that sstable
@@ -934,14 +932,14 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         new File(ssTables.iterator().next().descriptor.filenameFor(Component.STATS)).delete();
         cfs.loadNewSSTables();
 
-        // Add another column with a lower timestamp
-        putColsStandard(cfs, key, new Column(cname, ByteBufferUtil.bytes("b"), 1));
+        // Add another cell with a lower timestamp
+        putColsStandard(cfs, key, new Cell(cname, ByteBufferUtil.bytes("b"), 1));
 
-        // Test fetching the column by name returns the first column
+        // Test fetching the cell by name returns the first cell
         SliceByNamesReadCommand cmd = new SliceByNamesReadCommand(keyspaceName, key.key, cfName, System.currentTimeMillis(), new NamesQueryFilter(FBUtilities.singleton(cname, cfs.getComparator())));
         ColumnFamily cf = cmd.getRow(keyspace).cf;
-        Column column = cf.getColumn(cname);
-        assert column.value().equals(ByteBufferUtil.bytes("a")) : "expecting a, got " + ByteBufferUtil.string(column.value());
+        Cell cell = cf.getColumn(cname);
+        assert cell.value().equals(ByteBufferUtil.bytes("a")) : "expecting a, got " + ByteBufferUtil.string(cell.value());
     }
 
     private static void assertTotalColCount(Collection<Row> rows, int expectedCount)
@@ -964,7 +962,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfName);
         cfs.clearUnsafe();
 
-        Column[] cols = new Column[5];
+        Cell[] cols = new Cell[5];
         for (int i = 0; i < 5; i++)
             cols[i] = column("c" + i, "value", 1);
 
@@ -1080,7 +1078,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfName);
         cfs.clearUnsafe();
 
-        Column[] cols = new Column[4];
+        Cell[] cols = new Cell[4];
         for (int i = 0; i < 4; i++)
             cols[i] = column("c" + i, "value", 1);
 
@@ -1171,7 +1169,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
                 sb.append(":");
                 if (row.cf != null && !row.cf.isEmpty())
                 {
-                    for (Column c : row.cf)
+                    for (Cell c : row.cf)
                         sb.append(" ").append(row.cf.getComparator().getString(c.name()));
                 }
                 sb.append("} ");
@@ -1189,12 +1187,12 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         if (row == null || row.cf == null)
             throw new AssertionError("The row should not be empty");
 
-        Iterator<Column> columns = row.cf.getSortedColumns().iterator();
+        Iterator<Cell> columns = row.cf.getSortedColumns().iterator();
         Iterator<String> names = Arrays.asList(columnNames).iterator();
 
         while (columns.hasNext())
         {
-            Column c = columns.next();
+            Cell c = columns.next();
             assert names.hasNext() : "Got more columns that expected (first unexpected column: " + ByteBufferUtil.string(c.name().toByteBuffer()) + ")";
             String n = names.next();
             assert c.name().toByteBuffer().equals(ByteBufferUtil.bytes(n)) : "Expected " + n + ", got " + ByteBufferUtil.string(c.name().toByteBuffer());
@@ -1216,7 +1214,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfName);
         cfs.clearUnsafe();
 
-        Column[] cols = new Column[5];
+        Cell[] cols = new Cell[5];
         for (int i = 0; i < 5; i++)
             cols[i] = column("c" + i, "value", 1);
 
@@ -1313,10 +1311,10 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         cfs.clearUnsafe();
 
         String[] letters = new String[] { "a", "b", "c", "d", "i" };
-        Column[] cols = new Column[letters.length];
+        Cell[] cols = new Cell[letters.length];
         for (int i = 0; i < cols.length; i++)
         {
-            cols[i] = new Column(cellname("col" + letters[i].toUpperCase()),
+            cols[i] = new Cell(cellname("col" + letters[i].toUpperCase()),
                     ByteBuffer.wrap(new byte[1]), 1);
         }
 
@@ -1362,10 +1360,10 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         cfs.clearUnsafe();
 
         String[] letters = new String[] { "a", "b", "c", "d", "i" };
-        Column[] cols = new Column[letters.length];
+        Cell[] cols = new Cell[letters.length];
         for (int i = 0; i < cols.length; i++)
         {
-            cols[i] = new Column(cellname("col" + letters[i].toUpperCase()),
+            cols[i] = new Cell(cellname("col" + letters[i].toUpperCase()),
                     ByteBuffer.wrap(new byte[1366]), 1);
         }
 
@@ -1411,10 +1409,10 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         cfs.clearUnsafe();
 
         String[] letters = new String[] { "a", "b", "c", "d", "e", "f", "g", "h", "i" };
-        Column[] cols = new Column[letters.length];
+        Cell[] cols = new Cell[letters.length];
         for (int i = 0; i < cols.length; i++)
         {
-            cols[i] = new Column(cellname("col" + letters[i].toUpperCase()),
+            cols[i] = new Cell(cellname("col" + letters[i].toUpperCase()),
                     ByteBuffer.wrap(new byte[1]), 1);
         }
 
@@ -1461,10 +1459,10 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         cfs.clearUnsafe();
 
         String[] letters = new String[] { "a", "b", "c", "d", "e", "f", "g", "h", "i" };
-        Column[] cols = new Column[letters.length];
+        Cell[] cols = new Cell[letters.length];
         for (int i = 0; i < cols.length; i++)
         {
-            cols[i] = new Column(cellname("col" + letters[i].toUpperCase()),
+            cols[i] = new Cell(cellname("col" + letters[i].toUpperCase()),
                     ByteBuffer.wrap(new byte[1366]), 1);
         }
 
@@ -1509,10 +1507,10 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         cfs.clearUnsafe();
 
         String[] letters = new String[] { "a", "b", "c", "d", "e", "f", "g", "h", "i" };
-        Column[] cols = new Column[letters.length];
+        Cell[] cols = new Cell[letters.length];
         for (int i = 0; i < cols.length; i++)
         {
-            cols[i] = new Column(cellname("col" + letters[i].toUpperCase()),
+            cols[i] = new Cell(cellname("col" + letters[i].toUpperCase()),
                     // use 1366 so that three cols make an index segment
                     ByteBuffer.wrap(new byte[1366]), 1);
         }
@@ -1628,10 +1626,10 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         cfs.clearUnsafe();
 
         String[] letters = new String[] { "a", "b", "c", "d", "e", "f", "g", "h", "i", "j", "k", "l" };
-        Column[] cols = new Column[12];
+        Cell[] cols = new Cell[12];
         for (int i = 0; i < cols.length; i++)
         {
-            cols[i] = new Column(cellname("col" + letters[i]), ByteBuffer.wrap(new byte[valueSize]), 1);
+            cols[i] = new Cell(cellname("col" + letters[i]), ByteBuffer.wrap(new byte[valueSize]), 1);
         }
 
         for (int i = 0; i < 12; i++)
@@ -1863,11 +1861,11 @@ public class ColumnFamilyStoreTest extends SchemaLoader
                                            false);
         assertSame("unexpected number of rows ", 1, rows.size());
         Row row = rows.get(0);
-        Collection<Column> cols = !filter.isReversed() ? row.cf.getSortedColumns() : row.cf.getReverseSortedColumns();
+        Collection<Cell> cols = !filter.isReversed() ? row.cf.getSortedColumns() : row.cf.getReverseSortedColumns();
         // printRow(cfs, new String(row.key.key.array()), cols);
-        String[] returnedColsNames = Iterables.toArray(Iterables.transform(cols, new Function<Column, String>()
+        String[] returnedColsNames = Iterables.toArray(Iterables.transform(cols, new Function<Cell, String>()
         {
-            public String apply(Column arg0)
+            public String apply(Cell arg0)
             {
                 return Util.string(arg0.name().toByteBuffer());
             }
@@ -1877,29 +1875,29 @@ public class ColumnFamilyStoreTest extends SchemaLoader
                 "Columns did not match. Expected: " + Arrays.toString(colNames) + " but got:"
                         + Arrays.toString(returnedColsNames), Arrays.equals(colNames, returnedColsNames));
         int i = 0;
-        for (Column col : cols)
+        for (Cell col : cols)
         {
             assertEquals(colNames[i++], Util.string(col.name().toByteBuffer()));
         }
     }
 
-    private void printRow(ColumnFamilyStore cfs, String rowKey, Collection<Column> cols)
+    private void printRow(ColumnFamilyStore cfs, String rowKey, Collection<Cell> cols)
     {
         DecoratedKey ROW = Util.dk(rowKey);
         System.err.println("Original:");
         ColumnFamily cf = cfs.getColumnFamily(QueryFilter.getIdentityFilter(ROW, "Standard1", System.currentTimeMillis()));
         System.err.println("Row key: " + rowKey + " Cols: "
-                + Iterables.transform(cf.getSortedColumns(), new Function<Column, String>()
+                + Iterables.transform(cf.getSortedColumns(), new Function<Cell, String>()
                 {
-                    public String apply(Column arg0)
+                    public String apply(Cell arg0)
                     {
                         return Util.string(arg0.name().toByteBuffer());
                     }
                 }));
         System.err.println("Filtered:");
-        Iterable<String> transformed = Iterables.transform(cols, new Function<Column, String>()
+        Iterable<String> transformed = Iterables.transform(cols, new Function<Cell, String>()
         {
-            public String apply(Column arg0)
+            public String apply(Cell arg0)
             {
                 return Util.string(arg0.name().toByteBuffer());
             }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/test/unit/org/apache/cassandra/db/CounterCellTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/CounterCellTest.java b/test/unit/org/apache/cassandra/db/CounterCellTest.java
new file mode 100644
index 0000000..ca70cf8
--- /dev/null
+++ b/test/unit/org/apache/cassandra/db/CounterCellTest.java
@@ -0,0 +1,323 @@
+/*
+* 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.security.MessageDigest;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.Util;
+import org.apache.cassandra.db.composites.*;
+import org.apache.cassandra.db.context.CounterContext;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import static org.apache.cassandra.db.context.CounterContext.ContextState;
+import org.apache.cassandra.io.util.DataOutputBuffer;
+import org.apache.cassandra.utils.*;
+
+public class CounterCellTest extends SchemaLoader
+{
+    private static final CounterContext cc = new CounterContext();
+
+    private static final int idLength;
+    private static final int clockLength;
+    private static final int countLength;
+
+    private static final int stepLength;
+
+    static
+    {
+        idLength      = CounterId.LENGTH;
+        clockLength   = 8; // size of long
+        countLength   = 8; // size of long
+
+        stepLength    = idLength + clockLength + countLength;
+    }
+
+    @Test
+    public void testCreate()
+    {
+        long delta = 3L;
+        CounterUpdateCell cuc = new CounterUpdateCell(Util.cellname("x"), delta, 1L);
+        CounterCell column = cuc.localCopy(Keyspace.open("Keyspace5").getColumnFamilyStore("Counter1"));
+
+        Assert.assertEquals(delta, column.total());
+        Assert.assertEquals(1, column.value().getShort(0));
+        Assert.assertEquals(0, column.value().getShort(2));
+        Assert.assertTrue(CounterId.wrap(column.value(), 4).isLocalId());
+        Assert.assertEquals(1L, column.value().getLong(4 + 0*stepLength + idLength));
+        Assert.assertEquals(delta, column.value().getLong(4 + 0*stepLength + idLength + clockLength));
+    }
+
+    @Test
+    public void testReconcile()
+    {
+        Cell left;
+        Cell right;
+        Cell reconciled;
+
+        ByteBuffer context;
+
+        // tombstone + tombstone
+        left  = new DeletedCell(Util.cellname("x"), 1, 1L);
+        right = new DeletedCell(Util.cellname("x"), 2, 2L);
+
+        Assert.assertEquals(left.reconcile(right).getMarkedForDeleteAt(), right.getMarkedForDeleteAt());
+        Assert.assertEquals(right.reconcile(left).getMarkedForDeleteAt(), right.getMarkedForDeleteAt());
+
+        // tombstone > live
+        left  = new DeletedCell(Util.cellname("x"), 1, 2L);
+        right = new CounterCell(Util.cellname("x"), 0L, 1L);
+
+        Assert.assertEquals(left.reconcile(right), left);
+
+        // tombstone < live last delete
+        left  = new DeletedCell(Util.cellname("x"), 1, 1L);
+        right = new CounterCell(Util.cellname("x"), 0L, 4L, 2L);
+
+        Assert.assertEquals(left.reconcile(right), right);
+
+        // tombstone == live last delete
+        left  = new DeletedCell(Util.cellname("x"), 1, 2L);
+        right = new CounterCell(Util.cellname("x"), 0L, 4L, 2L);
+
+        Assert.assertEquals(left.reconcile(right), right);
+
+        // tombstone > live last delete
+        left  = new DeletedCell(Util.cellname("x"), 1, 4L);
+        right = new CounterCell(Util.cellname("x"), 0L, 9L, 1L);
+
+        reconciled = left.reconcile(right);
+        Assert.assertEquals(reconciled.name(), right.name());
+        Assert.assertEquals(reconciled.value(), right.value());
+        Assert.assertEquals(reconciled.timestamp(), right.timestamp());
+        Assert.assertEquals(((CounterCell)reconciled).timestampOfLastDelete(), left.getMarkedForDeleteAt());
+
+        // live < tombstone
+        left  = new CounterCell(Util.cellname("x"), 0L, 1L);
+        right = new DeletedCell(Util.cellname("x"), 1, 2L);
+
+        Assert.assertEquals(left.reconcile(right), right);
+
+        // live last delete > tombstone
+        left  = new CounterCell(Util.cellname("x"), 0L, 4L, 2L);
+        right = new DeletedCell(Util.cellname("x"), 1, 1L);
+
+        Assert.assertEquals(left.reconcile(right), left);
+
+        // live last delete == tombstone
+        left  = new CounterCell(Util.cellname("x"), 0L, 4L, 2L);
+        right = new DeletedCell(Util.cellname("x"), 1, 2L);
+
+        Assert.assertEquals(left.reconcile(right), left);
+
+        // live last delete < tombstone
+        left  = new CounterCell(Util.cellname("x"), 0L, 9L, 1L);
+        right = new DeletedCell(Util.cellname("x"), 1, 4L);
+
+        reconciled = left.reconcile(right);
+        Assert.assertEquals(reconciled.name(), left.name());
+        Assert.assertEquals(reconciled.value(), left.value());
+        Assert.assertEquals(reconciled.timestamp(), left.timestamp());
+        Assert.assertEquals(((CounterCell)reconciled).timestampOfLastDelete(), right.getMarkedForDeleteAt());
+
+        // live < live last delete
+        left  = new CounterCell(Util.cellname("x"), cc.create(CounterId.fromInt(1), 2L, 3L, false), 1L, Long.MIN_VALUE);
+        right = new CounterCell(Util.cellname("x"), cc.create(CounterId.fromInt(1), 1L, 1L, false), 4L, 3L);
+
+        Assert.assertEquals(left.reconcile(right), right);
+
+        // live last delete > live
+        left  = new CounterCell(Util.cellname("x"), cc.create(CounterId.fromInt(1), 2L, 3L, false), 6L, 5L);
+        right = new CounterCell(Util.cellname("x"), cc.create(CounterId.fromInt(1), 1L, 1L, false), 4L, 3L);
+
+        Assert.assertEquals(left.reconcile(right), left);
+
+        // live + live
+        left = new CounterCell(Util.cellname("x"), cc.create(CounterId.fromInt(1), 1L, 1L, false), 4L, Long.MIN_VALUE);
+        right = new CounterCell(Util.cellname("x"), cc.create(CounterId.fromInt(1), 2L, 3L, false), 1L, Long.MIN_VALUE);
+
+        reconciled = left.reconcile(right);
+        Assert.assertEquals(reconciled.name(), left.name());
+        Assert.assertEquals(3L, ((CounterCell)reconciled).total());
+        Assert.assertEquals(4L, reconciled.timestamp());
+
+        left = reconciled;
+        right = new CounterCell(Util.cellname("x"), cc.create(CounterId.fromInt(2), 1L, 5L, false), 2L, Long.MIN_VALUE);
+
+        reconciled = left.reconcile(right);
+        Assert.assertEquals(reconciled.name(), left.name());
+        Assert.assertEquals(8L, ((CounterCell)reconciled).total());
+        Assert.assertEquals(4L, reconciled.timestamp());
+
+        left = reconciled;
+        right = new CounterCell(Util.cellname("x"), cc.create(CounterId.fromInt(2), 2L, 2L, false), 6L, Long.MIN_VALUE);
+
+        reconciled = left.reconcile(right);
+        Assert.assertEquals(reconciled.name(), left.name());
+        Assert.assertEquals(5L, ((CounterCell)reconciled).total());
+        Assert.assertEquals(6L, reconciled.timestamp());
+
+        context = reconciled.value();
+        int hd = 2; // header
+        Assert.assertEquals(hd + 2 * stepLength, context.remaining());
+
+        Assert.assertTrue(Util.equalsCounterId(CounterId.fromInt(1), context, hd + 0 * stepLength));
+        Assert.assertEquals(2L, context.getLong(hd + 0*stepLength + idLength));
+        Assert.assertEquals(3L, context.getLong(hd + 0*stepLength + idLength + clockLength));
+
+        Assert.assertTrue(Util.equalsCounterId(CounterId.fromInt(2), context, hd + 1 * stepLength));
+        Assert.assertEquals(2L, context.getLong(hd + 1*stepLength + idLength));
+        Assert.assertEquals(2L, context.getLong(hd + 1*stepLength + idLength + clockLength));
+
+        Assert.assertEquals(Long.MIN_VALUE, ((CounterCell)reconciled).timestampOfLastDelete());
+    }
+
+    @Test
+    public void testDiff()
+    {
+        Allocator allocator = HeapAllocator.instance;
+        ContextState left;
+        ContextState right;
+
+        CounterCell leftCol;
+        CounterCell rightCol;
+
+        // timestamp
+        leftCol = new CounterCell(Util.cellname("x"), 0, 1L);
+        rightCol = new CounterCell(Util.cellname("x"), 0, 2L);
+
+        Assert.assertEquals(rightCol, leftCol.diff(rightCol));
+        Assert.assertNull(rightCol.diff(leftCol));
+
+        // timestampOfLastDelete
+        leftCol = new CounterCell(Util.cellname("x"), 0, 1L, 1L);
+        rightCol = new CounterCell(Util.cellname("x"), 0, 1L, 2L);
+
+        Assert.assertEquals(rightCol, leftCol.diff(rightCol));
+        Assert.assertNull(rightCol.diff(leftCol));
+
+        // equality: equal nodes, all counts same
+        left = ContextState.allocate(3, 0, allocator);
+        left.writeElement(CounterId.fromInt(3), 3L, 0L);
+        left.writeElement(CounterId.fromInt(6), 2L, 0L);
+        left.writeElement(CounterId.fromInt(9), 1L, 0L);
+        right = new ContextState(ByteBufferUtil.clone(left.context), 2);
+
+        leftCol  = new CounterCell(Util.cellname("x"), left.context,  1L);
+        rightCol = new CounterCell(Util.cellname("x"), right.context, 1L);
+        Assert.assertNull(leftCol.diff(rightCol));
+
+        // greater than: left has superset of nodes (counts equal)
+        left = ContextState.allocate(4, 0, allocator);
+        left.writeElement(CounterId.fromInt(3), 3L, 0L);
+        left.writeElement(CounterId.fromInt(6), 2L, 0L);
+        left.writeElement(CounterId.fromInt(9), 1L, 0L);
+        left.writeElement(CounterId.fromInt(12), 0L, 0L);
+
+        right = ContextState.allocate(3, 0, allocator);
+        right.writeElement(CounterId.fromInt(3), 3L, 0L);
+        right.writeElement(CounterId.fromInt(6), 2L, 0L);
+        right.writeElement(CounterId.fromInt(9), 1L, 0L);
+
+        leftCol  = new CounterCell(Util.cellname("x"), left.context,  1L);
+        rightCol = new CounterCell(Util.cellname("x"), right.context, 1L);
+        Assert.assertNull(leftCol.diff(rightCol));
+
+        // less than: right has subset of nodes (counts equal)
+        Assert.assertEquals(leftCol, rightCol.diff(leftCol));
+
+        // disjoint: right and left have disjoint node sets
+        left = ContextState.allocate(3, 0, allocator);
+        left.writeElement(CounterId.fromInt(3), 1L, 0L);
+        left.writeElement(CounterId.fromInt(4), 1L, 0L);
+        left.writeElement(CounterId.fromInt(9), 1L, 0L);
+
+        right = ContextState.allocate(3, 0, allocator);
+        right.writeElement(CounterId.fromInt(3), 1L, 0L);
+        right.writeElement(CounterId.fromInt(6), 1L, 0L);
+        right.writeElement(CounterId.fromInt(9), 1L, 0L);
+
+        leftCol  = new CounterCell(Util.cellname("x"), left.context,  1L);
+        rightCol = new CounterCell(Util.cellname("x"), right.context, 1L);
+        Assert.assertEquals(rightCol, leftCol.diff(rightCol));
+        Assert.assertEquals(leftCol, rightCol.diff(leftCol));
+    }
+
+    @Test
+    public void testSerializeDeserialize() throws IOException
+    {
+        Allocator allocator = HeapAllocator.instance;
+        CounterContext.ContextState state = CounterContext.ContextState.allocate(4, 2, allocator);
+        state.writeElement(CounterId.fromInt(1), 4L, 4L);
+        state.writeElement(CounterId.fromInt(2), 4L, 4L, true);
+        state.writeElement(CounterId.fromInt(3), 4L, 4L);
+        state.writeElement(CounterId.fromInt(4), 4L, 4L, true);
+
+        CellNameType type = new SimpleDenseCellNameType(UTF8Type.instance);
+        CounterCell original = new CounterCell(Util.cellname("x"), state.context, 1L);
+        byte[] serialized;
+        try (DataOutputBuffer bufOut = new DataOutputBuffer())
+        {
+            type.columnSerializer().serialize(original, bufOut);
+            serialized = bufOut.getData();
+        }
+
+        ByteArrayInputStream bufIn = new ByteArrayInputStream(serialized, 0, serialized.length);
+        CounterCell deserialized = (CounterCell) type.columnSerializer().deserialize(new DataInputStream(bufIn));
+        Assert.assertEquals(original, deserialized);
+
+        bufIn = new ByteArrayInputStream(serialized, 0, serialized.length);
+        CounterCell deserializedOnRemote = (CounterCell) type.columnSerializer().deserialize(new DataInputStream(bufIn), ColumnSerializer.Flag.FROM_REMOTE);
+        Assert.assertEquals(deserializedOnRemote.name(), original.name());
+        Assert.assertEquals(deserializedOnRemote.total(), original.total());
+        Assert.assertEquals(deserializedOnRemote.value(), cc.clearAllDelta(original.value()));
+        Assert.assertEquals(deserializedOnRemote.timestamp(), deserialized.timestamp());
+        Assert.assertEquals(deserializedOnRemote.timestampOfLastDelete(), deserialized.timestampOfLastDelete());
+    }
+
+    @Test
+    public void testUpdateDigest() throws Exception
+    {
+        Allocator allocator = HeapAllocator.instance;
+        MessageDigest digest1 = MessageDigest.getInstance("md5");
+        MessageDigest digest2 = MessageDigest.getInstance("md5");
+
+        CounterContext.ContextState state = CounterContext.ContextState.allocate(4, 2, allocator);
+        state.writeElement(CounterId.fromInt(1), 4L, 4L);
+        state.writeElement(CounterId.fromInt(2), 4L, 4L, true);
+        state.writeElement(CounterId.fromInt(3), 4L, 4L);
+        state.writeElement(CounterId.fromInt(4), 4L, 4L, true);
+
+        CounterCell original = new CounterCell(Util.cellname("x"), state.context, 1L);
+        CounterCell cleared = new CounterCell(Util.cellname("x"), cc.clearAllDelta(state.context), 1L);
+
+        original.updateDigest(digest1);
+        cleared.updateDigest(digest2);
+
+        Assert.assertTrue(Arrays.equals(digest1.digest(), digest2.digest()));
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/test/unit/org/apache/cassandra/db/CounterColumnTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/CounterColumnTest.java b/test/unit/org/apache/cassandra/db/CounterColumnTest.java
deleted file mode 100644
index 0ca2636..0000000
--- a/test/unit/org/apache/cassandra/db/CounterColumnTest.java
+++ /dev/null
@@ -1,323 +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.security.MessageDigest;
-
-import java.io.ByteArrayInputStream;
-import java.io.DataInputStream;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.Arrays;
-
-import org.junit.Assert;
-import org.junit.Test;
-
-import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.Util;
-import org.apache.cassandra.db.composites.*;
-import org.apache.cassandra.db.context.CounterContext;
-import org.apache.cassandra.db.marshal.UTF8Type;
-import static org.apache.cassandra.db.context.CounterContext.ContextState;
-import org.apache.cassandra.io.util.DataOutputBuffer;
-import org.apache.cassandra.utils.*;
-
-public class CounterColumnTest extends SchemaLoader
-{
-    private static final CounterContext cc = new CounterContext();
-
-    private static final int idLength;
-    private static final int clockLength;
-    private static final int countLength;
-
-    private static final int stepLength;
-
-    static
-    {
-        idLength      = CounterId.LENGTH;
-        clockLength   = 8; // size of long
-        countLength   = 8; // size of long
-
-        stepLength    = idLength + clockLength + countLength;
-    }
-
-    @Test
-    public void testCreate()
-    {
-        long delta = 3L;
-        CounterUpdateColumn cuc = new CounterUpdateColumn(Util.cellname("x"), delta, 1L);
-        CounterColumn column = cuc.localCopy(Keyspace.open("Keyspace5").getColumnFamilyStore("Counter1"));
-
-        Assert.assertEquals(delta, column.total());
-        Assert.assertEquals(1, column.value().getShort(0));
-        Assert.assertEquals(0, column.value().getShort(2));
-        Assert.assertTrue(CounterId.wrap(column.value(), 4).isLocalId());
-        Assert.assertEquals(1L, column.value().getLong(4 + 0*stepLength + idLength));
-        Assert.assertEquals(delta, column.value().getLong(4 + 0*stepLength + idLength + clockLength));
-    }
-
-    @Test
-    public void testReconcile()
-    {
-        Column left;
-        Column right;
-        Column reconciled;
-
-        ByteBuffer context;
-
-        // tombstone + tombstone
-        left  = new DeletedColumn(Util.cellname("x"), 1, 1L);
-        right = new DeletedColumn(Util.cellname("x"), 2, 2L);
-
-        Assert.assertEquals(left.reconcile(right).getMarkedForDeleteAt(), right.getMarkedForDeleteAt());
-        Assert.assertEquals(right.reconcile(left).getMarkedForDeleteAt(), right.getMarkedForDeleteAt());
-
-        // tombstone > live
-        left  = new DeletedColumn(Util.cellname("x"), 1, 2L);
-        right = new CounterColumn(Util.cellname("x"), 0L, 1L);
-
-        Assert.assertEquals(left.reconcile(right), left);
-
-        // tombstone < live last delete
-        left  = new DeletedColumn(Util.cellname("x"), 1, 1L);
-        right = new CounterColumn(Util.cellname("x"), 0L, 4L, 2L);
-
-        Assert.assertEquals(left.reconcile(right), right);
-
-        // tombstone == live last delete
-        left  = new DeletedColumn(Util.cellname("x"), 1, 2L);
-        right = new CounterColumn(Util.cellname("x"), 0L, 4L, 2L);
-
-        Assert.assertEquals(left.reconcile(right), right);
-
-        // tombstone > live last delete
-        left  = new DeletedColumn(Util.cellname("x"), 1, 4L);
-        right = new CounterColumn(Util.cellname("x"), 0L, 9L, 1L);
-
-        reconciled = left.reconcile(right);
-        Assert.assertEquals(reconciled.name(), right.name());
-        Assert.assertEquals(reconciled.value(), right.value());
-        Assert.assertEquals(reconciled.timestamp(), right.timestamp());
-        Assert.assertEquals(((CounterColumn)reconciled).timestampOfLastDelete(), left.getMarkedForDeleteAt());
-
-        // live < tombstone
-        left  = new CounterColumn(Util.cellname("x"), 0L, 1L);
-        right = new DeletedColumn(Util.cellname("x"), 1, 2L);
-
-        Assert.assertEquals(left.reconcile(right), right);
-
-        // live last delete > tombstone
-        left  = new CounterColumn(Util.cellname("x"), 0L, 4L, 2L);
-        right = new DeletedColumn(Util.cellname("x"), 1, 1L);
-
-        Assert.assertEquals(left.reconcile(right), left);
-
-        // live last delete == tombstone
-        left  = new CounterColumn(Util.cellname("x"), 0L, 4L, 2L);
-        right = new DeletedColumn(Util.cellname("x"), 1, 2L);
-
-        Assert.assertEquals(left.reconcile(right), left);
-
-        // live last delete < tombstone
-        left  = new CounterColumn(Util.cellname("x"), 0L, 9L, 1L);
-        right = new DeletedColumn(Util.cellname("x"), 1, 4L);
-
-        reconciled = left.reconcile(right);
-        Assert.assertEquals(reconciled.name(), left.name());
-        Assert.assertEquals(reconciled.value(), left.value());
-        Assert.assertEquals(reconciled.timestamp(), left.timestamp());
-        Assert.assertEquals(((CounterColumn)reconciled).timestampOfLastDelete(), right.getMarkedForDeleteAt());
-
-        // live < live last delete
-        left  = new CounterColumn(Util.cellname("x"), cc.create(CounterId.fromInt(1), 2L, 3L, false), 1L, Long.MIN_VALUE);
-        right = new CounterColumn(Util.cellname("x"), cc.create(CounterId.fromInt(1), 1L, 1L, false), 4L, 3L);
-
-        Assert.assertEquals(left.reconcile(right), right);
-
-        // live last delete > live
-        left  = new CounterColumn(Util.cellname("x"), cc.create(CounterId.fromInt(1), 2L, 3L, false), 6L, 5L);
-        right = new CounterColumn(Util.cellname("x"), cc.create(CounterId.fromInt(1), 1L, 1L, false), 4L, 3L);
-
-        Assert.assertEquals(left.reconcile(right), left);
-
-        // live + live
-        left = new CounterColumn(Util.cellname("x"), cc.create(CounterId.fromInt(1), 1L, 1L, false), 4L, Long.MIN_VALUE);
-        right = new CounterColumn(Util.cellname("x"), cc.create(CounterId.fromInt(1), 2L, 3L, false), 1L, Long.MIN_VALUE);
-
-        reconciled = left.reconcile(right);
-        Assert.assertEquals(reconciled.name(), left.name());
-        Assert.assertEquals(3L, ((CounterColumn)reconciled).total());
-        Assert.assertEquals(4L, reconciled.timestamp());
-
-        left = reconciled;
-        right = new CounterColumn(Util.cellname("x"), cc.create(CounterId.fromInt(2), 1L, 5L, false), 2L, Long.MIN_VALUE);
-
-        reconciled = left.reconcile(right);
-        Assert.assertEquals(reconciled.name(), left.name());
-        Assert.assertEquals(8L, ((CounterColumn)reconciled).total());
-        Assert.assertEquals(4L, reconciled.timestamp());
-
-        left = reconciled;
-        right = new CounterColumn(Util.cellname("x"), cc.create(CounterId.fromInt(2), 2L, 2L, false), 6L, Long.MIN_VALUE);
-
-        reconciled = left.reconcile(right);
-        Assert.assertEquals(reconciled.name(), left.name());
-        Assert.assertEquals(5L, ((CounterColumn)reconciled).total());
-        Assert.assertEquals(6L, reconciled.timestamp());
-
-        context = reconciled.value();
-        int hd = 2; // header
-        Assert.assertEquals(hd + 2 * stepLength, context.remaining());
-
-        Assert.assertTrue(Util.equalsCounterId(CounterId.fromInt(1), context, hd + 0 * stepLength));
-        Assert.assertEquals(2L, context.getLong(hd + 0*stepLength + idLength));
-        Assert.assertEquals(3L, context.getLong(hd + 0*stepLength + idLength + clockLength));
-
-        Assert.assertTrue(Util.equalsCounterId(CounterId.fromInt(2), context, hd + 1 * stepLength));
-        Assert.assertEquals(2L, context.getLong(hd + 1*stepLength + idLength));
-        Assert.assertEquals(2L, context.getLong(hd + 1*stepLength + idLength + clockLength));
-
-        Assert.assertEquals(Long.MIN_VALUE, ((CounterColumn)reconciled).timestampOfLastDelete());
-    }
-
-    @Test
-    public void testDiff()
-    {
-        Allocator allocator = HeapAllocator.instance;
-        ContextState left;
-        ContextState right;
-
-        CounterColumn leftCol;
-        CounterColumn rightCol;
-
-        // timestamp
-        leftCol = new CounterColumn(Util.cellname("x"), 0, 1L);
-        rightCol = new CounterColumn(Util.cellname("x"), 0, 2L);
-
-        Assert.assertEquals(rightCol, leftCol.diff(rightCol));
-        Assert.assertNull(rightCol.diff(leftCol));
-
-        // timestampOfLastDelete
-        leftCol = new CounterColumn(Util.cellname("x"), 0, 1L, 1L);
-        rightCol = new CounterColumn(Util.cellname("x"), 0, 1L, 2L);
-
-        Assert.assertEquals(rightCol, leftCol.diff(rightCol));
-        Assert.assertNull(rightCol.diff(leftCol));
-
-        // equality: equal nodes, all counts same
-        left = ContextState.allocate(3, 0, allocator);
-        left.writeElement(CounterId.fromInt(3), 3L, 0L);
-        left.writeElement(CounterId.fromInt(6), 2L, 0L);
-        left.writeElement(CounterId.fromInt(9), 1L, 0L);
-        right = new ContextState(ByteBufferUtil.clone(left.context), 2);
-
-        leftCol  = new CounterColumn(Util.cellname("x"), left.context,  1L);
-        rightCol = new CounterColumn(Util.cellname("x"), right.context, 1L);
-        Assert.assertNull(leftCol.diff(rightCol));
-
-        // greater than: left has superset of nodes (counts equal)
-        left = ContextState.allocate(4, 0, allocator);
-        left.writeElement(CounterId.fromInt(3), 3L, 0L);
-        left.writeElement(CounterId.fromInt(6), 2L, 0L);
-        left.writeElement(CounterId.fromInt(9), 1L, 0L);
-        left.writeElement(CounterId.fromInt(12), 0L, 0L);
-
-        right = ContextState.allocate(3, 0, allocator);
-        right.writeElement(CounterId.fromInt(3), 3L, 0L);
-        right.writeElement(CounterId.fromInt(6), 2L, 0L);
-        right.writeElement(CounterId.fromInt(9), 1L, 0L);
-
-        leftCol  = new CounterColumn(Util.cellname("x"), left.context,  1L);
-        rightCol = new CounterColumn(Util.cellname("x"), right.context, 1L);
-        Assert.assertNull(leftCol.diff(rightCol));
-
-        // less than: right has subset of nodes (counts equal)
-        Assert.assertEquals(leftCol, rightCol.diff(leftCol));
-
-        // disjoint: right and left have disjoint node sets
-        left = ContextState.allocate(3, 0, allocator);
-        left.writeElement(CounterId.fromInt(3), 1L, 0L);
-        left.writeElement(CounterId.fromInt(4), 1L, 0L);
-        left.writeElement(CounterId.fromInt(9), 1L, 0L);
-
-        right = ContextState.allocate(3, 0, allocator);
-        right.writeElement(CounterId.fromInt(3), 1L, 0L);
-        right.writeElement(CounterId.fromInt(6), 1L, 0L);
-        right.writeElement(CounterId.fromInt(9), 1L, 0L);
-
-        leftCol  = new CounterColumn(Util.cellname("x"), left.context,  1L);
-        rightCol = new CounterColumn(Util.cellname("x"), right.context, 1L);
-        Assert.assertEquals(rightCol, leftCol.diff(rightCol));
-        Assert.assertEquals(leftCol, rightCol.diff(leftCol));
-    }
-
-    @Test
-    public void testSerializeDeserialize() throws IOException
-    {
-        Allocator allocator = HeapAllocator.instance;
-        CounterContext.ContextState state = CounterContext.ContextState.allocate(4, 2, allocator);
-        state.writeElement(CounterId.fromInt(1), 4L, 4L);
-        state.writeElement(CounterId.fromInt(2), 4L, 4L, true);
-        state.writeElement(CounterId.fromInt(3), 4L, 4L);
-        state.writeElement(CounterId.fromInt(4), 4L, 4L, true);
-
-        CellNameType type = new SimpleDenseCellNameType(UTF8Type.instance);
-        CounterColumn original = new CounterColumn(Util.cellname("x"), state.context, 1L);
-        byte[] serialized;
-        try (DataOutputBuffer bufOut = new DataOutputBuffer())
-        {
-            type.columnSerializer().serialize(original, bufOut);
-            serialized = bufOut.getData();
-        }
-
-        ByteArrayInputStream bufIn = new ByteArrayInputStream(serialized, 0, serialized.length);
-        CounterColumn deserialized = (CounterColumn) type.columnSerializer().deserialize(new DataInputStream(bufIn));
-        Assert.assertEquals(original, deserialized);
-
-        bufIn = new ByteArrayInputStream(serialized, 0, serialized.length);
-        CounterColumn deserializedOnRemote = (CounterColumn) type.columnSerializer().deserialize(new DataInputStream(bufIn), ColumnSerializer.Flag.FROM_REMOTE);
-        Assert.assertEquals(deserializedOnRemote.name(), original.name());
-        Assert.assertEquals(deserializedOnRemote.total(), original.total());
-        Assert.assertEquals(deserializedOnRemote.value(), cc.clearAllDelta(original.value()));
-        Assert.assertEquals(deserializedOnRemote.timestamp(), deserialized.timestamp());
-        Assert.assertEquals(deserializedOnRemote.timestampOfLastDelete(), deserialized.timestampOfLastDelete());
-    }
-
-    @Test
-    public void testUpdateDigest() throws Exception
-    {
-        Allocator allocator = HeapAllocator.instance;
-        MessageDigest digest1 = MessageDigest.getInstance("md5");
-        MessageDigest digest2 = MessageDigest.getInstance("md5");
-
-        CounterContext.ContextState state = CounterContext.ContextState.allocate(4, 2, allocator);
-        state.writeElement(CounterId.fromInt(1), 4L, 4L);
-        state.writeElement(CounterId.fromInt(2), 4L, 4L, true);
-        state.writeElement(CounterId.fromInt(3), 4L, 4L);
-        state.writeElement(CounterId.fromInt(4), 4L, 4L, true);
-
-        CounterColumn original = new CounterColumn(Util.cellname("x"), state.context, 1L);
-        CounterColumn cleared = new CounterColumn(Util.cellname("x"), cc.clearAllDelta(state.context), 1L);
-
-        original.updateDigest(digest1);
-        cleared.updateDigest(digest2);
-
-        Assert.assertTrue(Arrays.equals(digest1.digest(), digest2.digest()));
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/test/unit/org/apache/cassandra/db/CounterMutationTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/CounterMutationTest.java b/test/unit/org/apache/cassandra/db/CounterMutationTest.java
index 74deb37..5cc97e4 100644
--- a/test/unit/org/apache/cassandra/db/CounterMutationTest.java
+++ b/test/unit/org/apache/cassandra/db/CounterMutationTest.java
@@ -67,12 +67,12 @@ public class CounterMutationTest extends SchemaLoader
         ColumnFamily cf = Util.getColumnFamily(Keyspace.open("Keyspace1"), dk, "Counter1");
 
         // First merges old shards
-        CounterColumn.mergeAndRemoveOldShards(dk, cf, Integer.MIN_VALUE, Integer.MAX_VALUE, false);
+        CounterCell.mergeAndRemoveOldShards(dk, cf, Integer.MIN_VALUE, Integer.MAX_VALUE, false);
         long now = System.currentTimeMillis();
-        Column c = cf.getColumn(Util.cellname("Column1"));
+        Cell c = cf.getColumn(Util.cellname("Column1"));
         assert c != null;
-        assert c instanceof CounterColumn;
-        assert ((CounterColumn)c).total() == 12L;
+        assert c instanceof CounterCell;
+        assert ((CounterCell)c).total() == 12L;
         ContextState s = new ContextState(c.value());
         assert s.getCounterId().equals(id1);
         assert s.getCount() == 0L;
@@ -88,11 +88,11 @@ public class CounterMutationTest extends SchemaLoader
         assert s.getCount() == 12L;
 
         // Then collect old shards
-        CounterColumn.mergeAndRemoveOldShards(dk, cf, Integer.MAX_VALUE, Integer.MIN_VALUE, false);
+        CounterCell.mergeAndRemoveOldShards(dk, cf, Integer.MAX_VALUE, Integer.MIN_VALUE, false);
         c = cf.getColumn(Util.cellname("Column1"));
         assert c != null;
-        assert c instanceof CounterColumn;
-        assert ((CounterColumn)c).total() == 12L;
+        assert c instanceof CounterCell;
+        assert ((CounterCell)c).total() == 12L;
         s = new ContextState(c.value());
         assert s.getCounterId().equals(id3);
         assert s.getCount() == 12L;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/test/unit/org/apache/cassandra/db/KeyspaceTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/KeyspaceTest.java b/test/unit/org/apache/cassandra/db/KeyspaceTest.java
index 1c392c9..fa2ddaf 100644
--- a/test/unit/org/apache/cassandra/db/KeyspaceTest.java
+++ b/test/unit/org/apache/cassandra/db/KeyspaceTest.java
@@ -35,9 +35,6 @@ import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.composites.*;
 import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.db.filter.QueryFilter;
-import org.apache.cassandra.db.marshal.BytesType;
-import org.apache.cassandra.db.marshal.CompositeType;
-import org.apache.cassandra.db.marshal.IntegerType;
 import org.apache.cassandra.utils.WrappedRunnable;
 import static org.apache.cassandra.Util.column;
 import static org.apache.cassandra.Util.expiringColumn;
@@ -230,7 +227,7 @@ public class KeyspaceTest extends SchemaLoader
         for (int i = 0; i < 10; i++)
         {
             ColumnFamily cf = TreeMapBackedSortedColumns.factory.create("Keyspace1", "StandardLong1");
-            cf.addColumn(new Column(cellname((long)i), ByteBufferUtil.EMPTY_BYTE_BUFFER, 0));
+            cf.addColumn(new Cell(cellname((long)i), ByteBufferUtil.EMPTY_BYTE_BUFFER, 0));
             RowMutation rm = new RowMutation("Keyspace1", ROW.key, cf);
             rm.apply();
         }
@@ -240,7 +237,7 @@ public class KeyspaceTest extends SchemaLoader
         for (int i = 10; i < 20; i++)
         {
             ColumnFamily cf = TreeMapBackedSortedColumns.factory.create("Keyspace1", "StandardLong1");
-            cf.addColumn(new Column(cellname((long)i), ByteBufferUtil.EMPTY_BYTE_BUFFER, 0));
+            cf.addColumn(new Cell(cellname((long)i), ByteBufferUtil.EMPTY_BYTE_BUFFER, 0));
             RowMutation rm = new RowMutation("Keyspace1", ROW.key, cf);
             rm.apply();
 
@@ -453,7 +450,7 @@ public class KeyspaceTest extends SchemaLoader
         ColumnFamily cf = cfStore.getColumnFamily(key, Composites.EMPTY, cellname("col1499"), false, 1000, System.currentTimeMillis());
         assertEquals(cfStore.metric.sstablesPerReadHistogram.max(), 5, 0.1);
         int i = 0;
-        for (Column c : cf.getSortedColumns())
+        for (Cell c : cf.getSortedColumns())
         {
             assertEquals(ByteBufferUtil.string(c.name.toByteBuffer()), "col" + (1000 + i++));
         }
@@ -462,7 +459,7 @@ public class KeyspaceTest extends SchemaLoader
         cf = cfStore.getColumnFamily(key, cellname("col1500"), cellname("col2000"), false, 1000, System.currentTimeMillis());
         assertEquals(cfStore.metric.sstablesPerReadHistogram.max(), 5, 0.1);
 
-        for (Column c : cf.getSortedColumns())
+        for (Cell c : cf.getSortedColumns())
         {
             assertEquals(ByteBufferUtil.string(c.name.toByteBuffer()), "col"+(1000 + i++));
         }
@@ -473,7 +470,7 @@ public class KeyspaceTest extends SchemaLoader
         cf = cfStore.getColumnFamily(key, cellname("col2000"), cellname("col1500"), true, 1000, System.currentTimeMillis());
         assertEquals(cfStore.metric.sstablesPerReadHistogram.max(), 5, 0.1);
         i = 500;
-        for (Column c : cf.getSortedColumns())
+        for (Cell c : cf.getSortedColumns())
         {
             assertEquals(ByteBufferUtil.string(c.name.toByteBuffer()), "col"+(1000 + i++));
         }
@@ -521,7 +518,7 @@ public class KeyspaceTest extends SchemaLoader
         cfs.metric.sstablesPerReadHistogram.clear();
         ColumnFamily cf = cfs.getColumnFamily(key, start, finish, false, 1000, System.currentTimeMillis());
         int colCount = 0;
-        for (Column c : cf)
+        for (Cell c : cf)
             colCount++;
         assertEquals(2, colCount);
         assertEquals(2, cfs.metric.sstablesPerReadHistogram.max(), 0.1);
@@ -554,13 +551,13 @@ public class KeyspaceTest extends SchemaLoader
 
 
         cf = cfStore.getColumnFamily(key, cellname("col1996"), Composites.EMPTY, true, 1000, System.currentTimeMillis());
-        Column[] columns = cf.getSortedColumns().toArray(new Column[0]);
+        Cell[] cells = cf.getSortedColumns().toArray(new Cell[0]);
         for (int i = 1000; i < 1996; i++)
         {
             String expectedName = "col" + i;
-            Column column = columns[i - 1000];
-            assertEquals(ByteBufferUtil.string(column.name().toByteBuffer()), expectedName);
-            assertEquals(ByteBufferUtil.string(column.value()), ("v" + i));
+            Cell cell = cells[i - 1000];
+            assertEquals(ByteBufferUtil.string(cell.name().toByteBuffer()), expectedName);
+            assertEquals(ByteBufferUtil.string(cell.value()), ("v" + i));
         }
 
         cf = cfStore.getColumnFamily(key, cellname("col1990"), Composites.EMPTY, false, 3, System.currentTimeMillis());
@@ -590,11 +587,11 @@ public class KeyspaceTest extends SchemaLoader
 
     public static void assertColumns(ColumnFamily container, String... columnNames)
     {
-        Collection<Column> columns = container == null ? new TreeSet<Column>() : container.getSortedColumns();
+        Collection<Cell> cells = container == null ? new TreeSet<Cell>() : container.getSortedColumns();
         List<String> L = new ArrayList<String>();
-        for (Column column : columns)
+        for (Cell cell : cells)
         {
-            L.add(Util.string(column.name().toByteBuffer()));
+            L.add(Util.string(cell.name().toByteBuffer()));
         }
 
         List<String> names = new ArrayList<String>(columnNames.length);
@@ -602,11 +599,11 @@ public class KeyspaceTest extends SchemaLoader
         names.addAll(Arrays.asList(columnNames));
 
         String[] columnNames1 = names.toArray(new String[0]);
-        String[] la = L.toArray(new String[columns.size()]);
+        String[] la = L.toArray(new String[cells.size()]);
 
         assert Arrays.equals(la, columnNames1)
                 : String.format("Columns [%s])] is not expected [%s]",
-                                ((container == null) ? "" : CellNames.getColumnsString(container.getComparator(), columns)),
+                                ((container == null) ? "" : CellNames.getColumnsString(container.getComparator(), cells)),
                                 StringUtils.join(columnNames1, ","));
     }
 
@@ -615,10 +612,10 @@ public class KeyspaceTest extends SchemaLoader
         assertColumn(cf.getColumn(cellname(name)), value, timestamp);
     }
 
-    public static void assertColumn(Column column, String value, long timestamp)
+    public static void assertColumn(Cell cell, String value, long timestamp)
     {
-        assertNotNull(column);
-        assertEquals(0, ByteBufferUtil.compareUnsigned(column.value(), ByteBufferUtil.bytes(value)));
-        assertEquals(timestamp, column.timestamp());
+        assertNotNull(cell);
+        assertEquals(0, ByteBufferUtil.compareUnsigned(cell.value(), ByteBufferUtil.bytes(value)));
+        assertEquals(timestamp, cell.timestamp());
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/test/unit/org/apache/cassandra/db/NameSortTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/NameSortTest.java b/test/unit/org/apache/cassandra/db/NameSortTest.java
index 523a33a..16a1bc0 100644
--- a/test/unit/org/apache/cassandra/db/NameSortTest.java
+++ b/test/unit/org/apache/cassandra/db/NameSortTest.java
@@ -67,7 +67,7 @@ public class NameSortTest extends SchemaLoader
             {
                 ByteBuffer bytes = j % 2 == 0 ? ByteBufferUtil.bytes("a") : ByteBufferUtil.bytes("b");
                 rm = new RowMutation("Keyspace1", key);
-                rm.add("Standard1", Util.cellname("Column-" + j), bytes, j);
+                rm.add("Standard1", Util.cellname("Cell-" + j), bytes, j);
                 rm.applyUnsafe();
             }
 
@@ -99,13 +99,13 @@ public class NameSortTest extends SchemaLoader
             ColumnFamily cf;
 
             cf = Util.getColumnFamily(keyspace, key, "Standard1");
-            Collection<Column> columns = cf.getSortedColumns();
-            for (Column column : columns)
+            Collection<Cell> cells = cf.getSortedColumns();
+            for (Cell cell : cells)
             {
-                String name = ByteBufferUtil.string(column.name().toByteBuffer());
+                String name = ByteBufferUtil.string(cell.name().toByteBuffer());
                 int j = Integer.valueOf(name.substring(name.length() - 1));
                 byte[] bytes = j % 2 == 0 ? "a".getBytes() : "b".getBytes();
-                assertEquals(new String(bytes), ByteBufferUtil.string(column.value()));
+                assertEquals(new String(bytes), ByteBufferUtil.string(cell.value()));
             }
         }
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java b/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java
index 740eee4..5c37a30 100644
--- a/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java
+++ b/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java
@@ -94,17 +94,17 @@ public class RangeTombstoneTest extends SchemaLoader
         cf = cfs.getColumnFamily(QueryFilter.getNamesFilter(dk(key), CFNAME, columns, System.currentTimeMillis()));
 
         for (int i : live)
-            assert isLive(cf, cf.getColumn(b(i))) : "Column " + i + " should be live";
+            assert isLive(cf, cf.getColumn(b(i))) : "Cell " + i + " should be live";
         for (int i : dead)
-            assert !isLive(cf, cf.getColumn(b(i))) : "Column " + i + " shouldn't be live";
+            assert !isLive(cf, cf.getColumn(b(i))) : "Cell " + i + " shouldn't be live";
 
         // Queries by slices
         cf = cfs.getColumnFamily(QueryFilter.getSliceFilter(dk(key), CFNAME, b(7), b(30), false, Integer.MAX_VALUE, System.currentTimeMillis()));
 
         for (int i : new int[]{ 7, 8, 9, 11, 13, 15, 17, 28, 29, 30 })
-            assert isLive(cf, cf.getColumn(b(i))) : "Column " + i + " should be live";
+            assert isLive(cf, cf.getColumn(b(i))) : "Cell " + i + " should be live";
         for (int i : new int[]{ 10, 12, 14, 16, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27 })
-            assert !isLive(cf, cf.getColumn(b(i))) : "Column " + i + " shouldn't be live";
+            assert !isLive(cf, cf.getColumn(b(i))) : "Cell " + i + " shouldn't be live";
     }
 
     @Test
@@ -146,22 +146,22 @@ public class RangeTombstoneTest extends SchemaLoader
         cf = cfs.getColumnFamily(QueryFilter.getIdentityFilter(dk(key), CFNAME, System.currentTimeMillis()));
 
         for (int i = 0; i < 5; i++)
-            assert isLive(cf, cf.getColumn(b(i))) : "Column " + i + " should be live";
+            assert isLive(cf, cf.getColumn(b(i))) : "Cell " + i + " should be live";
         for (int i = 16; i < 20; i++)
-            assert isLive(cf, cf.getColumn(b(i))) : "Column " + i + " should be live";
+            assert isLive(cf, cf.getColumn(b(i))) : "Cell " + i + " should be live";
         for (int i = 5; i <= 15; i++)
-            assert !isLive(cf, cf.getColumn(b(i))) : "Column " + i + " shouldn't be live";
+            assert !isLive(cf, cf.getColumn(b(i))) : "Cell " + i + " shouldn't be live";
 
         // Compact everything and re-test
         CompactionManager.instance.performMaximal(cfs);
         cf = cfs.getColumnFamily(QueryFilter.getIdentityFilter(dk(key), CFNAME, System.currentTimeMillis()));
 
         for (int i = 0; i < 5; i++)
-            assert isLive(cf, cf.getColumn(b(i))) : "Column " + i + " should be live";
+            assert isLive(cf, cf.getColumn(b(i))) : "Cell " + i + " should be live";
         for (int i = 16; i < 20; i++)
-            assert isLive(cf, cf.getColumn(b(i))) : "Column " + i + " should be live";
+            assert isLive(cf, cf.getColumn(b(i))) : "Cell " + i + " should be live";
         for (int i = 5; i <= 15; i++)
-            assert !isLive(cf, cf.getColumn(b(i))) : "Column " + i + " shouldn't be live";
+            assert !isLive(cf, cf.getColumn(b(i))) : "Cell " + i + " shouldn't be live";
     }
 
     @Test
@@ -253,7 +253,7 @@ public class RangeTombstoneTest extends SchemaLoader
             if (cnt == 0)
                 assertTrue(atom instanceof RangeTombstone);
             if (cnt > 0)
-                assertTrue(atom instanceof Column);
+                assertTrue(atom instanceof Cell);
             cnt++;
         }
         assertEquals(2, cnt);
@@ -345,7 +345,7 @@ public class RangeTombstoneTest extends SchemaLoader
         assertEquals(index.deletes.get(0), index.inserts.get(0));
     }
 
-    private static boolean isLive(ColumnFamily cf, Column c)
+    private static boolean isLive(ColumnFamily cf, Cell c)
     {
         return c != null && !c.isMarkedForDelete(System.currentTimeMillis()) && !cf.deletionInfo().isDeleted(c);
     }
@@ -376,8 +376,8 @@ public class RangeTombstoneTest extends SchemaLoader
 
     public static class TestIndex extends PerColumnSecondaryIndex
     {
-        public List<Column> inserts = new ArrayList<>();
-        public List<Column> deletes = new ArrayList<>();
+        public List<Cell> inserts = new ArrayList<>();
+        public List<Cell> deletes = new ArrayList<>();
 
         public void resetCounts()
         {
@@ -385,17 +385,17 @@ public class RangeTombstoneTest extends SchemaLoader
             deletes.clear();
         }
 
-        public void delete(ByteBuffer rowKey, Column col)
+        public void delete(ByteBuffer rowKey, Cell col)
         {
             deletes.add(col);
         }
 
-        public void insert(ByteBuffer rowKey, Column col)
+        public void insert(ByteBuffer rowKey, Cell col)
         {
             inserts.add(col);
         }
 
-        public void update(ByteBuffer rowKey, Column col){}
+        public void update(ByteBuffer rowKey, Cell col){}
 
         public void init(){}
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/test/unit/org/apache/cassandra/db/ReadMessageTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/ReadMessageTest.java b/test/unit/org/apache/cassandra/db/ReadMessageTest.java
index 7af4256..1f73b35 100644
--- a/test/unit/org/apache/cassandra/db/ReadMessageTest.java
+++ b/test/unit/org/apache/cassandra/db/ReadMessageTest.java
@@ -93,7 +93,7 @@ public class ReadMessageTest extends SchemaLoader
 
         ReadCommand command = new SliceByNamesReadCommand("Keyspace1", dk.key, "Standard1", System.currentTimeMillis(), new NamesQueryFilter(FBUtilities.singleton(Util.cellname("Column1"), type)));
         Row row = command.getRow(keyspace);
-        Column col = row.cf.getColumn(Util.cellname("Column1"));
+        Cell col = row.cf.getColumn(Util.cellname("Column1"));
         assertEquals(col.value(), ByteBuffer.wrap("abcd".getBytes()));
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/test/unit/org/apache/cassandra/db/RecoveryManagerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RecoveryManagerTest.java b/test/unit/org/apache/cassandra/db/RecoveryManagerTest.java
index 323da7c..0c3c23a 100644
--- a/test/unit/org/apache/cassandra/db/RecoveryManagerTest.java
+++ b/test/unit/org/apache/cassandra/db/RecoveryManagerTest.java
@@ -30,7 +30,6 @@ import org.junit.Test;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.db.commitlog.CommitLog;
 import org.apache.cassandra.db.commitlog.CommitLogArchiver;
-import org.apache.cassandra.utils.ByteBufferUtil;
 
 import static org.apache.cassandra.Util.column;
 import static org.apache.cassandra.db.KeyspaceTest.assertColumns;
@@ -85,7 +84,7 @@ public class RecoveryManagerTest extends SchemaLoader
         for (int i = 0; i < 10; ++i)
         {
             cf = TreeMapBackedSortedColumns.factory.create("Keyspace1", "Counter1");
-            cf.addColumn(new CounterColumn(cellname("col"), 1L, 1L));
+            cf.addColumn(new CounterCell(cellname("col"), 1L, 1L));
             rm = new RowMutation("Keyspace1", dk.key, cf);
             rm.apply();
         }
@@ -98,10 +97,10 @@ public class RecoveryManagerTest extends SchemaLoader
         cf = Util.getColumnFamily(keyspace1, dk, "Counter1");
 
         assert cf.getColumnCount() == 1;
-        Column c = cf.getColumn(cellname("col"));
+        Cell c = cf.getColumn(cellname("col"));
 
         assert c != null;
-        assert ((CounterColumn)c).total() == 10L;
+        assert ((CounterCell)c).total() == 10L;
     }
 
     @Test

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/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 28868ad..a5e3876 100644
--- a/test/unit/org/apache/cassandra/db/RecoveryManagerTruncateTest.java
+++ b/test/unit/org/apache/cassandra/db/RecoveryManagerTruncateTest.java
@@ -28,7 +28,6 @@ import java.util.concurrent.ExecutionException;
 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.junit.Test;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
@@ -64,7 +63,7 @@ public class RecoveryManagerTruncateTest extends SchemaLoader
 		assertNull(getFromTable(keyspace, "Standard1", "keymulti", "col1"));
 	}
 
-	private Column getFromTable(Keyspace keyspace, String cfName, String keyName, String columnName)
+	private Cell getFromTable(Keyspace keyspace, String cfName, String keyName, String columnName)
 	{
 		ColumnFamily cf;
 		ColumnFamilyStore cfStore = keyspace.getColumnFamilyStore(cfName);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/test/unit/org/apache/cassandra/db/RemoveCellTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RemoveCellTest.java b/test/unit/org/apache/cassandra/db/RemoveCellTest.java
new file mode 100644
index 0000000..1b03122
--- /dev/null
+++ b/test/unit/org/apache/cassandra/db/RemoveCellTest.java
@@ -0,0 +1,86 @@
+/*
+* 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 org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.cassandra.db.filter.QueryFilter;
+
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.Util;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+
+public class RemoveCellTest extends SchemaLoader
+{
+    @Test
+    public void testRemoveColumn()
+    {
+        Keyspace keyspace = Keyspace.open("Keyspace1");
+        ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard1");
+        RowMutation rm;
+        DecoratedKey dk = Util.dk("key1");
+
+        // add data
+        rm = new RowMutation("Keyspace1", dk.key);
+        rm.add("Standard1", Util.cellname("Column1"), ByteBufferUtil.bytes("asdf"), 0);
+        rm.apply();
+        store.forceBlockingFlush();
+
+        // remove
+        rm = new RowMutation("Keyspace1", dk.key);
+        rm.delete("Standard1", Util.cellname("Column1"), 1);
+        rm.apply();
+
+        ColumnFamily retrieved = store.getColumnFamily(Util.namesQueryFilter(store, dk, "Column1"));
+        assert retrieved.getColumn(Util.cellname("Column1")).isMarkedForDelete(System.currentTimeMillis());
+        assertNull(Util.cloneAndRemoveDeleted(retrieved, Integer.MAX_VALUE));
+        assertNull(Util.cloneAndRemoveDeleted(store.getColumnFamily(QueryFilter.getIdentityFilter(dk,
+                                                                                                  "Standard1",
+                                                                                                  System.currentTimeMillis())),
+                                              Integer.MAX_VALUE));
+    }
+
+    private static DeletedCell dc(String name, int ldt, long timestamp)
+    {
+        return new DeletedCell(Util.cellname(name), ldt, timestamp);
+    }
+
+    @Test
+    public void deletedColumnShouldAlwaysBeMarkedForDelete()
+    {
+        // Check for bug in #4307
+        long timestamp = System.currentTimeMillis();
+        int localDeletionTime = (int) (timestamp / 1000);
+        Cell c = dc("dc1", localDeletionTime, timestamp);
+        assertTrue("DeletedCell was not marked for delete", c.isMarkedForDelete(timestamp));
+
+        // Simulate a node that is 30 seconds behind
+        c = dc("dc2", localDeletionTime + 30, timestamp + 30000);
+        assertTrue("DeletedCell was not marked for delete", c.isMarkedForDelete(timestamp));
+
+        // Simulate a node that is 30 ahead behind
+        c = dc("dc3", localDeletionTime - 30, timestamp - 30000);
+        assertTrue("DeletedCell was not marked for delete", c.isMarkedForDelete(timestamp));
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/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
deleted file mode 100644
index 7ac66e6..0000000
--- a/test/unit/org/apache/cassandra/db/RemoveColumnTest.java
+++ /dev/null
@@ -1,86 +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 org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-
-import org.apache.cassandra.db.filter.QueryFilter;
-
-import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.Util;
-import org.apache.cassandra.utils.ByteBufferUtil;
-
-
-public class RemoveColumnTest extends SchemaLoader
-{
-    @Test
-    public void testRemoveColumn()
-    {
-        Keyspace keyspace = Keyspace.open("Keyspace1");
-        ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard1");
-        RowMutation rm;
-        DecoratedKey dk = Util.dk("key1");
-
-        // add data
-        rm = new RowMutation("Keyspace1", dk.key);
-        rm.add("Standard1", Util.cellname("Column1"), ByteBufferUtil.bytes("asdf"), 0);
-        rm.apply();
-        store.forceBlockingFlush();
-
-        // remove
-        rm = new RowMutation("Keyspace1", dk.key);
-        rm.delete("Standard1", Util.cellname("Column1"), 1);
-        rm.apply();
-
-        ColumnFamily retrieved = store.getColumnFamily(Util.namesQueryFilter(store, dk, "Column1"));
-        assert retrieved.getColumn(Util.cellname("Column1")).isMarkedForDelete(System.currentTimeMillis());
-        assertNull(Util.cloneAndRemoveDeleted(retrieved, Integer.MAX_VALUE));
-        assertNull(Util.cloneAndRemoveDeleted(store.getColumnFamily(QueryFilter.getIdentityFilter(dk,
-                                                                                                  "Standard1",
-                                                                                                  System.currentTimeMillis())),
-                                              Integer.MAX_VALUE));
-    }
-
-    private static DeletedColumn dc(String name, int ldt, long timestamp)
-    {
-        return new DeletedColumn(Util.cellname(name), ldt, timestamp);
-    }
-
-    @Test
-    public void deletedColumnShouldAlwaysBeMarkedForDelete()
-    {
-        // Check for bug in #4307
-        long timestamp = System.currentTimeMillis();
-        int localDeletionTime = (int) (timestamp / 1000);
-        Column c = dc("dc1", localDeletionTime, timestamp);
-        assertTrue("DeletedColumn was not marked for delete", c.isMarkedForDelete(timestamp));
-
-        // Simulate a node that is 30 seconds behind
-        c = dc("dc2", localDeletionTime + 30, timestamp + 30000);
-        assertTrue("DeletedColumn was not marked for delete", c.isMarkedForDelete(timestamp));
-
-        // Simulate a node that is 30 ahead behind
-        c = dc("dc3", localDeletionTime - 30, timestamp - 30000);
-        assertTrue("DeletedColumn was not marked for delete", c.isMarkedForDelete(timestamp));
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/test/unit/org/apache/cassandra/db/RemoveSubCellTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RemoveSubCellTest.java b/test/unit/org/apache/cassandra/db/RemoveSubCellTest.java
new file mode 100644
index 0000000..9c3d29c
--- /dev/null
+++ b/test/unit/org/apache/cassandra/db/RemoveSubCellTest.java
@@ -0,0 +1,100 @@
+/*
+* 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.nio.ByteBuffer;
+import java.util.concurrent.TimeUnit;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertNull;
+import org.apache.cassandra.db.composites.*;
+import org.apache.cassandra.db.filter.QueryFilter;
+import org.apache.cassandra.db.marshal.CompositeType;
+import static org.apache.cassandra.Util.getBytes;
+import org.apache.cassandra.Util;
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import com.google.common.util.concurrent.Uninterruptibles;
+
+
+public class RemoveSubCellTest extends SchemaLoader
+{
+    @Test
+    public void testRemoveSubColumn()
+    {
+        Keyspace keyspace = Keyspace.open("Keyspace1");
+        ColumnFamilyStore store = keyspace.getColumnFamilyStore("Super1");
+        RowMutation rm;
+        DecoratedKey dk = Util.dk("key1");
+
+        // add data
+        rm = new RowMutation("Keyspace1", dk.key);
+        Util.addMutation(rm, "Super1", "SC1", 1, "asdf", 0);
+        rm.apply();
+        store.forceBlockingFlush();
+
+        CellName cname = CellNames.compositeDense(ByteBufferUtil.bytes("SC1"), getBytes(1L));
+        // remove
+        rm = new RowMutation("Keyspace1", dk.key);
+        rm.delete("Super1", cname, 1);
+        rm.apply();
+
+        ColumnFamily retrieved = store.getColumnFamily(QueryFilter.getIdentityFilter(dk, "Super1", System.currentTimeMillis()));
+        assert retrieved.getColumn(cname).isMarkedForDelete(System.currentTimeMillis());
+        assertNull(Util.cloneAndRemoveDeleted(retrieved, Integer.MAX_VALUE));
+    }
+
+    @Test
+    public void testRemoveSubColumnAndContainer()
+    {
+        Keyspace keyspace = Keyspace.open("Keyspace1");
+        ColumnFamilyStore store = keyspace.getColumnFamilyStore("Super1");
+        RowMutation rm;
+        DecoratedKey dk = Util.dk("key2");
+
+        // add data
+        rm = new RowMutation("Keyspace1", dk.key);
+        Util.addMutation(rm, "Super1", "SC1", 1, "asdf", 0);
+        rm.apply();
+        store.forceBlockingFlush();
+
+        // remove the SC
+        ByteBuffer scName = ByteBufferUtil.bytes("SC1");
+        CellName cname = CellNames.compositeDense(scName, getBytes(1L));
+        rm = new RowMutation("Keyspace1", dk.key);
+        rm.deleteRange("Super1", SuperColumns.startOf(scName), SuperColumns.endOf(scName), 1);
+        rm.apply();
+
+        // Mark current time and make sure the next insert happens at least
+        // one second after the previous one (since gc resolution is the second)
+        QueryFilter filter = QueryFilter.getIdentityFilter(dk, "Super1", System.currentTimeMillis());
+        Uninterruptibles.sleepUninterruptibly(1, TimeUnit.SECONDS);
+
+        // remove the column itself
+        rm = new RowMutation("Keyspace1", dk.key);
+        rm.delete("Super1", cname, 2);
+        rm.apply();
+
+        ColumnFamily retrieved = store.getColumnFamily(filter);
+        assert retrieved.getColumn(cname).isMarkedForDelete(System.currentTimeMillis());
+        assertNull(Util.cloneAndRemoveDeleted(retrieved, Integer.MAX_VALUE));
+    }
+}