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

[1/6] Rename Column to Cell

Updated Branches:
  refs/heads/trunk 362cc0535 -> e50d6af12


http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/test/unit/org/apache/cassandra/db/RemoveSubColumnTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RemoveSubColumnTest.java b/test/unit/org/apache/cassandra/db/RemoveSubColumnTest.java
deleted file mode 100644
index e112b1b..0000000
--- a/test/unit/org/apache/cassandra/db/RemoveSubColumnTest.java
+++ /dev/null
@@ -1,100 +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.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 RemoveSubColumnTest 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));
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/test/unit/org/apache/cassandra/db/RowCacheTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RowCacheTest.java b/test/unit/org/apache/cassandra/db/RowCacheTest.java
index 6c3a620..238f61e 100644
--- a/test/unit/org/apache/cassandra/db/RowCacheTest.java
+++ b/test/unit/org/apache/cassandra/db/RowCacheTest.java
@@ -72,15 +72,15 @@ public class RowCacheTest extends SchemaLoader
             assert CacheService.instance.rowCache.size() == i + 1;
             assert cachedStore.containsCachedRow(key); // current key should be stored in the cache
 
-            // checking if column is read correctly after cache
+            // checking if cell is read correctly after cache
             ColumnFamily cf = cachedStore.getColumnFamily(key, Composites.EMPTY, Composites.EMPTY, false, 1, System.currentTimeMillis());
-            Collection<Column> columns = cf.getSortedColumns();
+            Collection<Cell> cells = cf.getSortedColumns();
 
-            Column column = columns.iterator().next();
+            Cell cell = cells.iterator().next();
 
-            assert columns.size() == 1;
-            assert column.name().toByteBuffer().equals(ByteBufferUtil.bytes("col" + i));
-            assert column.value().equals(ByteBufferUtil.bytes("val" + i));
+            assert cells.size() == 1;
+            assert cell.name().toByteBuffer().equals(ByteBufferUtil.bytes("col" + i));
+            assert cell.value().equals(ByteBufferUtil.bytes("val" + i));
         }
 
         // insert 10 more keys
@@ -93,15 +93,15 @@ public class RowCacheTest extends SchemaLoader
             cachedStore.getColumnFamily(key, Composites.EMPTY, Composites.EMPTY, false, 1, System.currentTimeMillis());
             assert cachedStore.containsCachedRow(key); // cache should be populated with the latest rows read (old ones should be popped)
 
-            // checking if column is read correctly after cache
+            // checking if cell is read correctly after cache
             ColumnFamily cf = cachedStore.getColumnFamily(key, Composites.EMPTY, Composites.EMPTY, false, 1, System.currentTimeMillis());
-            Collection<Column> columns = cf.getSortedColumns();
+            Collection<Cell> cells = cf.getSortedColumns();
 
-            Column column = columns.iterator().next();
+            Cell cell = cells.iterator().next();
 
-            assert columns.size() == 1;
-            assert column.name().toByteBuffer().equals(ByteBufferUtil.bytes("col" + i));
-            assert column.value().equals(ByteBufferUtil.bytes("val" + i));
+            assert cells.size() == 1;
+            assert cell.name().toByteBuffer().equals(ByteBufferUtil.bytes("col" + i));
+            assert cell.value().equals(ByteBufferUtil.bytes("val" + i));
         }
 
         // clear 100 rows from the cache

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/test/unit/org/apache/cassandra/db/RowTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RowTest.java b/test/unit/org/apache/cassandra/db/RowTest.java
index d770a6c..d5d22c0 100644
--- a/test/unit/org/apache/cassandra/db/RowTest.java
+++ b/test/unit/org/apache/cassandra/db/RowTest.java
@@ -67,7 +67,7 @@ public class RowTest extends SchemaLoader
     @Test
     public void testExpiringColumnExpiration()
     {
-        Column c = new ExpiringColumn(CellNames.simpleDense(ByteBufferUtil.bytes("one")), ByteBufferUtil.bytes("A"), 0, 1);
+        Cell c = new ExpiringCell(CellNames.simpleDense(ByteBufferUtil.bytes("one")), ByteBufferUtil.bytes("A"), 0, 1);
         assert !c.isMarkedForDelete(System.currentTimeMillis());
 
         // Because we keep the local deletion time with a precision of a

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/test/unit/org/apache/cassandra/db/ScrubTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/ScrubTest.java b/test/unit/org/apache/cassandra/db/ScrubTest.java
index a83d3c6..614858d 100644
--- a/test/unit/org/apache/cassandra/db/ScrubTest.java
+++ b/test/unit/org/apache/cassandra/db/ScrubTest.java
@@ -129,7 +129,7 @@ public class ScrubTest extends SchemaLoader
          * The test also assumes an ordered partitioner.
          *
         ColumnFamily cf = ArrayBackedSortedColumns.factory.create(cfs.metadata);
-        cf.addColumn(new Column(ByteBufferUtil.bytes("someName"), ByteBufferUtil.bytes("someValue"), 0L));
+        cf.addColumn(new Cell(ByteBufferUtil.bytes("someName"), ByteBufferUtil.bytes("someValue"), 0L));
 
         SSTableWriter writer = new SSTableWriter(cfs.getTempSSTablePath(new File(System.getProperty("corrupt-sstable-root"))),
                                                  cfs.metadata.getIndexInterval(),

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/test/unit/org/apache/cassandra/db/SecondaryIndexCellSizeTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/SecondaryIndexCellSizeTest.java b/test/unit/org/apache/cassandra/db/SecondaryIndexCellSizeTest.java
new file mode 100644
index 0000000..940a565
--- /dev/null
+++ b/test/unit/org/apache/cassandra/db/SecondaryIndexCellSizeTest.java
@@ -0,0 +1,230 @@
+/*
+* 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.Set;
+
+import org.junit.Test;
+
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.db.composites.*;
+import org.apache.cassandra.db.index.PerColumnSecondaryIndex;
+import org.apache.cassandra.db.index.PerRowSecondaryIndex;
+import org.apache.cassandra.db.index.SecondaryIndexSearcher;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+public class SecondaryIndexCellSizeTest
+{
+    @Test
+    public void test64kColumn()
+    {
+        // a byte buffer more than 64k
+        ByteBuffer buffer = ByteBuffer.allocate(1024 * 65);
+        buffer.clear();
+
+        //read more than 64k
+        for (int i=0; i<1024*64/4 + 1; i++)
+            buffer.putInt(0);
+
+        // for read
+        buffer.flip();
+        Cell cell = new Cell(CellNames.simpleDense(ByteBufferUtil.bytes("test")), buffer, 0);
+
+        SecondaryIndexCellSizeTest.MockRowIndex mockRowIndex = new SecondaryIndexCellSizeTest.MockRowIndex();
+        SecondaryIndexCellSizeTest.MockColumnIndex mockColumnIndex = new SecondaryIndexCellSizeTest.MockColumnIndex();
+
+        assertTrue(mockRowIndex.validate(cell));
+        assertFalse(mockColumnIndex.validate(cell));
+
+        // test less than 64k value
+        buffer.flip();
+        buffer.clear();
+        buffer.putInt(20);
+        buffer.flip();
+
+        assertTrue(mockRowIndex.validate(cell));
+        assertTrue(mockColumnIndex.validate(cell));
+    }
+
+    private class MockRowIndex extends PerRowSecondaryIndex
+    {
+        @Override
+        public void init()
+        {
+        }
+
+        @Override
+        public void validateOptions() throws ConfigurationException
+        {
+        }
+
+        @Override
+        public String getIndexName()
+        {
+            return null;
+        }
+
+        @Override
+        protected SecondaryIndexSearcher createSecondaryIndexSearcher(Set<ByteBuffer> columns)
+        {
+            return null;
+        }
+
+        @Override
+        public void forceBlockingFlush()
+        {
+        }
+
+        @Override
+        public long getLiveSize()
+        {
+            return 0;
+        }
+
+        @Override
+        public ColumnFamilyStore getIndexCfs()
+        {
+            return null;
+        }
+
+        @Override
+        public void removeIndex(ByteBuffer columnName)
+        {
+        }
+
+        @Override
+        public void invalidate()
+        {
+        }
+
+        @Override
+        public void truncateBlocking(long truncatedAt)
+        {
+        }
+
+        public void index(ByteBuffer rowKey, ColumnFamily cf)
+        {
+        }
+
+        public void index(ByteBuffer rowKey)
+        {
+        }
+
+        public void delete(DecoratedKey key)
+        {
+        }
+
+        @Override
+        public void reload()
+        {
+        }
+
+        public boolean indexes(CellName name)
+        {
+            return true;
+        }
+    }
+
+
+    private class MockColumnIndex extends PerColumnSecondaryIndex
+    {
+        @Override
+        public void init()
+        {
+        }
+
+        @Override
+        public void validateOptions() throws ConfigurationException
+        {
+        }
+
+        @Override
+        public String getIndexName()
+        {
+            return null;
+        }
+
+        @Override
+        protected SecondaryIndexSearcher createSecondaryIndexSearcher(Set<ByteBuffer> columns)
+        {
+            return null;
+        }
+
+        @Override
+        public void forceBlockingFlush()
+        {
+        }
+
+        @Override
+        public long getLiveSize()
+        {
+            return 0;
+        }
+
+        @Override
+        public ColumnFamilyStore getIndexCfs()
+        {
+            return null;
+        }
+
+        @Override
+        public void removeIndex(ByteBuffer columnName)
+        {
+        }
+
+        @Override
+        public void invalidate()
+        {
+        }
+
+        @Override
+        public void truncateBlocking(long truncatedAt)
+        {
+        }
+
+        @Override
+        public void delete(ByteBuffer rowKey, Cell col)
+        {
+        }
+
+        @Override
+        public void insert(ByteBuffer rowKey, Cell col)
+        {
+        }
+
+        @Override
+        public void update(ByteBuffer rowKey, Cell col)
+        {
+        }
+
+        @Override
+        public void reload()
+        {
+        }
+
+        public boolean indexes(CellName name)
+        {
+            return true;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/test/unit/org/apache/cassandra/db/SecondaryIndexColumnSizeTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/SecondaryIndexColumnSizeTest.java b/test/unit/org/apache/cassandra/db/SecondaryIndexColumnSizeTest.java
deleted file mode 100644
index 0dbc12a..0000000
--- a/test/unit/org/apache/cassandra/db/SecondaryIndexColumnSizeTest.java
+++ /dev/null
@@ -1,230 +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.nio.ByteBuffer;
-import java.util.Set;
-
-import org.junit.Test;
-
-import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.db.composites.*;
-import org.apache.cassandra.db.index.PerColumnSecondaryIndex;
-import org.apache.cassandra.db.index.PerRowSecondaryIndex;
-import org.apache.cassandra.db.index.SecondaryIndexSearcher;
-import org.apache.cassandra.utils.ByteBufferUtil;
-
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-public class SecondaryIndexColumnSizeTest
-{
-    @Test
-    public void test64kColumn()
-    {
-        // a byte buffer more than 64k
-        ByteBuffer buffer = ByteBuffer.allocate(1024 * 65);
-        buffer.clear();
-
-        //read more than 64k
-        for (int i=0; i<1024*64/4 + 1; i++)
-            buffer.putInt(0);
-
-        // for read
-        buffer.flip();
-        Column column = new Column(CellNames.simpleDense(ByteBufferUtil.bytes("test")), buffer, 0);
-
-        SecondaryIndexColumnSizeTest.MockRowIndex mockRowIndex = new SecondaryIndexColumnSizeTest.MockRowIndex();
-        SecondaryIndexColumnSizeTest.MockColumnIndex mockColumnIndex = new SecondaryIndexColumnSizeTest.MockColumnIndex();
-
-        assertTrue(mockRowIndex.validate(column));
-        assertFalse(mockColumnIndex.validate(column));
-
-        // test less than 64k value
-        buffer.flip();
-        buffer.clear();
-        buffer.putInt(20);
-        buffer.flip();
-
-        assertTrue(mockRowIndex.validate(column));
-        assertTrue(mockColumnIndex.validate(column));
-    }
-
-    private class MockRowIndex extends PerRowSecondaryIndex
-    {
-        @Override
-        public void init()
-        {
-        }
-
-        @Override
-        public void validateOptions() throws ConfigurationException
-        {
-        }
-
-        @Override
-        public String getIndexName()
-        {
-            return null;
-        }
-
-        @Override
-        protected SecondaryIndexSearcher createSecondaryIndexSearcher(Set<ByteBuffer> columns)
-        {
-            return null;
-        }
-
-        @Override
-        public void forceBlockingFlush()
-        {
-        }
-
-        @Override
-        public long getLiveSize()
-        {
-            return 0;
-        }
-
-        @Override
-        public ColumnFamilyStore getIndexCfs()
-        {
-            return null;
-        }
-
-        @Override
-        public void removeIndex(ByteBuffer columnName)
-        {
-        }
-
-        @Override
-        public void invalidate()
-        {
-        }
-
-        @Override
-        public void truncateBlocking(long truncatedAt)
-        {
-        }
-
-        public void index(ByteBuffer rowKey, ColumnFamily cf)
-        {
-        }
-
-        public void index(ByteBuffer rowKey)
-        {
-        }
-
-        public void delete(DecoratedKey key)
-        {
-        }
-
-        @Override
-        public void reload()
-        {
-        }
-
-        public boolean indexes(CellName name)
-        {
-            return true;
-        }
-    }
-
-
-    private class MockColumnIndex extends PerColumnSecondaryIndex
-    {
-        @Override
-        public void init()
-        {
-        }
-
-        @Override
-        public void validateOptions() throws ConfigurationException
-        {
-        }
-
-        @Override
-        public String getIndexName()
-        {
-            return null;
-        }
-
-        @Override
-        protected SecondaryIndexSearcher createSecondaryIndexSearcher(Set<ByteBuffer> columns)
-        {
-            return null;
-        }
-
-        @Override
-        public void forceBlockingFlush()
-        {
-        }
-
-        @Override
-        public long getLiveSize()
-        {
-            return 0;
-        }
-
-        @Override
-        public ColumnFamilyStore getIndexCfs()
-        {
-            return null;
-        }
-
-        @Override
-        public void removeIndex(ByteBuffer columnName)
-        {
-        }
-
-        @Override
-        public void invalidate()
-        {
-        }
-
-        @Override
-        public void truncateBlocking(long truncatedAt)
-        {
-        }
-
-        @Override
-        public void delete(ByteBuffer rowKey, Column col)
-        {
-        }
-
-        @Override
-        public void insert(ByteBuffer rowKey, Column col)
-        {
-        }
-
-        @Override
-        public void update(ByteBuffer rowKey, Column col)
-        {
-        }
-
-        @Override
-        public void reload()
-        {
-        }
-
-        public boolean indexes(CellName name)
-        {
-            return true;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/test/unit/org/apache/cassandra/db/SerializationsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/SerializationsTest.java b/test/unit/org/apache/cassandra/db/SerializationsTest.java
index 7b7edfa..e3a6077 100644
--- a/test/unit/org/apache/cassandra/db/SerializationsTest.java
+++ b/test/unit/org/apache/cassandra/db/SerializationsTest.java
@@ -24,7 +24,6 @@ import org.apache.cassandra.db.composites.*;
 import org.apache.cassandra.db.filter.*;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.marshal.BytesType;
-import org.apache.cassandra.db.marshal.CompositeType;
 import org.apache.cassandra.dht.AbstractBounds;
 import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.dht.Range;
@@ -377,21 +376,21 @@ public class SerializationsTest extends AbstractSerializationsTester
 
         private Statics()
         {
-            StandardCf.addColumn(new Column(cn("aaaa")));
-            StandardCf.addColumn(new Column(cn("bbbb"), bb("bbbbb-value")));
-            StandardCf.addColumn(new Column(cn("cccc"), bb("ccccc-value"), 1000L));
-            StandardCf.addColumn(new DeletedColumn(cn("dddd"), 500, 1000));
-            StandardCf.addColumn(new DeletedColumn(cn("eeee"), bb("eeee-value"), 1001));
-            StandardCf.addColumn(new ExpiringColumn(cn("ffff"), bb("ffff-value"), 2000, 1000));
-            StandardCf.addColumn(new ExpiringColumn(cn("gggg"), bb("gggg-value"), 2001, 1000, 2002));
-
-            SuperCf.addColumn(new Column(CellNames.compositeDense(SC, bb("aaaa"))));
-            SuperCf.addColumn(new Column(CellNames.compositeDense(SC, bb("bbbb")), bb("bbbbb-value")));
-            SuperCf.addColumn(new Column(CellNames.compositeDense(SC, bb("cccc")), bb("ccccc-value"), 1000L));
-            SuperCf.addColumn(new DeletedColumn(CellNames.compositeDense(SC, bb("dddd")), 500, 1000));
-            SuperCf.addColumn(new DeletedColumn(CellNames.compositeDense(SC, bb("eeee")), bb("eeee-value"), 1001));
-            SuperCf.addColumn(new ExpiringColumn(CellNames.compositeDense(SC, bb("ffff")), bb("ffff-value"), 2000, 1000));
-            SuperCf.addColumn(new ExpiringColumn(CellNames.compositeDense(SC, bb("gggg")), bb("gggg-value"), 2001, 1000, 2002));
+            StandardCf.addColumn(new Cell(cn("aaaa")));
+            StandardCf.addColumn(new Cell(cn("bbbb"), bb("bbbbb-value")));
+            StandardCf.addColumn(new Cell(cn("cccc"), bb("ccccc-value"), 1000L));
+            StandardCf.addColumn(new DeletedCell(cn("dddd"), 500, 1000));
+            StandardCf.addColumn(new DeletedCell(cn("eeee"), bb("eeee-value"), 1001));
+            StandardCf.addColumn(new ExpiringCell(cn("ffff"), bb("ffff-value"), 2000, 1000));
+            StandardCf.addColumn(new ExpiringCell(cn("gggg"), bb("gggg-value"), 2001, 1000, 2002));
+
+            SuperCf.addColumn(new Cell(CellNames.compositeDense(SC, bb("aaaa"))));
+            SuperCf.addColumn(new Cell(CellNames.compositeDense(SC, bb("bbbb")), bb("bbbbb-value")));
+            SuperCf.addColumn(new Cell(CellNames.compositeDense(SC, bb("cccc")), bb("ccccc-value"), 1000L));
+            SuperCf.addColumn(new DeletedCell(CellNames.compositeDense(SC, bb("dddd")), 500, 1000));
+            SuperCf.addColumn(new DeletedCell(CellNames.compositeDense(SC, bb("eeee")), bb("eeee-value"), 1001));
+            SuperCf.addColumn(new ExpiringCell(CellNames.compositeDense(SC, bb("ffff")), bb("ffff-value"), 2000, 1000));
+            SuperCf.addColumn(new ExpiringCell(CellNames.compositeDense(SC, bb("gggg")), bb("gggg-value"), 2001, 1000, 2002));
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/test/unit/org/apache/cassandra/db/TimeSortTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/TimeSortTest.java b/test/unit/org/apache/cassandra/db/TimeSortTest.java
index a4daf4f..f8e9dbc 100644
--- a/test/unit/org/apache/cassandra/db/TimeSortTest.java
+++ b/test/unit/org/apache/cassandra/db/TimeSortTest.java
@@ -19,7 +19,6 @@
 package org.apache.cassandra.db;
 
 import java.io.IOException;
-import java.nio.ByteBuffer;
 import java.util.concurrent.ExecutionException;
 import java.util.*;
 
@@ -33,7 +32,6 @@ import org.apache.cassandra.Util;
 
 import org.apache.cassandra.db.composites.*;
 import org.apache.cassandra.db.filter.QueryFilter;
-import org.apache.cassandra.db.marshal.LongType;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 
@@ -57,8 +55,8 @@ public class TimeSortTest extends SchemaLoader
         rm.apply();
 
         ColumnFamily cf = cfStore.getColumnFamily(key, cellname(10), Composites.EMPTY, false, 1000, System.currentTimeMillis());
-        Collection<Column> columns = cf.getSortedColumns();
-        assert columns.size() == 1;
+        Collection<Cell> cells = cf.getSortedColumns();
+        assert cells.size() == 1;
     }
 
     @Test
@@ -98,14 +96,14 @@ public class TimeSortTest extends SchemaLoader
 
         // verify
         ColumnFamily cf = cfStore.getColumnFamily(key, cellname(0), Composites.EMPTY, false, 1000, System.currentTimeMillis());
-        Collection<Column> columns = cf.getSortedColumns();
-        assertEquals(12, columns.size());
-        Iterator<Column> iter = columns.iterator();
-        Column column;
+        Collection<Cell> cells = cf.getSortedColumns();
+        assertEquals(12, cells.size());
+        Iterator<Cell> iter = cells.iterator();
+        Cell cell;
         for (int j = 0; j < 8; j++)
         {
-            column = iter.next();
-            assert column.name().toByteBuffer().equals(getBytes(j));
+            cell = iter.next();
+            assert cell.name().toByteBuffer().equals(getBytes(j));
         }
         TreeSet<CellName> columnNames = new TreeSet<CellName>(cfStore.getComparator());
         columnNames.add(cellname(10));
@@ -124,10 +122,10 @@ public class TimeSortTest extends SchemaLoader
             {
                 ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("StandardLong1");
                 ColumnFamily cf = cfs.getColumnFamily(key, cellname(j * 2), Composites.EMPTY, false, 1000, System.currentTimeMillis());
-                Collection<Column> columns = cf.getSortedColumns();
-                assert columns.size() == 8 - j;
+                Collection<Cell> cells = cf.getSortedColumns();
+                assert cells.size() == 8 - j;
                 int k = j;
-                for (Column c : columns)
+                for (Cell c : cells)
                 {
                     assertEquals((k++) * 2, c.timestamp());
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/test/unit/org/apache/cassandra/db/compaction/CompactionsPurgeTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/CompactionsPurgeTest.java b/test/unit/org/apache/cassandra/db/compaction/CompactionsPurgeTest.java
index c5f5483..03a8c71 100644
--- a/test/unit/org/apache/cassandra/db/compaction/CompactionsPurgeTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/CompactionsPurgeTest.java
@@ -23,11 +23,11 @@ import java.util.Collection;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Future;
 
+import org.apache.cassandra.db.Cell;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.cql3.UntypedResultSet;
-import org.apache.cassandra.db.Column;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.RowMutation;
@@ -277,7 +277,7 @@ public class CompactionsPurgeTest extends SchemaLoader
         // Check that the second insert did went in
         ColumnFamily cf = cfs.getColumnFamily(QueryFilter.getIdentityFilter(key, cfName, System.currentTimeMillis()));
         assertEquals(10, cf.getColumnCount());
-        for (Column c : cf)
+        for (Cell c : cf)
             assert !c.isMarkedForDelete(System.currentTimeMillis());
     }
 
@@ -321,7 +321,7 @@ public class CompactionsPurgeTest extends SchemaLoader
         // Check that the second insert went in
         cf = cfs.getColumnFamily(filter);
         assertEquals(10, cf.getColumnCount());
-        for (Column c : cf)
+        for (Cell c : cf)
             assert !c.isMarkedForDelete(System.currentTimeMillis());
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/test/unit/org/apache/cassandra/db/index/PerRowSecondaryIndexTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/index/PerRowSecondaryIndexTest.java b/test/unit/org/apache/cassandra/db/index/PerRowSecondaryIndexTest.java
index 7b422b2..a4fb875 100644
--- a/test/unit/org/apache/cassandra/db/index/PerRowSecondaryIndexTest.java
+++ b/test/unit/org/apache/cassandra/db/index/PerRowSecondaryIndexTest.java
@@ -90,9 +90,9 @@ public class PerRowSecondaryIndexTest extends SchemaLoader
         ColumnFamily indexedRow = PerRowSecondaryIndexTest.TestIndex.LAST_INDEXED_ROW;
         assertNotNull(indexedRow);
 
-        for (Column column : indexedRow.getSortedColumns())
+        for (Cell cell : indexedRow.getSortedColumns())
         {
-            assertTrue(column.isMarkedForDelete(System.currentTimeMillis()));
+            assertTrue(cell.isMarkedForDelete(System.currentTimeMillis()));
         }
         assertTrue(Arrays.equals("k2".getBytes(), PerRowSecondaryIndexTest.TestIndex.LAST_INDEXED_KEY.array()));
     }
@@ -108,9 +108,9 @@ public class PerRowSecondaryIndexTest extends SchemaLoader
 
         ColumnFamily indexedRow = PerRowSecondaryIndexTest.TestIndex.LAST_INDEXED_ROW;
         assertNotNull(indexedRow);
-        for (Column column : indexedRow.getSortedColumns())
+        for (Cell cell : indexedRow.getSortedColumns())
         {
-            assertTrue(column.isMarkedForDelete(System.currentTimeMillis()));
+            assertTrue(cell.isMarkedForDelete(System.currentTimeMillis()));
         }
         assertTrue(Arrays.equals("k3".getBytes(), PerRowSecondaryIndexTest.TestIndex.LAST_INDEXED_KEY.array()));
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/test/unit/org/apache/cassandra/db/marshal/CompositeTypeTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/marshal/CompositeTypeTest.java b/test/unit/org/apache/cassandra/db/marshal/CompositeTypeTest.java
index b7452b1..ac85ccf 100644
--- a/test/unit/org/apache/cassandra/db/marshal/CompositeTypeTest.java
+++ b/test/unit/org/apache/cassandra/db/marshal/CompositeTypeTest.java
@@ -184,7 +184,7 @@ public class CompositeTypeTest extends SchemaLoader
 
         ColumnFamily cf = cfs.getColumnFamily(QueryFilter.getIdentityFilter(Util.dk("k"), cfName, System.currentTimeMillis()));
 
-        Iterator<Column> iter = cf.getSortedColumns().iterator();
+        Iterator<Cell> iter = cf.getSortedColumns().iterator();
 
         assert iter.next().name().toByteBuffer().equals(cname1);
         assert iter.next().name().toByteBuffer().equals(cname2);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/test/unit/org/apache/cassandra/db/marshal/DynamicCompositeTypeTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/marshal/DynamicCompositeTypeTest.java b/test/unit/org/apache/cassandra/db/marshal/DynamicCompositeTypeTest.java
index fd489ed..15b7650 100644
--- a/test/unit/org/apache/cassandra/db/marshal/DynamicCompositeTypeTest.java
+++ b/test/unit/org/apache/cassandra/db/marshal/DynamicCompositeTypeTest.java
@@ -183,7 +183,7 @@ public class DynamicCompositeTypeTest extends SchemaLoader
 
         ColumnFamily cf = cfs.getColumnFamily(QueryFilter.getIdentityFilter(Util.dk("k"), cfName, System.currentTimeMillis()));
 
-        Iterator<Column> iter = cf.getSortedColumns().iterator();
+        Iterator<Cell> iter = cf.getSortedColumns().iterator();
 
         assert iter.next().name().toByteBuffer().equals(cname1);
         assert iter.next().name().toByteBuffer().equals(cname2);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/test/unit/org/apache/cassandra/io/sstable/IndexSummaryManagerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/IndexSummaryManagerTest.java b/test/unit/org/apache/cassandra/io/sstable/IndexSummaryManagerTest.java
index 94cc2d4..385c06b 100644
--- a/test/unit/org/apache/cassandra/io/sstable/IndexSummaryManagerTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/IndexSummaryManagerTest.java
@@ -32,7 +32,6 @@ import org.apache.cassandra.Util;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.filter.QueryFilter;
 import org.apache.cassandra.metrics.RestorableMeter;
-import org.apache.cassandra.utils.ByteBufferUtil;
 
 import static org.apache.cassandra.io.sstable.Downsampling.BASE_SAMPLING_LEVEL;
 import static org.apache.cassandra.io.sstable.Downsampling.MIN_SAMPLING_LEVEL;
@@ -78,9 +77,9 @@ public class IndexSummaryManagerTest extends SchemaLoader
             QueryFilter filter = QueryFilter.getIdentityFilter(key, cfs.getColumnFamilyName(), System.currentTimeMillis());
             ColumnFamily row = cfs.getColumnFamily(filter);
             assertNotNull(row);
-            Column column = row.getColumn(Util.cellname("column"));
-            assertNotNull(column);
-            assertEquals(100, column.value().array().length);
+            Cell cell = row.getColumn(Util.cellname("cell"));
+            assertNotNull(cell);
+            assertEquals(100, cell.value().array().length);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/test/unit/org/apache/cassandra/io/sstable/SSTableSimpleWriterTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableSimpleWriterTest.java b/test/unit/org/apache/cassandra/io/sstable/SSTableSimpleWriterTest.java
index b61dc4c..9aaeca5 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableSimpleWriterTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableSimpleWriterTest.java
@@ -92,9 +92,9 @@ public class SSTableSimpleWriterTest extends SchemaLoader
         ColumnFamily cf = Util.getColumnFamily(t, Util.dk("Key10"), cfname);
         assert cf.getColumnCount() == INC * NBCOL : "expecting " + (INC * NBCOL) + " columns, got " + cf.getColumnCount();
         int i = 0;
-        for (Column c : cf)
+        for (Cell c : cf)
         {
-            assert toInt(c.name().toByteBuffer()) == i : "Column name should be " + i + ", got " + toInt(c.name().toByteBuffer());
+            assert toInt(c.name().toByteBuffer()) == i : "Cell name should be " + i + ", got " + toInt(c.name().toByteBuffer());
             assert c.value().equals(bytes("v"));
             assert c.timestamp() == 1;
             ++i;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/test/unit/org/apache/cassandra/io/sstable/SSTableUtils.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableUtils.java b/test/unit/org/apache/cassandra/io/sstable/SSTableUtils.java
index f7687c1..561b36c 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableUtils.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableUtils.java
@@ -36,11 +36,11 @@ public class SSTableUtils
     public static String KEYSPACENAME = "Keyspace1";
     public static String CFNAME = "Standard1";
 
-    public static ColumnFamily createCF(long mfda, int ldt, Column... cols)
+    public static ColumnFamily createCF(long mfda, int ldt, Cell... cols)
     {
         ColumnFamily cf = TreeMapBackedSortedColumns.factory.create(KEYSPACENAME, CFNAME);
         cf.delete(new DeletionInfo(mfda, ldt));
-        for (Column col : cols)
+        for (Cell col : cols)
             cf.addColumn(col);
         return cf;
     }
@@ -98,9 +98,9 @@ public class SSTableUtils
         // iterate columns
         while (lhs.hasNext())
         {
-            Column clhs = (Column)lhs.next();
+            Cell clhs = (Cell)lhs.next();
             assert rhs.hasNext() : "LHS contained more columns than RHS for " + lhs.getKey();
-            Column crhs = (Column)rhs.next();
+            Cell crhs = (Cell)rhs.next();
 
             assertEquals("Mismatched columns for " + lhs.getKey(), clhs, crhs);
         }
@@ -163,7 +163,7 @@ public class SSTableUtils
             for (String key : keys)
             {
                 ColumnFamily cf = TreeMapBackedSortedColumns.factory.create(ksname, cfname);
-                cf.addColumn(new Column(Util.cellname(key), ByteBufferUtil.bytes(key), 0));
+                cf.addColumn(new Cell(Util.cellname(key), ByteBufferUtil.bytes(key), 0));
                 map.put(key, cf);
             }
             return write(map);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/test/unit/org/apache/cassandra/service/QueryPagerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/QueryPagerTest.java b/test/unit/org/apache/cassandra/service/QueryPagerTest.java
index 0d0efb9..9e28fe6 100644
--- a/test/unit/org/apache/cassandra/service/QueryPagerTest.java
+++ b/test/unit/org/apache/cassandra/service/QueryPagerTest.java
@@ -108,7 +108,7 @@ public class QueryPagerTest extends SchemaLoader
             return "";
 
         StringBuilder sb = new StringBuilder();
-        for (Column c : cf)
+        for (Cell c : cf)
             sb.append(" ").append(string(c.name()));
         return sb.toString();
     }
@@ -153,7 +153,7 @@ public class QueryPagerTest extends SchemaLoader
         assertNotNull(r.cf);
         assertEquals(toString(r.cf), names.length, r.cf.getColumnCount());
         int i = 0;
-        for (Column c : r.cf)
+        for (Cell c : r.cf)
         {
             String expected = names[i++];
             assertEquals("column " + i + " doesn't match: " + toString(r.cf), expected, string(c.name()));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java b/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
index 1da33fc..412d9d0 100644
--- a/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
+++ b/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
@@ -229,7 +229,7 @@ public class StreamingTransferTest extends SchemaLoader
                 long val = key.hashCode();
                 ColumnFamily cf = TreeMapBackedSortedColumns.factory.create(keyspace.getName(), cfs.name);
                 cf.addColumn(column(col, "v", timestamp));
-                cf.addColumn(new Column(cellname("birthdate"), ByteBufferUtil.bytes(val), timestamp));
+                cf.addColumn(new Cell(cellname("birthdate"), ByteBufferUtil.bytes(val), timestamp));
                 RowMutation rm = new RowMutation("Keyspace1", ByteBufferUtil.bytes(key), cf);
                 logger.debug("Applying row to transfer " + rm);
                 rm.apply();
@@ -320,8 +320,8 @@ public class StreamingTransferTest extends SchemaLoader
                 state.writeElement(CounterId.fromInt(4), 4L, 2L);
                 state.writeElement(CounterId.fromInt(6), 3L, 3L);
                 state.writeElement(CounterId.fromInt(8), 2L, 4L);
-                cf.addColumn(new CounterColumn(cellname(col), state.context, timestamp));
-                cfCleaned.addColumn(new CounterColumn(cellname(col), cc.clearAllDelta(state.context), timestamp));
+                cf.addColumn(new CounterCell(cellname(col), state.context, timestamp));
+                cfCleaned.addColumn(new CounterCell(cellname(col), cc.clearAllDelta(state.context), timestamp));
 
                 entries.put(key, cf);
                 cleanedEntries.put(key, cfCleaned);
@@ -453,7 +453,7 @@ public class StreamingTransferTest extends SchemaLoader
             {
                 ColumnFamily cf = TreeMapBackedSortedColumns.factory.create(keyspace.getName(), cfs.name);
                 cf.addColumn(column(colName, "value", timestamp));
-                cf.addColumn(new Column(cellname("birthdate"), ByteBufferUtil.bytes(new Date(timestamp).toString()), timestamp));
+                cf.addColumn(new Cell(cellname("birthdate"), ByteBufferUtil.bytes(new Date(timestamp).toString()), timestamp));
                 RowMutation rm = new RowMutation("Keyspace1", ByteBufferUtil.bytes(key), cf);
                 logger.debug("Applying row to transfer " + rm);
                 rm.apply();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/test/unit/org/apache/cassandra/tools/SSTableExportTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/tools/SSTableExportTest.java b/test/unit/org/apache/cassandra/tools/SSTableExportTest.java
index c2de740..ea7516f 100644
--- a/test/unit/org/apache/cassandra/tools/SSTableExportTest.java
+++ b/test/unit/org/apache/cassandra/tools/SSTableExportTest.java
@@ -123,7 +123,7 @@ public class SSTableExportTest extends SchemaLoader
         int nowInSec = (int)(System.currentTimeMillis() / 1000) + 42; //live for 42 seconds
         // Add rowA
         cfamily.addColumn(Util.cellname("colA"), ByteBufferUtil.bytes("valA"), System.currentTimeMillis());
-        cfamily.addColumn(new ExpiringColumn(Util.cellname("colExp"), ByteBufferUtil.bytes("valExp"), System.currentTimeMillis(), 42, nowInSec));
+        cfamily.addColumn(new ExpiringCell(Util.cellname("colExp"), ByteBufferUtil.bytes("valExp"), System.currentTimeMillis(), 42, nowInSec));
         writer.append(Util.dk("rowA"), cfamily);
         cfamily.clear();
 
@@ -216,7 +216,7 @@ public class SSTableExportTest extends SchemaLoader
         SSTableWriter writer = new SSTableWriter(tempSS.getPath(), 2);
 
         // Add rowA
-        cfamily.addColumn(new CounterColumn(Util.cellname("colA"), 42, System.currentTimeMillis()));
+        cfamily.addColumn(new CounterCell(Util.cellname("colA"), 42, System.currentTimeMillis()));
         writer.append(Util.dk("rowA"), cfamily);
         cfamily.clear();
 
@@ -247,7 +247,7 @@ public class SSTableExportTest extends SchemaLoader
         SSTableWriter writer = new SSTableWriter(tempSS.getPath(), 2);
 
         // Add rowA
-        cfamily.addColumn(new Column(Util.cellname("data"), UTF8Type.instance.fromString("{\"foo\":\"bar\"}")));
+        cfamily.addColumn(new Cell(Util.cellname("data"), UTF8Type.instance.fromString("{\"foo\":\"bar\"}")));
         writer.append(Util.dk("rowA"), cfamily);
         cfamily.clear();
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/test/unit/org/apache/cassandra/tools/SSTableImportTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/tools/SSTableImportTest.java b/test/unit/org/apache/cassandra/tools/SSTableImportTest.java
index fdfaa82..6434143 100644
--- a/test/unit/org/apache/cassandra/tools/SSTableImportTest.java
+++ b/test/unit/org/apache/cassandra/tools/SSTableImportTest.java
@@ -34,7 +34,6 @@ import org.apache.cassandra.Util;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.columniterator.OnDiskAtomIterator;
 import org.apache.cassandra.db.filter.QueryFilter;
-import org.apache.cassandra.db.marshal.CompositeType;
 import org.apache.cassandra.io.sstable.Descriptor;
 import org.apache.cassandra.io.sstable.SSTableReader;
 import org.apache.cassandra.utils.ByteBufferUtil;
@@ -56,11 +55,11 @@ public class SSTableImportTest extends SchemaLoader
         ColumnFamily cf = cloneForAdditions(iter);
         while (iter.hasNext()) cf.addAtom(iter.next());
         assert cf.getColumn(Util.cellname("colAA")).value().equals(hexToBytes("76616c4141"));
-        assert !(cf.getColumn(Util.cellname("colAA")) instanceof DeletedColumn);
-        Column expCol = cf.getColumn(Util.cellname("colAC"));
+        assert !(cf.getColumn(Util.cellname("colAA")) instanceof DeletedCell);
+        Cell expCol = cf.getColumn(Util.cellname("colAC"));
         assert expCol.value().equals(hexToBytes("76616c4143"));
-        assert expCol instanceof ExpiringColumn;
-        assert ((ExpiringColumn)expCol).getTimeToLive() == 42 && expCol.getLocalDeletionTime() == 2000000000;
+        assert expCol instanceof ExpiringCell;
+        assert ((ExpiringCell)expCol).getTimeToLive() == 42 && expCol.getLocalDeletionTime() == 2000000000;
     }
 
     private ColumnFamily cloneForAdditions(OnDiskAtomIterator iter)
@@ -90,11 +89,11 @@ public class SSTableImportTest extends SchemaLoader
         ColumnFamily cf = cloneForAdditions(iter);
         while (iter.hasNext()) cf.addAtom(iter.next());
         assert cf.getColumn(Util.cellname("colAA")).value().equals(hexToBytes("76616c4141"));
-        assert !(cf.getColumn(Util.cellname("colAA")) instanceof DeletedColumn);
-        Column expCol = cf.getColumn(Util.cellname("colAC"));
+        assert !(cf.getColumn(Util.cellname("colAA")) instanceof DeletedCell);
+        Cell expCol = cf.getColumn(Util.cellname("colAC"));
         assert expCol.value().equals(hexToBytes("76616c4143"));
-        assert expCol instanceof ExpiringColumn;
-        assert ((ExpiringColumn)expCol).getTimeToLive() == 42 && expCol.getLocalDeletionTime() == 2000000000;
+        assert expCol instanceof ExpiringCell;
+        assert ((ExpiringCell)expCol).getTimeToLive() == 42 && expCol.getLocalDeletionTime() == 2000000000;
     }
 
     @Test
@@ -112,8 +111,8 @@ public class SSTableImportTest extends SchemaLoader
 
         DeletionTime delTime = cf.deletionInfo().rangeCovering(cf.getComparator().make(ByteBufferUtil.bytes("superA")));
         assertEquals("supercolumn deletion time did not match the expected time", new DeletionInfo(0, 0), new DeletionInfo(delTime));
-        Column subColumn = cf.getColumn(Util.cellname("superA", "636f6c4141"));
-        assert subColumn.value().equals(hexToBytes("76616c75654141"));
+        Cell subCell = cf.getColumn(Util.cellname("superA", "636f6c4141"));
+        assert subCell.value().equals(hexToBytes("76616c75654141"));
     }
 
     @Test
@@ -141,11 +140,11 @@ public class SSTableImportTest extends SchemaLoader
         while (iter.hasNext())
             cf.addAtom(iter.next());
         assert cf.getColumn(Util.cellname("colAA")).value().equals(hexToBytes("76616c4141"));
-        assert !(cf.getColumn(Util.cellname("colAA")) instanceof DeletedColumn);
-        Column expCol = cf.getColumn(Util.cellname("colAC"));
+        assert !(cf.getColumn(Util.cellname("colAA")) instanceof DeletedCell);
+        Cell expCol = cf.getColumn(Util.cellname("colAC"));
         assert expCol.value().equals(hexToBytes("76616c4143"));
-        assert expCol instanceof ExpiringColumn;
-        assert ((ExpiringColumn) expCol).getTimeToLive() == 42 && expCol.getLocalDeletionTime() == 2000000000;
+        assert expCol instanceof ExpiringCell;
+        assert ((ExpiringCell) expCol).getTimeToLive() == 42 && expCol.getLocalDeletionTime() == 2000000000;
     }
 
     @Test
@@ -165,11 +164,11 @@ public class SSTableImportTest extends SchemaLoader
         while (iter.hasNext())
             cf.addAtom(iter.next());
         assert cf.getColumn(Util.cellname("colAA")).value().equals(hexToBytes("76616c4141"));
-        assert !(cf.getColumn(Util.cellname("colAA")) instanceof DeletedColumn);
-        Column expCol = cf.getColumn(Util.cellname("colAC"));
+        assert !(cf.getColumn(Util.cellname("colAA")) instanceof DeletedCell);
+        Cell expCol = cf.getColumn(Util.cellname("colAC"));
         assert expCol.value().equals(hexToBytes("76616c4143"));
-        assert expCol instanceof ExpiringColumn;
-        assert ((ExpiringColumn) expCol).getTimeToLive() == 42 && expCol.getLocalDeletionTime() == 2000000000;
+        assert expCol instanceof ExpiringCell;
+        assert ((ExpiringCell) expCol).getTimeToLive() == 42 && expCol.getLocalDeletionTime() == 2000000000;
     }
 
     @Test
@@ -186,8 +185,8 @@ public class SSTableImportTest extends SchemaLoader
         OnDiskAtomIterator iter = qf.getSSTableColumnIterator(reader);
         ColumnFamily cf = cloneForAdditions(iter);
         while (iter.hasNext()) cf.addAtom(iter.next());
-        Column c = cf.getColumn(Util.cellname("colAA"));
-        assert c instanceof CounterColumn: c;
-        assert ((CounterColumn) c).total() == 42;
+        Cell c = cf.getColumn(Util.cellname("colAA"));
+        assert c instanceof CounterCell : c;
+        assert ((CounterCell) c).total() == 42;
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/tools/stress/README.txt
----------------------------------------------------------------------
diff --git a/tools/stress/README.txt b/tools/stress/README.txt
index f39a8d7..9f745c1 100644
--- a/tools/stress/README.txt
+++ b/tools/stress/README.txt
@@ -30,8 +30,8 @@ Important options:
     -y or --family-type:
         Sets the ColumnFamily type.  One of 'Standard' or 'Super'.  If using super,
         you probably want to set the -u option also.
-    -c or --columns:
-        the number of columns per row, defaults to 5
+    -c or --cells:
+        the number of cells per row, defaults to 5
     -u or --supercolumns:
         use the number of supercolumns specified NOTE: you must set the -y
         option appropriately, or this option has no effect.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/tools/stress/src/org/apache/cassandra/stress/Session.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/Session.java b/tools/stress/src/org/apache/cassandra/stress/Session.java
index c26c759..8d138f5 100644
--- a/tools/stress/src/org/apache/cassandra/stress/Session.java
+++ b/tools/stress/src/org/apache/cassandra/stress/Session.java
@@ -76,9 +76,9 @@ public class Session implements Serializable
         availableOptions.addOption("F",  "num-different-keys",   true,   "Number of different keys (if < NUM-KEYS, the same key will re-used multiple times), default:NUM-KEYS");
         availableOptions.addOption("N",  "skip-keys",            true,   "Fraction of keys to skip initially, default:0");
         availableOptions.addOption("t",  "threads",              true,   "Number of threads to use, default:50");
-        availableOptions.addOption("c",  "columns",              true,   "Number of columns per key, default:5");
+        availableOptions.addOption("c",  "cells",                true,   "Number of cells per key, default:5");
         availableOptions.addOption("S",  "column-size",          true,   "Size of column values in bytes, default:34");
-        availableOptions.addOption("C",  "cardinality",          true,   "Number of unique values stored in columns, default:50");
+        availableOptions.addOption("C",  "cardinality",          true,   "Number of unique values stored in cells, default:50");
         availableOptions.addOption("d",  "nodes",                true,   "Host nodes (comma separated), default:locahost");
         availableOptions.addOption("D",  "nodesfile",            true,   "File containing host nodes (one per line)");
         availableOptions.addOption("s",  "stdev",                true,   "Standard Deviation Factor, default:0.1");
@@ -107,7 +107,7 @@ public class Session implements Serializable
         availableOptions.addOption("I",  "compression",          true,   "Specify the compression to use for sstable, default:no compression");
         availableOptions.addOption("Q",  "query-names",          true,   "Comma-separated list of column names to retrieve from each row.");
         availableOptions.addOption("Z",  "compaction-strategy",  true,   "CompactionStrategy to use.");
-        availableOptions.addOption("U",  "comparator",           true,   "Column Comparator to use. Currently supported types are: TimeUUIDType, AsciiType, UTF8Type.");
+        availableOptions.addOption("U",  "comparator",           true,   "Cell Comparator to use. Currently supported types are: TimeUUIDType, AsciiType, UTF8Type.");
         availableOptions.addOption("tf", "transport-factory",    true,   "Fully-qualified TTransportFactory class name for creating a connection. Note: For Thrift over SSL, use org.apache.cassandra.stress.SSLTransportFactory.");
         availableOptions.addOption("ns", "no-statistics",        false,  "Turn off the aggegate statistics that is normally output after completion.");
         availableOptions.addOption("ts", SSL_TRUSTSTORE,         true, "SSL: full path to truststore");

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/tools/stress/src/org/apache/cassandra/stress/operations/CqlInserter.java
----------------------------------------------------------------------
diff --git a/tools/stress/src/org/apache/cassandra/stress/operations/CqlInserter.java b/tools/stress/src/org/apache/cassandra/stress/operations/CqlInserter.java
index 3572c36..d593e57 100644
--- a/tools/stress/src/org/apache/cassandra/stress/operations/CqlInserter.java
+++ b/tools/stress/src/org/apache/cassandra/stress/operations/CqlInserter.java
@@ -91,7 +91,7 @@ public class CqlInserter extends CQLOperation
         List<String> queryParms = new ArrayList<String>();
         for (int i = 0; i < session.getColumnsPerKey(); i++)
         {
-            // Column value
+            // Cell value
             queryParms.add(getUnQuotedCqlBlob(values.get(i % values.size()).array(), session.cqlVersion.startsWith("3")));
         }
 


[2/6] Rename Column to Cell

Posted by sl...@apache.org.
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));
+    }
+}


[3/6] Rename Column to Cell

Posted by sl...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnCollectionValue.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnCollectionValue.java b/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnCollectionValue.java
index 9bf297b..1d780cd 100644
--- a/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnCollectionValue.java
+++ b/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnCollectionValue.java
@@ -62,9 +62,9 @@ public class CompositesIndexOnCollectionValue extends CompositesIndex
         return ((CollectionType)columnDef.type).valueComparator();
     }
 
-    protected ByteBuffer getIndexedValue(ByteBuffer rowKey, Column column)
+    protected ByteBuffer getIndexedValue(ByteBuffer rowKey, Cell cell)
     {
-        return column.value();
+        return cell.value();
     }
 
     protected Composite makeIndexColumnPrefix(ByteBuffer rowKey, Composite cellName)
@@ -77,7 +77,7 @@ public class CompositesIndexOnCollectionValue extends CompositesIndex
         return builder.build();
     }
 
-    public IndexedEntry decodeEntry(DecoratedKey indexedValue, Column indexEntry)
+    public IndexedEntry decodeEntry(DecoratedKey indexedValue, Cell indexEntry)
     {
         int prefixSize = columnDef.position();
         CellName name = indexEntry.name();
@@ -98,11 +98,11 @@ public class CompositesIndexOnCollectionValue extends CompositesIndex
     public boolean isStale(IndexedEntry entry, ColumnFamily data, long now)
     {
         CellName name = data.getComparator().create(entry.indexedEntryPrefix, columnDef.name, entry.indexedEntryCollectionKey);
-        Column liveColumn = data.getColumn(name);
-        if (liveColumn == null || liveColumn.isMarkedForDelete(now))
+        Cell liveCell = data.getColumn(name);
+        if (liveCell == null || liveCell.isMarkedForDelete(now))
             return true;
 
-        ByteBuffer liveValue = liveColumn.value();
+        ByteBuffer liveValue = liveCell.value();
         return ((CollectionType)columnDef.type).valueComparator().compare(entry.indexValue.key, liveValue) != 0;
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnPartitionKey.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnPartitionKey.java b/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnPartitionKey.java
index 6df1e8d..b791545 100644
--- a/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnPartitionKey.java
+++ b/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnPartitionKey.java
@@ -58,7 +58,7 @@ public class CompositesIndexOnPartitionKey extends CompositesIndex
         return new CompoundDenseCellNameType(types);
     }
 
-    protected ByteBuffer getIndexedValue(ByteBuffer rowKey, Column column)
+    protected ByteBuffer getIndexedValue(ByteBuffer rowKey, Cell cell)
     {
         CompositeType keyComparator = (CompositeType)baseCfs.metadata.getKeyValidator();
         ByteBuffer[] components = keyComparator.split(rowKey);
@@ -75,7 +75,7 @@ public class CompositesIndexOnPartitionKey extends CompositesIndex
         return builder.build();
     }
 
-    public IndexedEntry decodeEntry(DecoratedKey indexedValue, Column indexEntry)
+    public IndexedEntry decodeEntry(DecoratedKey indexedValue, Cell indexEntry)
     {
         int ckCount = baseCfs.metadata.clusteringColumns().size();
         CBuilder builder = baseCfs.getComparator().builder();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnRegular.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnRegular.java b/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnRegular.java
index 6903b77..f1b0954 100644
--- a/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnRegular.java
+++ b/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnRegular.java
@@ -57,9 +57,9 @@ public class CompositesIndexOnRegular extends CompositesIndex
         return new CompoundDenseCellNameType(types);
     }
 
-    protected ByteBuffer getIndexedValue(ByteBuffer rowKey, Column column)
+    protected ByteBuffer getIndexedValue(ByteBuffer rowKey, Cell cell)
     {
-        return column.value();
+        return cell.value();
     }
 
     protected Composite makeIndexColumnPrefix(ByteBuffer rowKey, Composite cellName)
@@ -71,7 +71,7 @@ public class CompositesIndexOnRegular extends CompositesIndex
         return builder.build();
     }
 
-    public IndexedEntry decodeEntry(DecoratedKey indexedValue, Column indexEntry)
+    public IndexedEntry decodeEntry(DecoratedKey indexedValue, Cell indexEntry)
     {
         CBuilder builder = baseCfs.getComparator().builder();
         for (int i = 0; i < columnDef.position(); i++)
@@ -90,11 +90,11 @@ public class CompositesIndexOnRegular extends CompositesIndex
     public boolean isStale(IndexedEntry entry, ColumnFamily data, long now)
     {
         CellName name = data.getComparator().create(entry.indexedEntryPrefix, columnDef.name);
-        Column liveColumn = data.getColumn(name);
-        if (liveColumn == null || liveColumn.isMarkedForDelete(now))
+        Cell liveCell = data.getColumn(name);
+        if (liveCell == null || liveCell.isMarkedForDelete(now))
             return true;
 
-        ByteBuffer liveValue = liveColumn.value();
+        ByteBuffer liveValue = liveCell.value();
         return columnDef.type.compare(entry.indexValue.key, liveValue) != 0;
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/db/index/composites/CompositesSearcher.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/index/composites/CompositesSearcher.java b/src/java/org/apache/cassandra/db/index/composites/CompositesSearcher.java
index 97602af..7dba9b0 100644
--- a/src/java/org/apache/cassandra/db/index/composites/CompositesSearcher.java
+++ b/src/java/org/apache/cassandra/db/index/composites/CompositesSearcher.java
@@ -104,7 +104,7 @@ public class CompositesSearcher extends SecondaryIndexSearcher
         return new ColumnFamilyStore.AbstractScanIterator()
         {
             private Composite lastSeenPrefix = startPrefix;
-            private Deque<Column> indexColumns;
+            private Deque<Cell> indexCells;
             private int columnsRead = Integer.MAX_VALUE;
             private int limit = filter.currentLimit();
             private int columnsCount = 0;
@@ -145,7 +145,7 @@ public class CompositesSearcher extends SecondaryIndexSearcher
                     if (columnsCount >= limit)
                         return makeReturn(currentKey, data);
 
-                    if (indexColumns == null || indexColumns.isEmpty())
+                    if (indexCells == null || indexCells.isEmpty())
                     {
                         if (columnsRead < rowsPerQuery)
                         {
@@ -168,31 +168,31 @@ public class CompositesSearcher extends SecondaryIndexSearcher
                         if (indexRow == null || indexRow.getColumnCount() == 0)
                             return makeReturn(currentKey, data);
 
-                        Collection<Column> sortedColumns = indexRow.getSortedColumns();
-                        columnsRead = sortedColumns.size();
-                        indexColumns = new ArrayDeque<>(sortedColumns);
-                        Column firstColumn = sortedColumns.iterator().next();
+                        Collection<Cell> sortedCells = indexRow.getSortedColumns();
+                        columnsRead = sortedCells.size();
+                        indexCells = new ArrayDeque<>(sortedCells);
+                        Cell firstCell = sortedCells.iterator().next();
 
                         // Paging is racy, so it is possible the first column of a page is not the last seen one.
-                        if (lastSeenPrefix != startPrefix && lastSeenPrefix.equals(firstColumn.name()))
+                        if (lastSeenPrefix != startPrefix && lastSeenPrefix.equals(firstCell.name()))
                         {
                             // skip the row we already saw w/ the last page of results
-                            indexColumns.poll();
-                            logger.trace("Skipping {}", indexComparator.getString(firstColumn.name()));
+                            indexCells.poll();
+                            logger.trace("Skipping {}", indexComparator.getString(firstCell.name()));
                         }
                     }
 
-                    while (!indexColumns.isEmpty() && columnsCount <= limit)
+                    while (!indexCells.isEmpty() && columnsCount <= limit)
                     {
-                        Column column = indexColumns.poll();
-                        lastSeenPrefix = column.name();
-                        if (column.isMarkedForDelete(filter.timestamp))
+                        Cell cell = indexCells.poll();
+                        lastSeenPrefix = cell.name();
+                        if (cell.isMarkedForDelete(filter.timestamp))
                         {
-                            logger.trace("skipping {}", column.name());
+                            logger.trace("skipping {}", cell.name());
                             continue;
                         }
 
-                        CompositesIndex.IndexedEntry entry = index.decodeEntry(indexKey, column);
+                        CompositesIndex.IndexedEntry entry = index.decodeEntry(indexKey, cell);
                         DecoratedKey dk = baseCfs.partitioner.decorateKey(entry.indexedKey);
 
                         // Are we done for this row?
@@ -206,7 +206,7 @@ public class CompositesSearcher extends SecondaryIndexSearcher
                             currentKey = dk;
 
                             // We're done with the previous row, return it if it had data, continue otherwise
-                            indexColumns.addFirst(column);
+                            indexCells.addFirst(cell);
                             if (data == null)
                                 continue;
                             else
@@ -229,7 +229,7 @@ public class CompositesSearcher extends SecondaryIndexSearcher
                             }
                         }
 
-                        // Check if this entry cannot be a hit due to the original column filter
+                        // Check if this entry cannot be a hit due to the original cell filter
                         Composite start = entry.indexedEntryPrefix;
                         if (!filter.columnFilter(dk.key).maySelectPrefix(baseComparator, start))
                             continue;
@@ -244,7 +244,7 @@ public class CompositesSearcher extends SecondaryIndexSearcher
                         else
                             previousPrefix = null;
 
-                        logger.trace("Adding index hit to current row for {}", indexComparator.getString(column.name()));
+                        logger.trace("Adding index hit to current row for {}", indexComparator.getString(cell.name()));
 
                         // We always query the whole CQL3 row. In the case where the original filter was a name filter this might be
                         // slightly wasteful, but this probably doesn't matter in practice and it simplify things.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/db/index/keys/KeysIndex.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/index/keys/KeysIndex.java b/src/java/org/apache/cassandra/db/index/keys/KeysIndex.java
index ee56c36..e532a53 100644
--- a/src/java/org/apache/cassandra/db/index/keys/KeysIndex.java
+++ b/src/java/org/apache/cassandra/db/index/keys/KeysIndex.java
@@ -20,10 +20,10 @@ package org.apache.cassandra.db.index.keys;
 import java.nio.ByteBuffer;
 import java.util.Set;
 
+import org.apache.cassandra.db.Cell;
 import org.apache.cassandra.db.composites.CellName;
 import org.apache.cassandra.db.composites.CellNames;
 import org.apache.cassandra.db.ColumnFamily;
-import org.apache.cassandra.db.Column;
 import org.apache.cassandra.db.index.AbstractSimplePerColumnSecondaryIndex;
 import org.apache.cassandra.db.index.SecondaryIndexSearcher;
 import org.apache.cassandra.db.marshal.AbstractType;
@@ -35,12 +35,12 @@ import org.apache.cassandra.exceptions.ConfigurationException;
  */
 public class KeysIndex extends AbstractSimplePerColumnSecondaryIndex
 {
-    protected ByteBuffer getIndexedValue(ByteBuffer rowKey, Column column)
+    protected ByteBuffer getIndexedValue(ByteBuffer rowKey, Cell cell)
     {
-        return column.value();
+        return cell.value();
     }
 
-    protected CellName makeIndexColumnName(ByteBuffer rowKey, Column column)
+    protected CellName makeIndexColumnName(ByteBuffer rowKey, Cell cell)
     {
         return CellNames.simpleDense(rowKey);
     }
@@ -52,11 +52,11 @@ public class KeysIndex extends AbstractSimplePerColumnSecondaryIndex
 
     public boolean isIndexEntryStale(ByteBuffer indexedValue, ColumnFamily data, long now)
     {
-        Column liveColumn = data.getColumn(data.getComparator().makeCellName(columnDef.name.bytes));
-        if (liveColumn == null || liveColumn.isMarkedForDelete(now))
+        Cell liveCell = data.getColumn(data.getComparator().makeCellName(columnDef.name.bytes));
+        if (liveCell == null || liveCell.isMarkedForDelete(now))
             return true;
 
-        ByteBuffer liveValue = liveColumn.value();
+        ByteBuffer liveValue = liveCell.value();
         return columnDef.type.compare(indexedValue, liveValue) != 0;
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/db/index/keys/KeysSearcher.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/index/keys/KeysSearcher.java b/src/java/org/apache/cassandra/db/index/keys/KeysSearcher.java
index 0101a0b..932d446 100644
--- a/src/java/org/apache/cassandra/db/index/keys/KeysSearcher.java
+++ b/src/java/org/apache/cassandra/db/index/keys/KeysSearcher.java
@@ -87,7 +87,7 @@ public class KeysSearcher extends SecondaryIndexSearcher
         return new ColumnFamilyStore.AbstractScanIterator()
         {
             private Composite lastSeenKey = startKey;
-            private Iterator<Column> indexColumns;
+            private Iterator<Cell> indexColumns;
             private int columnsRead = Integer.MAX_VALUE;
 
             protected Row computeNext()
@@ -124,19 +124,19 @@ public class KeysSearcher extends SecondaryIndexSearcher
                             return endOfData();
                         }
 
-                        Collection<Column> sortedColumns = indexRow.getSortedColumns();
-                        columnsRead = sortedColumns.size();
-                        indexColumns = sortedColumns.iterator();
-                        Column firstColumn = sortedColumns.iterator().next();
+                        Collection<Cell> sortedCells = indexRow.getSortedColumns();
+                        columnsRead = sortedCells.size();
+                        indexColumns = sortedCells.iterator();
+                        Cell firstCell = sortedCells.iterator().next();
 
                         // Paging is racy, so it is possible the first column of a page is not the last seen one.
-                        if (lastSeenKey != startKey && lastSeenKey.equals(firstColumn.name()))
+                        if (lastSeenKey != startKey && lastSeenKey.equals(firstCell.name()))
                         {
                             // skip the row we already saw w/ the last page of results
                             indexColumns.next();
-                            logger.trace("Skipping {}", baseCfs.metadata.getKeyValidator().getString(firstColumn.name().toByteBuffer()));
+                            logger.trace("Skipping {}", baseCfs.metadata.getKeyValidator().getString(firstCell.name().toByteBuffer()));
                         }
-                        else if (range instanceof Range && indexColumns.hasNext() && firstColumn.name().equals(startKey))
+                        else if (range instanceof Range && indexColumns.hasNext() && firstCell.name().equals(startKey))
                         {
                             // skip key excluded by range
                             indexColumns.next();
@@ -146,11 +146,11 @@ public class KeysSearcher extends SecondaryIndexSearcher
 
                     while (indexColumns.hasNext())
                     {
-                        Column column = indexColumns.next();
-                        lastSeenKey = column.name();
-                        if (column.isMarkedForDelete(filter.timestamp))
+                        Cell cell = indexColumns.next();
+                        lastSeenKey = cell.name();
+                        if (cell.isMarkedForDelete(filter.timestamp))
                         {
-                            logger.trace("skipping {}", column.name());
+                            logger.trace("skipping {}", cell.name());
                             continue;
                         }
 
@@ -168,7 +168,7 @@ public class KeysSearcher extends SecondaryIndexSearcher
 
                         logger.trace("Returning index hit for {}", dk);
                         ColumnFamily data = baseCfs.getColumnFamily(new QueryFilter(dk, baseCfs.name, filter.columnFilter(lastSeenKey.toByteBuffer()), filter.timestamp));
-                        // While the column family we'll get in the end should contains the primary clause column, the initialFilter may not have found it and can thus be null
+                        // While the column family we'll get in the end should contains the primary clause cell, the initialFilter may not have found it and can thus be null
                         if (data == null)
                             data = TreeMapBackedSortedColumns.factory.create(baseCfs.metadata);
 
@@ -185,8 +185,8 @@ public class KeysSearcher extends SecondaryIndexSearcher
                         if (((KeysIndex)index).isIndexEntryStale(indexKey.key, data, filter.timestamp))
                         {
                             // delete the index entry w/ its own timestamp
-                            Column dummyColumn = new Column(primaryColumn, indexKey.key, column.timestamp());
-                            ((PerColumnSecondaryIndex)index).delete(dk.key, dummyColumn);
+                            Cell dummyCell = new Cell(primaryColumn, indexKey.key, cell.timestamp());
+                            ((PerColumnSecondaryIndex)index).delete(dk.key, dummyCell);
                             continue;
                         }
                         return new Row(dk, data);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/db/marshal/AbstractCommutativeType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/AbstractCommutativeType.java b/src/java/org/apache/cassandra/db/marshal/AbstractCommutativeType.java
index a7162ae..4b26d5d 100644
--- a/src/java/org/apache/cassandra/db/marshal/AbstractCommutativeType.java
+++ b/src/java/org/apache/cassandra/db/marshal/AbstractCommutativeType.java
@@ -18,8 +18,9 @@
 package org.apache.cassandra.db.marshal;
 
 import java.nio.ByteBuffer;
+
+import org.apache.cassandra.db.Cell;
 import org.apache.cassandra.db.composites.CellName;
-import org.apache.cassandra.db.Column;
 import org.apache.cassandra.db.context.CounterContext;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
@@ -45,5 +46,5 @@ public abstract class AbstractCommutativeType extends AbstractType<Long>
     /**
      * create commutative column
      */
-    public abstract Column createColumn(CellName name, ByteBuffer value, long timestamp);
+    public abstract Cell createColumn(CellName name, ByteBuffer value, long timestamp);
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/db/marshal/CollectionType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/CollectionType.java b/src/java/org/apache/cassandra/db/marshal/CollectionType.java
index 0f3f564..b9816a6 100644
--- a/src/java/org/apache/cassandra/db/marshal/CollectionType.java
+++ b/src/java/org/apache/cassandra/db/marshal/CollectionType.java
@@ -24,10 +24,9 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.cql3.CQL3Type;
-import org.apache.cassandra.db.Column;
+import org.apache.cassandra.db.Cell;
 import org.apache.cassandra.serializers.MarshalException;
 import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.Pair;
 
 /**
  * The abstract validator that is the base for maps, sets and lists.
@@ -58,7 +57,7 @@ public abstract class CollectionType<T> extends AbstractType<T>
 
     protected abstract void appendToStringBuilder(StringBuilder sb);
 
-    public abstract ByteBuffer serialize(List<Column> columns);
+    public abstract ByteBuffer serialize(List<Cell> cells);
 
     @Override
     public String toString()
@@ -113,14 +112,14 @@ public abstract class CollectionType<T> extends AbstractType<T>
         return (ByteBuffer)result.flip();
     }
 
-    protected List<Column> enforceLimit(List<Column> columns)
+    protected List<Cell> enforceLimit(List<Cell> cells)
     {
-        if (columns.size() <= MAX_ELEMENTS)
-            return columns;
+        if (cells.size() <= MAX_ELEMENTS)
+            return cells;
 
         logger.error("Detected collection with {} elements, more than the {} limit. Only the first {} elements will be returned to the client. "
-                   + "Please see http://cassandra.apache.org/doc/cql3/CQL.html#collections for more details.", columns.size(), MAX_ELEMENTS, MAX_ELEMENTS);
-        return columns.subList(0, MAX_ELEMENTS);
+                   + "Please see http://cassandra.apache.org/doc/cql3/CQL.html#collections for more details.", cells.size(), MAX_ELEMENTS, MAX_ELEMENTS);
+        return cells.subList(0, MAX_ELEMENTS);
     }
 
     public static ByteBuffer pack(List<ByteBuffer> buffers, int elements)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/db/marshal/CounterColumnType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/CounterColumnType.java b/src/java/org/apache/cassandra/db/marshal/CounterColumnType.java
index 37cd59b..e1a886d 100644
--- a/src/java/org/apache/cassandra/db/marshal/CounterColumnType.java
+++ b/src/java/org/apache/cassandra/db/marshal/CounterColumnType.java
@@ -48,9 +48,9 @@ public class CounterColumnType extends AbstractCommutativeType
     /**
      * create commutative column
      */
-    public Column createColumn(CellName name, ByteBuffer value, long timestamp)
+    public Cell createColumn(CellName name, ByteBuffer value, long timestamp)
     {
-        return new CounterUpdateColumn(name, value, timestamp);
+        return new CounterUpdateCell(name, value, timestamp);
     }
 
     public ByteBuffer fromString(String source)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/db/marshal/ListType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/ListType.java b/src/java/org/apache/cassandra/db/marshal/ListType.java
index 58ba6f1..eabda0b 100644
--- a/src/java/org/apache/cassandra/db/marshal/ListType.java
+++ b/src/java/org/apache/cassandra/db/marshal/ListType.java
@@ -20,12 +20,11 @@ package org.apache.cassandra.db.marshal;
 import java.nio.ByteBuffer;
 import java.util.*;
 
-import org.apache.cassandra.db.Column;
+import org.apache.cassandra.db.Cell;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.SyntaxException;
 import org.apache.cassandra.serializers.TypeSerializer;
 import org.apache.cassandra.serializers.ListSerializer;
-import org.apache.cassandra.utils.Pair;
 
 public class ListType<T> extends CollectionType<List<T>>
 {
@@ -82,17 +81,17 @@ public class ListType<T> extends CollectionType<List<T>>
         sb.append(getClass().getName()).append(TypeParser.stringifyTypeParameters(Collections.<AbstractType<?>>singletonList(elements)));
     }
 
-    public ByteBuffer serialize(List<Column> columns)
+    public ByteBuffer serialize(List<Cell> cells)
     {
-        columns = enforceLimit(columns);
+        cells = enforceLimit(cells);
 
-        List<ByteBuffer> bbs = new ArrayList<ByteBuffer>(columns.size());
+        List<ByteBuffer> bbs = new ArrayList<ByteBuffer>(cells.size());
         int size = 0;
-        for (Column c : columns)
+        for (Cell c : cells)
         {
             bbs.add(c.value());
             size += 2 + c.value().remaining();
         }
-        return pack(bbs, columns.size(), size);
+        return pack(bbs, cells.size(), size);
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/db/marshal/MapType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/MapType.java b/src/java/org/apache/cassandra/db/marshal/MapType.java
index 17bd7a7..4d834e7 100644
--- a/src/java/org/apache/cassandra/db/marshal/MapType.java
+++ b/src/java/org/apache/cassandra/db/marshal/MapType.java
@@ -20,7 +20,7 @@ package org.apache.cassandra.db.marshal;
 import java.nio.ByteBuffer;
 import java.util.*;
 
-import org.apache.cassandra.db.Column;
+import org.apache.cassandra.db.Cell;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.SyntaxException;
 import org.apache.cassandra.serializers.TypeSerializer;
@@ -89,13 +89,13 @@ public class MapType<K, V> extends CollectionType<Map<K, V>>
     /**
      * Creates the same output than serialize, but from the internal representation.
      */
-    public ByteBuffer serialize(List<Column> columns)
+    public ByteBuffer serialize(List<Cell> cells)
     {
-        columns = enforceLimit(columns);
+        cells = enforceLimit(cells);
 
-        List<ByteBuffer> bbs = new ArrayList<ByteBuffer>(2 * columns.size());
+        List<ByteBuffer> bbs = new ArrayList<ByteBuffer>(2 * cells.size());
         int size = 0;
-        for (Column c : columns)
+        for (Cell c : cells)
         {
             ByteBuffer key = c.name().collectionElement();
             ByteBuffer value = c.value();
@@ -103,6 +103,6 @@ public class MapType<K, V> extends CollectionType<Map<K, V>>
             bbs.add(value);
             size += 4 + key.remaining() + value.remaining();
         }
-        return pack(bbs, columns.size(), size);
+        return pack(bbs, cells.size(), size);
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/db/marshal/SetType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/SetType.java b/src/java/org/apache/cassandra/db/marshal/SetType.java
index 9e45f8f..292b832 100644
--- a/src/java/org/apache/cassandra/db/marshal/SetType.java
+++ b/src/java/org/apache/cassandra/db/marshal/SetType.java
@@ -20,12 +20,11 @@ package org.apache.cassandra.db.marshal;
 import java.nio.ByteBuffer;
 import java.util.*;
 
-import org.apache.cassandra.db.Column;
+import org.apache.cassandra.db.Cell;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.SyntaxException;
 import org.apache.cassandra.serializers.TypeSerializer;
 import org.apache.cassandra.serializers.SetSerializer;
-import org.apache.cassandra.utils.Pair;
 
 public class SetType<T> extends CollectionType<Set<T>>
 {
@@ -82,18 +81,18 @@ public class SetType<T> extends CollectionType<Set<T>>
         sb.append(getClass().getName()).append(TypeParser.stringifyTypeParameters(Collections.<AbstractType<?>>singletonList(elements)));
     }
 
-    public ByteBuffer serialize(List<Column> columns)
+    public ByteBuffer serialize(List<Cell> cells)
     {
-        columns = enforceLimit(columns);
+        cells = enforceLimit(cells);
 
-        List<ByteBuffer> bbs = new ArrayList<ByteBuffer>(columns.size());
+        List<ByteBuffer> bbs = new ArrayList<ByteBuffer>(cells.size());
         int size = 0;
-        for (Column c : columns)
+        for (Cell c : cells)
         {
             ByteBuffer key = c.name().collectionElement();
             bbs.add(key);
             size += 2 + key.remaining();
         }
-        return pack(bbs, columns.size(), size);
+        return pack(bbs, cells.size(), size);
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/hadoop/ColumnFamilyInputFormat.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/ColumnFamilyInputFormat.java b/src/java/org/apache/cassandra/hadoop/ColumnFamilyInputFormat.java
index 4327aa9..fbd5bf2 100644
--- a/src/java/org/apache/cassandra/hadoop/ColumnFamilyInputFormat.java
+++ b/src/java/org/apache/cassandra/hadoop/ColumnFamilyInputFormat.java
@@ -21,8 +21,8 @@ import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.*;
 
+import org.apache.cassandra.db.Cell;
 import org.apache.cassandra.db.composites.CellName;
-import org.apache.cassandra.db.Column;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.Reporter;
@@ -45,15 +45,15 @@ import org.apache.hadoop.mapreduce.*;
  *
  * The default split size is 64k rows.
  */
-public class ColumnFamilyInputFormat extends AbstractColumnFamilyInputFormat<ByteBuffer, SortedMap<CellName, Column>>
+public class ColumnFamilyInputFormat extends AbstractColumnFamilyInputFormat<ByteBuffer, SortedMap<CellName, Cell>>
 {
     
-    public RecordReader<ByteBuffer, SortedMap<CellName, Column>> createRecordReader(InputSplit inputSplit, TaskAttemptContext taskAttemptContext) throws IOException, InterruptedException
+    public RecordReader<ByteBuffer, SortedMap<CellName, Cell>> createRecordReader(InputSplit inputSplit, TaskAttemptContext taskAttemptContext) throws IOException, InterruptedException
     {
         return new ColumnFamilyRecordReader();
     }
 
-    public org.apache.hadoop.mapred.RecordReader<ByteBuffer, SortedMap<CellName, Column>> getRecordReader(org.apache.hadoop.mapred.InputSplit split, JobConf jobConf, final Reporter reporter) throws IOException
+    public org.apache.hadoop.mapred.RecordReader<ByteBuffer, SortedMap<CellName, Cell>> getRecordReader(org.apache.hadoop.mapred.InputSplit split, JobConf jobConf, final Reporter reporter) throws IOException
     {
         TaskAttemptContext tac = new TaskAttemptContext(jobConf, TaskAttemptID.forName(jobConf.get(MAPRED_TASK_ID)))
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordReader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordReader.java b/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordReader.java
index 7bda3fb..be18f5f 100644
--- a/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordReader.java
+++ b/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordReader.java
@@ -24,11 +24,11 @@ import java.nio.ByteBuffer;
 import java.util.*;
 
 import com.google.common.collect.*;
+import org.apache.cassandra.db.Cell;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.db.Column;
 import org.apache.cassandra.db.composites.*;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.marshal.TypeParser;
@@ -44,8 +44,8 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.thrift.TException;
 import org.apache.thrift.transport.TTransport;
 
-public class ColumnFamilyRecordReader extends RecordReader<ByteBuffer, SortedMap<CellName, Column>>
-    implements org.apache.hadoop.mapred.RecordReader<ByteBuffer, SortedMap<CellName, Column>>
+public class ColumnFamilyRecordReader extends RecordReader<ByteBuffer, SortedMap<CellName, Cell>>
+    implements org.apache.hadoop.mapred.RecordReader<ByteBuffer, SortedMap<CellName, Cell>>
 {
     private static final Logger logger = LoggerFactory.getLogger(ColumnFamilyRecordReader.class);
 
@@ -53,7 +53,7 @@ public class ColumnFamilyRecordReader extends RecordReader<ByteBuffer, SortedMap
 
     private ColumnFamilySplit split;
     private RowIterator iter;
-    private Pair<ByteBuffer, SortedMap<CellName, Column>> currentRow;
+    private Pair<ByteBuffer, SortedMap<CellName, Cell>> currentRow;
     private SlicePredicate predicate;
     private boolean isEmptyPredicate;
     private int totalRowCount; // total number of rows to fetch
@@ -92,7 +92,7 @@ public class ColumnFamilyRecordReader extends RecordReader<ByteBuffer, SortedMap
         return currentRow.left;
     }
 
-    public SortedMap<CellName, Column> getCurrentValue()
+    public SortedMap<CellName, Cell> getCurrentValue()
     {
         return currentRow.right;
     }
@@ -210,7 +210,7 @@ public class ColumnFamilyRecordReader extends RecordReader<ByteBuffer, SortedMap
         return split.getLocations()[0];
     }
 
-    private abstract class RowIterator extends AbstractIterator<Pair<ByteBuffer, SortedMap<CellName, Column>>>
+    private abstract class RowIterator extends AbstractIterator<Pair<ByteBuffer, SortedMap<CellName, Cell>>>
     {
         protected List<KeySlice> rows;
         protected int totalRead = 0;
@@ -279,50 +279,50 @@ public class ColumnFamilyRecordReader extends RecordReader<ByteBuffer, SortedMap
             return totalRead;
         }
 
-        protected List<Column> unthriftify(ColumnOrSuperColumn cosc)
+        protected List<Cell> unthriftify(ColumnOrSuperColumn cosc)
         {
             if (cosc.counter_column != null)
-                return Collections.<Column>singletonList(unthriftifyCounter(cosc.counter_column));
+                return Collections.<Cell>singletonList(unthriftifyCounter(cosc.counter_column));
             if (cosc.counter_super_column != null)
                 return unthriftifySuperCounter(cosc.counter_super_column);
             if (cosc.super_column != null)
                 return unthriftifySuper(cosc.super_column);
             assert cosc.column != null;
-            return Collections.<Column>singletonList(unthriftifySimple(cosc.column));
+            return Collections.<Cell>singletonList(unthriftifySimple(cosc.column));
         }
 
-        private List<Column> unthriftifySuper(SuperColumn super_column)
+        private List<Cell> unthriftifySuper(SuperColumn super_column)
         {
-            List<Column> columns = new ArrayList<Column>(super_column.columns.size());
+            List<Cell> cells = new ArrayList<Cell>(super_column.columns.size());
             for (org.apache.cassandra.thrift.Column column : super_column.columns)
             {
-                Column c = unthriftifySimple(column);
-                columns.add(c.withUpdatedName(comparator.makeCellName(super_column.name, c.name().toByteBuffer())));
+                Cell c = unthriftifySimple(column);
+                cells.add(c.withUpdatedName(comparator.makeCellName(super_column.name, c.name().toByteBuffer())));
             }
-            return columns;
+            return cells;
         }
 
-        protected Column unthriftifySimple(org.apache.cassandra.thrift.Column column)
+        protected Cell unthriftifySimple(org.apache.cassandra.thrift.Column column)
         {
-            return new Column(comparator.cellFromByteBuffer(column.name), column.value, column.timestamp);
+            return new Cell(comparator.cellFromByteBuffer(column.name), column.value, column.timestamp);
         }
 
-        private Column unthriftifyCounter(CounterColumn column)
+        private Cell unthriftifyCounter(CounterColumn column)
         {
             //CounterColumns read the counterID from the System keyspace, so need the StorageService running and access
-            //to cassandra.yaml. To avoid a Hadoop needing access to yaml return a regular Column.
-            return new Column(comparator.cellFromByteBuffer(column.name), ByteBufferUtil.bytes(column.value), 0);
+            //to cassandra.yaml. To avoid a Hadoop needing access to yaml return a regular Cell.
+            return new Cell(comparator.cellFromByteBuffer(column.name), ByteBufferUtil.bytes(column.value), 0);
         }
 
-        private List<Column> unthriftifySuperCounter(CounterSuperColumn super_column)
+        private List<Cell> unthriftifySuperCounter(CounterSuperColumn super_column)
         {
-            List<Column> columns = new ArrayList<Column>(super_column.columns.size());
+            List<Cell> cells = new ArrayList<Cell>(super_column.columns.size());
             for (CounterColumn column : super_column.columns)
             {
-                Column c = unthriftifyCounter(column);
-                columns.add(c.withUpdatedName(comparator.makeCellName(super_column.name, c.name().toByteBuffer())));
+                Cell c = unthriftifyCounter(column);
+                cells.add(c.withUpdatedName(comparator.makeCellName(super_column.name, c.name().toByteBuffer())));
             }
-            return columns;
+            return cells;
         }
     }
 
@@ -401,7 +401,7 @@ public class ColumnFamilyRecordReader extends RecordReader<ByteBuffer, SortedMap
             }
         }
 
-        protected Pair<ByteBuffer, SortedMap<CellName, Column>> computeNext()
+        protected Pair<ByteBuffer, SortedMap<CellName, Cell>> computeNext()
         {
             maybeInit();
             if (rows == null)
@@ -409,12 +409,12 @@ public class ColumnFamilyRecordReader extends RecordReader<ByteBuffer, SortedMap
 
             totalRead++;
             KeySlice ks = rows.get(i++);
-            SortedMap<CellName, Column> map = new TreeMap<CellName, Column>(comparator);
+            SortedMap<CellName, Cell> map = new TreeMap<CellName, Cell>(comparator);
             for (ColumnOrSuperColumn cosc : ks.columns)
             {
-                List<Column> columns = unthriftify(cosc);
-                for (Column column : columns)
-                    map.put(column.name(), column);
+                List<Cell> cells = unthriftify(cosc);
+                for (Cell cell : cells)
+                    map.put(cell.name(), cell);
             }
             return Pair.create(ks.key, map);
         }
@@ -422,7 +422,7 @@ public class ColumnFamilyRecordReader extends RecordReader<ByteBuffer, SortedMap
 
     private class WideRowIterator extends RowIterator
     {
-        private PeekingIterator<Pair<ByteBuffer, SortedMap<CellName, Column>>> wideColumns;
+        private PeekingIterator<Pair<ByteBuffer, SortedMap<CellName, Cell>>> wideColumns;
         private Composite lastColumn = Composites.EMPTY;
         private ByteBuffer lastCountedKey = ByteBufferUtil.EMPTY_BYTE_BUFFER;
 
@@ -471,13 +471,13 @@ public class ColumnFamilyRecordReader extends RecordReader<ByteBuffer, SortedMap
             }
         }
 
-        protected Pair<ByteBuffer, SortedMap<CellName, Column>> computeNext()
+        protected Pair<ByteBuffer, SortedMap<CellName, Cell>> computeNext()
         {
             maybeInit();
             if (rows == null)
                 return endOfData();
 
-            Pair<ByteBuffer, SortedMap<CellName, Column>> next = wideColumns.next();
+            Pair<ByteBuffer, SortedMap<CellName, Cell>> next = wideColumns.next();
             lastColumn = next.right.values().iterator().next().name();
 
             maybeIncreaseRowCounter(next);
@@ -489,7 +489,7 @@ public class ColumnFamilyRecordReader extends RecordReader<ByteBuffer, SortedMap
          * Increases the row counter only if we really moved to the next row.
          * @param next just fetched row slice
          */
-        private void maybeIncreaseRowCounter(Pair<ByteBuffer, SortedMap<CellName, Column>> next)
+        private void maybeIncreaseRowCounter(Pair<ByteBuffer, SortedMap<CellName, Cell>> next)
         {
             ByteBuffer currentKey = next.left;
             if (!currentKey.equals(lastCountedKey))
@@ -499,7 +499,7 @@ public class ColumnFamilyRecordReader extends RecordReader<ByteBuffer, SortedMap
             }
         }
 
-        private class WideColumnIterator extends AbstractIterator<Pair<ByteBuffer, SortedMap<CellName, Column>>>
+        private class WideColumnIterator extends AbstractIterator<Pair<ByteBuffer, SortedMap<CellName, Cell>>>
         {
             private final Iterator<KeySlice> rows;
             private Iterator<ColumnOrSuperColumn> columns;
@@ -520,7 +520,7 @@ public class ColumnFamilyRecordReader extends RecordReader<ByteBuffer, SortedMap
                 columns = currentRow.columns.iterator();
             }
 
-            protected Pair<ByteBuffer, SortedMap<CellName, Column>> computeNext()
+            protected Pair<ByteBuffer, SortedMap<CellName, Cell>> computeNext()
             {
                 CellNameType cellType = subComparator == null
                                       ? comparator
@@ -530,20 +530,20 @@ public class ColumnFamilyRecordReader extends RecordReader<ByteBuffer, SortedMap
                     if (columns.hasNext())
                     {
                         ColumnOrSuperColumn cosc = columns.next();
-                        SortedMap<CellName, Column> map;
-                        List<Column> columns = unthriftify(cosc);
-                        if (columns.size() == 1)
+                        SortedMap<CellName, Cell> map;
+                        List<Cell> cells = unthriftify(cosc);
+                        if (cells.size() == 1)
                         {
-                            map = ImmutableSortedMap.of(columns.get(0).name(), columns.get(0));
+                            map = ImmutableSortedMap.of(cells.get(0).name(), cells.get(0));
                         }
                         else
                         {
                             assert isSuper;
-                            map = new TreeMap<CellName, Column>(cellType);
-                            for (Column column : columns)
-                                map.put(column.name(), column);
+                            map = new TreeMap<CellName, Cell>(cellType);
+                            for (Cell cell : cells)
+                                map.put(cell.name(), cell);
                         }
-                        return Pair.<ByteBuffer, SortedMap<CellName, Column>>create(currentRow.key, map);
+                        return Pair.<ByteBuffer, SortedMap<CellName, Cell>>create(currentRow.key, map);
                     }
 
                     if (!rows.hasNext())
@@ -560,7 +560,7 @@ public class ColumnFamilyRecordReader extends RecordReader<ByteBuffer, SortedMap
     // to the old. Thus, expect a small performance hit.
     // And obviously this wouldn't work for wide rows. But since ColumnFamilyInputFormat
     // and ColumnFamilyRecordReader don't support them, it should be fine for now.
-    public boolean next(ByteBuffer key, SortedMap<CellName, Column> value) throws IOException
+    public boolean next(ByteBuffer key, SortedMap<CellName, Cell> value) throws IOException
     {
         if (this.nextKeyValue())
         {
@@ -581,9 +581,9 @@ public class ColumnFamilyRecordReader extends RecordReader<ByteBuffer, SortedMap
         return ByteBuffer.wrap(new byte[this.keyBufferSize]);
     }
 
-    public SortedMap<CellName, Column> createValue()
+    public SortedMap<CellName, Cell> createValue()
     {
-        return new TreeMap<CellName, Column>();
+        return new TreeMap<CellName, Cell>();
     }
 
     public long getPos() throws IOException

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/hadoop/pig/AbstractCassandraStorage.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/pig/AbstractCassandraStorage.java b/src/java/org/apache/cassandra/hadoop/pig/AbstractCassandraStorage.java
index e5b8bb1..25ffc29 100644
--- a/src/java/org/apache/cassandra/hadoop/pig/AbstractCassandraStorage.java
+++ b/src/java/org/apache/cassandra/hadoop/pig/AbstractCassandraStorage.java
@@ -26,13 +26,12 @@ import java.nio.charset.CharacterCodingException;
 import java.util.*;
 
 
+import org.apache.cassandra.db.Cell;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.SyntaxException;
 import org.apache.cassandra.auth.IAuthenticator;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.cql3.ColumnIdentifier;
-import org.apache.cassandra.db.Column;
 import org.apache.cassandra.db.marshal.*;
 import org.apache.cassandra.db.marshal.AbstractCompositeType.CompositeComponent;
 import org.apache.cassandra.hadoop.*;
@@ -118,7 +117,7 @@ public abstract class AbstractCassandraStorage extends LoadFunc implements Store
     }
 
     /** convert a column to a tuple */
-    protected Tuple columnToTuple(Column col, CfInfo cfInfo, AbstractType comparator) throws IOException
+    protected Tuple columnToTuple(Cell col, CfInfo cfInfo, AbstractType comparator) throws IOException
     {
         CfDef cfDef = cfInfo.cfDef;
         Tuple pair = TupleFactory.getInstance().newTuple(2);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/hadoop/pig/CassandraStorage.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/pig/CassandraStorage.java b/src/java/org/apache/cassandra/hadoop/pig/CassandraStorage.java
index aeec4a6..ae18d20 100644
--- a/src/java/org/apache/cassandra/hadoop/pig/CassandraStorage.java
+++ b/src/java/org/apache/cassandra/hadoop/pig/CassandraStorage.java
@@ -22,10 +22,10 @@ import java.nio.ByteBuffer;
 import java.nio.charset.CharacterCodingException;
 import java.util.*;
 
+import org.apache.cassandra.db.Cell;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.db.Column;
 import org.apache.cassandra.db.marshal.*;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.InvalidRequestException;
@@ -65,7 +65,7 @@ public class CassandraStorage extends AbstractCassandraStorage
     private boolean slice_reverse = false;
     private boolean allow_deletes = false;
 
-    private RecordReader<ByteBuffer, Map<ByteBuffer, Column>> reader;
+    private RecordReader<ByteBuffer, Map<ByteBuffer, Cell>> reader;
     private RecordWriter<ByteBuffer, List<Mutation>> writer;
 
     private boolean widerows = false;
@@ -73,7 +73,7 @@ public class CassandraStorage extends AbstractCassandraStorage
     
     // wide row hacks
     private ByteBuffer lastKey;
-    private Map<ByteBuffer, Column> lastRow;
+    private Map<ByteBuffer, Cell> lastRow;
     private boolean hasNext = true;
 
     public CassandraStorage()
@@ -125,7 +125,7 @@ public class CassandraStorage extends AbstractCassandraStorage
                             key = (ByteBuffer)reader.getCurrentKey();
                             tuple = keyToTuple(key, cfDef, parseType(cfDef.getKey_validation_class()));
                         }
-                        for (Map.Entry<ByteBuffer, Column> entry : lastRow.entrySet())
+                        for (Map.Entry<ByteBuffer, Cell> entry : lastRow.entrySet())
                         {
                             bag.add(columnToTuple(entry.getValue(), cfInfo, parseType(cfDef.getComparator_type())));
                         }
@@ -149,7 +149,7 @@ public class CassandraStorage extends AbstractCassandraStorage
                 {
                     // read too much, hold on to it for next time
                     lastKey = (ByteBuffer)reader.getCurrentKey();
-                    lastRow = (SortedMap<ByteBuffer, Column>)reader.getCurrentValue();
+                    lastRow = (SortedMap<ByteBuffer, Cell>)reader.getCurrentValue();
                     // but return what we have so far
                     tuple.append(bag);
                     return tuple;
@@ -163,13 +163,13 @@ public class CassandraStorage extends AbstractCassandraStorage
                             tuple = keyToTuple(lastKey, cfDef, parseType(cfDef.getKey_validation_class()));
                         else
                             addKeyToTuple(tuple, lastKey, cfDef, parseType(cfDef.getKey_validation_class()));
-                        for (Map.Entry<ByteBuffer, Column> entry : lastRow.entrySet())
+                        for (Map.Entry<ByteBuffer, Cell> entry : lastRow.entrySet())
                         {
                             bag.add(columnToTuple(entry.getValue(), cfInfo, parseType(cfDef.getComparator_type())));
                         }
                         tuple.append(bag);
                         lastKey = key;
-                        lastRow = (SortedMap<ByteBuffer, Column>)reader.getCurrentValue();
+                        lastRow = (SortedMap<ByteBuffer, Cell>)reader.getCurrentValue();
                         return tuple;
                     }
                     if (tuple == null)
@@ -177,17 +177,17 @@ public class CassandraStorage extends AbstractCassandraStorage
                     else
                         addKeyToTuple(tuple, lastKey, cfDef, parseType(cfDef.getKey_validation_class()));
                 }
-                SortedMap<ByteBuffer, Column> row = (SortedMap<ByteBuffer, Column>)reader.getCurrentValue();
+                SortedMap<ByteBuffer, Cell> row = (SortedMap<ByteBuffer, Cell>)reader.getCurrentValue();
                 if (lastRow != null) // prepend what was read last time
                 {
-                    for (Map.Entry<ByteBuffer, Column> entry : lastRow.entrySet())
+                    for (Map.Entry<ByteBuffer, Cell> entry : lastRow.entrySet())
                     {
                         bag.add(columnToTuple(entry.getValue(), cfInfo, parseType(cfDef.getComparator_type())));
                     }
                     lastKey = null;
                     lastRow = null;
                 }
-                for (Map.Entry<ByteBuffer, Column> entry : row.entrySet())
+                for (Map.Entry<ByteBuffer, Cell> entry : row.entrySet())
                 {
                     bag.add(columnToTuple(entry.getValue(), cfInfo, parseType(cfDef.getComparator_type())));
                 }
@@ -214,7 +214,7 @@ public class CassandraStorage extends AbstractCassandraStorage
             CfInfo cfInfo = getCfInfo(loadSignature);
             CfDef cfDef = cfInfo.cfDef;
             ByteBuffer key = reader.getCurrentKey();
-            Map<ByteBuffer, Column> cf = reader.getCurrentValue();
+            Map<ByteBuffer, Cell> cf = reader.getCurrentValue();
             assert key != null && cf != null;
 
             // output tuple, will hold the key, each indexed column in a tuple, then a bag of the rest
@@ -248,7 +248,7 @@ public class CassandraStorage extends AbstractCassandraStorage
                 added.put(cdef.name, true);
             }
             // now add all the other columns
-            for (Map.Entry<ByteBuffer, Column> entry : cf.entrySet())
+            for (Map.Entry<ByteBuffer, Cell> entry : cf.entrySet())
             {
                 if (!added.containsKey(entry.getKey()))
                     bag.add(columnToTuple(entry.getValue(), cfInfo, parseType(cfDef.getComparator_type())));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/hadoop/pig/CqlStorage.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hadoop/pig/CqlStorage.java b/src/java/org/apache/cassandra/hadoop/pig/CqlStorage.java
index 7ce78de..3dda264 100644
--- a/src/java/org/apache/cassandra/hadoop/pig/CqlStorage.java
+++ b/src/java/org/apache/cassandra/hadoop/pig/CqlStorage.java
@@ -25,7 +25,7 @@ import java.util.*;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.db.composites.CellNames;
-import org.apache.cassandra.db.Column;
+import org.apache.cassandra.db.Cell;
 import org.apache.cassandra.db.marshal.*;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.hadoop.*;
@@ -112,9 +112,9 @@ public class CqlStorage extends AbstractCassandraStorage
                 ByteBuffer columnValue = columns.get(ByteBufferUtil.string(cdef.name.duplicate()));
                 if (columnValue != null)
                 {
-                    Column column = new Column(CellNames.simpleDense(cdef.name), columnValue);
+                    Cell cell = new Cell(CellNames.simpleDense(cdef.name), columnValue);
                     AbstractType<?> validator = getValidatorMap(cfDef).get(cdef.name);
-                    setTupleValue(tuple, i, cqlColumnToObj(column, cfDef), validator);
+                    setTupleValue(tuple, i, cqlColumnToObj(cell, cfDef), validator);
                 }
                 else
                     tuple.set(i, null);
@@ -183,7 +183,7 @@ public class CqlStorage extends AbstractCassandraStorage
     }
 
     /** convert a cql column to an object */
-    private Object cqlColumnToObj(Column col, CfDef cfDef) throws IOException
+    private Object cqlColumnToObj(Cell col, CfDef cfDef) throws IOException
     {
         // standard
         Map<ByteBuffer,AbstractType> validators = getValidatorMap(cfDef);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java b/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
index 27f1c12..83037ae 100644
--- a/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
@@ -109,16 +109,16 @@ public abstract class AbstractSSTableSimpleWriter
         currentSuperColumn = name;
     }
 
-    private void addColumn(Column column)
+    private void addColumn(Cell cell)
     {
         if (columnFamily.metadata().isSuper())
         {
             if (currentSuperColumn == null)
-                throw new IllegalStateException("Trying to add a column to a super column family, but no super column has been started.");
+                throw new IllegalStateException("Trying to add a cell to a super column family, but no super cell has been started.");
 
-            column = column.withUpdatedName(columnFamily.getComparator().makeCellName(currentSuperColumn, column.name().toByteBuffer()));
+            cell = cell.withUpdatedName(columnFamily.getComparator().makeCellName(currentSuperColumn, cell.name().toByteBuffer()));
         }
-        columnFamily.addColumn(column);
+        columnFamily.addColumn(cell);
     }
 
     /**
@@ -129,7 +129,7 @@ public abstract class AbstractSSTableSimpleWriter
      */
     public void addColumn(ByteBuffer name, ByteBuffer value, long timestamp)
     {
-        addColumn(new Column(metadata.comparator.cellFromByteBuffer(name), value, timestamp));
+        addColumn(new Cell(metadata.comparator.cellFromByteBuffer(name), value, timestamp));
     }
 
     /**
@@ -144,7 +144,7 @@ public abstract class AbstractSSTableSimpleWriter
      */
     public void addExpiringColumn(ByteBuffer name, ByteBuffer value, long timestamp, int ttl, long expirationTimestampMS)
     {
-        addColumn(new ExpiringColumn(metadata.comparator.cellFromByteBuffer(name), value, timestamp, ttl, (int)(expirationTimestampMS / 1000)));
+        addColumn(new ExpiringCell(metadata.comparator.cellFromByteBuffer(name), value, timestamp, ttl, (int)(expirationTimestampMS / 1000)));
     }
 
     /**
@@ -154,7 +154,7 @@ public abstract class AbstractSSTableSimpleWriter
      */
     public void addCounterColumn(ByteBuffer name, long value)
     {
-        addColumn(new CounterColumn(metadata.comparator.cellFromByteBuffer(name), CounterContext.instance().create(counterid, 1L, value, false), System.currentTimeMillis()));
+        addColumn(new CounterCell(metadata.comparator.cellFromByteBuffer(name), CounterContext.instance().create(counterid, 1L, value, false), System.currentTimeMillis()));
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/io/sstable/SSTableWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableWriter.java b/src/java/org/apache/cassandra/io/sstable/SSTableWriter.java
index 17d9b0e..ac8e2b2 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableWriter.java
@@ -246,8 +246,8 @@ public class SSTableWriter extends SSTable
                 OnDiskAtom atom = iter.next();
                 if (atom == null)
                     break;
-                if (atom instanceof CounterColumn)
-                    atom = ((CounterColumn) atom).markDeltaToBeCleared();
+                if (atom instanceof CounterCell)
+                    atom = ((CounterCell) atom).markDeltaToBeCleared();
 
                 int deletionTime = atom.getLocalDeletionTime();
                 if (deletionTime < Integer.MAX_VALUE)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/service/RowDataResolver.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/RowDataResolver.java b/src/java/org/apache/cassandra/service/RowDataResolver.java
index 2cbda33..f1948cd 100644
--- a/src/java/org/apache/cassandra/service/RowDataResolver.java
+++ b/src/java/org/apache/cassandra/service/RowDataResolver.java
@@ -149,7 +149,7 @@ public class RowDataResolver extends AbstractRowResolver
         // this will handle removing columns and subcolumns that are supressed by a row or
         // supercolumn tombstone.
         QueryFilter filter = new QueryFilter(null, resolved.metadata().cfName, new IdentityQueryFilter(), now);
-        List<CloseableIterator<Column>> iters = new ArrayList<CloseableIterator<Column>>();
+        List<CloseableIterator<Cell>> iters = new ArrayList<CloseableIterator<Cell>>();
         for (ColumnFamily version : versions)
         {
             if (version == null)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/service/StorageProxy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageProxy.java b/src/java/org/apache/cassandra/service/StorageProxy.java
index cf53865..6f362db 100644
--- a/src/java/org/apache/cassandra/service/StorageProxy.java
+++ b/src/java/org/apache/cassandra/service/StorageProxy.java
@@ -297,9 +297,9 @@ public class StorageProxy implements StorageProxyMBean
         // that excepted don't have. So we just check that for each columns in expected:
         //   - if it is a tombstone, whether current has no column or a tombstone;
         //   - otherwise, that current has a live column with the same value.
-        for (Column e : expected)
+        for (Cell e : expected)
         {
-            Column c = current.getColumn(e.name());
+            Cell c = current.getColumn(e.name());
             if (e.isLive(now))
             {
                 if (!(c != null && c.isLive(now) && c.value().equals(e.value())))

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/service/pager/AbstractQueryPager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/pager/AbstractQueryPager.java b/src/java/org/apache/cassandra/service/pager/AbstractQueryPager.java
index d4748b0..825f270 100644
--- a/src/java/org/apache/cassandra/service/pager/AbstractQueryPager.java
+++ b/src/java/org/apache/cassandra/service/pager/AbstractQueryPager.java
@@ -249,14 +249,14 @@ abstract class AbstractQueryPager implements QueryPager
              : discardTail(cf, toDiscard, newCf, cf.iterator(), tester);
     }
 
-    private int discardHead(ColumnFamily cf, int toDiscard, ColumnFamily copy, Iterator<Column> iter, DeletionInfo.InOrderTester tester)
+    private int discardHead(ColumnFamily cf, int toDiscard, ColumnFamily copy, Iterator<Cell> iter, DeletionInfo.InOrderTester tester)
     {
         ColumnCounter counter = columnCounter();
 
         // Discard the first 'toDiscard' live
         while (iter.hasNext())
         {
-            Column c = iter.next();
+            Cell c = iter.next();
             counter.count(c, tester);
             if (counter.live() > toDiscard)
             {
@@ -268,7 +268,7 @@ abstract class AbstractQueryPager implements QueryPager
         return Math.min(counter.live(), toDiscard);
     }
 
-    private int discardTail(ColumnFamily cf, int toDiscard, ColumnFamily copy, Iterator<Column> iter, DeletionInfo.InOrderTester tester)
+    private int discardTail(ColumnFamily cf, int toDiscard, ColumnFamily copy, Iterator<Cell> iter, DeletionInfo.InOrderTester tester)
     {
         // Redoing the counting like that is not extremely efficient.
         // This is called only for reversed slices or in the case of a race between
@@ -279,7 +279,7 @@ abstract class AbstractQueryPager implements QueryPager
         // Discard the last 'toDiscard' live (so stop adding as sound as we're past 'liveCount - toDiscard')
         while (iter.hasNext())
         {
-            Column c = iter.next();
+            Cell c = iter.next();
             counter.count(c, tester);
             if (counter.live() > liveCount - toDiscard)
                 break;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/service/paxos/Commit.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/paxos/Commit.java b/src/java/org/apache/cassandra/service/paxos/Commit.java
index 9904045..1f95d04 100644
--- a/src/java/org/apache/cassandra/service/paxos/Commit.java
+++ b/src/java/org/apache/cassandra/service/paxos/Commit.java
@@ -30,7 +30,7 @@ import java.nio.ByteBuffer;
 import com.google.common.base.Objects;
 
 import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.db.Column;
+import org.apache.cassandra.db.Cell;
 import org.apache.cassandra.db.ColumnFamily;
 import org.apache.cassandra.db.ColumnSerializer;
 import org.apache.cassandra.db.EmptyColumns;
@@ -120,8 +120,8 @@ public class Commit
         // the collection and we want that to have a lower timestamp and our new values. Since tombstones wins over normal insert, using t-1
         // should not be a problem in general (see #6069).
         cf.deletionInfo().updateAllTimestamp(t-1);
-        for (Column column : updates)
-            cf.addAtom(column.withUpdatedTimestamp(t));
+        for (Cell cell : updates)
+            cf.addAtom(cell.withUpdatedTimestamp(t));
         return cf;
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/thrift/CassandraServer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/thrift/CassandraServer.java b/src/java/org/apache/cassandra/thrift/CassandraServer.java
index 4705513..8c0e594 100644
--- a/src/java/org/apache/cassandra/thrift/CassandraServer.java
+++ b/src/java/org/apache/cassandra/thrift/CassandraServer.java
@@ -128,65 +128,65 @@ public class CassandraServer implements Cassandra.Iface
         return columnFamilyKeyMap;
     }
 
-    public List<ColumnOrSuperColumn> thriftifyColumns(Collection<org.apache.cassandra.db.Column> columns, boolean reverseOrder, long now)
+    public List<ColumnOrSuperColumn> thriftifyColumns(Collection<Cell> cells, boolean reverseOrder, long now)
     {
-        ArrayList<ColumnOrSuperColumn> thriftColumns = new ArrayList<ColumnOrSuperColumn>(columns.size());
-        for (org.apache.cassandra.db.Column column : columns)
+        ArrayList<ColumnOrSuperColumn> thriftColumns = new ArrayList<ColumnOrSuperColumn>(cells.size());
+        for (Cell cell : cells)
         {
-            if (column.isMarkedForDelete(now))
+            if (cell.isMarkedForDelete(now))
                 continue;
 
-            thriftColumns.add(thriftifyColumnWithName(column, column.name().toByteBuffer()));
+            thriftColumns.add(thriftifyColumnWithName(cell, cell.name().toByteBuffer()));
         }
 
         // we have to do the reversing here, since internally we pass results around in ColumnFamily
-        // objects, which always sort their columns in the "natural" order
+        // objects, which always sort their cells in the "natural" order
         // TODO this is inconvenient for direct users of StorageProxy
         if (reverseOrder)
             Collections.reverse(thriftColumns);
         return thriftColumns;
     }
 
-    private ColumnOrSuperColumn thriftifyColumnWithName(org.apache.cassandra.db.Column column, ByteBuffer newName)
+    private ColumnOrSuperColumn thriftifyColumnWithName(Cell cell, ByteBuffer newName)
     {
-        if (column instanceof org.apache.cassandra.db.CounterColumn)
-            return new ColumnOrSuperColumn().setCounter_column(thriftifySubCounter(column).setName(newName));
+        if (cell instanceof CounterCell)
+            return new ColumnOrSuperColumn().setCounter_column(thriftifySubCounter(cell).setName(newName));
         else
-            return new ColumnOrSuperColumn().setColumn(thriftifySubColumn(column).setName(newName));
+            return new ColumnOrSuperColumn().setColumn(thriftifySubColumn(cell).setName(newName));
     }
 
-    private Column thriftifySubColumn(org.apache.cassandra.db.Column column)
+    private Column thriftifySubColumn(Cell cell)
     {
-        assert !(column instanceof org.apache.cassandra.db.CounterColumn);
+        assert !(cell instanceof CounterCell);
 
-        Column thrift_column = new Column(column.name().toByteBuffer()).setValue(column.value()).setTimestamp(column.timestamp());
-        if (column instanceof ExpiringColumn)
+        Column thrift_column = new Column(cell.name().toByteBuffer()).setValue(cell.value()).setTimestamp(cell.timestamp());
+        if (cell instanceof ExpiringCell)
         {
-            thrift_column.setTtl(((ExpiringColumn) column).getTimeToLive());
+            thrift_column.setTtl(((ExpiringCell) cell).getTimeToLive());
         }
         return thrift_column;
     }
 
-    private List<Column> thriftifyColumnsAsColumns(Collection<org.apache.cassandra.db.Column> columns, long now)
+    private List<Column> thriftifyColumnsAsColumns(Collection<Cell> cells, long now)
     {
-        List<Column> thriftColumns = new ArrayList<Column>(columns.size());
-        for (org.apache.cassandra.db.Column column : columns)
+        List<Column> thriftColumns = new ArrayList<Column>(cells.size());
+        for (Cell cell : cells)
         {
-            if (column.isMarkedForDelete(now))
+            if (cell.isMarkedForDelete(now))
                 continue;
 
-            thriftColumns.add(thriftifySubColumn(column));
+            thriftColumns.add(thriftifySubColumn(cell));
         }
         return thriftColumns;
     }
 
-    private CounterColumn thriftifySubCounter(org.apache.cassandra.db.Column column)
+    private CounterColumn thriftifySubCounter(Cell cell)
     {
-        assert column instanceof org.apache.cassandra.db.CounterColumn;
-        return new CounterColumn(column.name().toByteBuffer(), CounterContext.instance().total(column.value()));
+        assert cell instanceof CounterCell;
+        return new CounterColumn(cell.name().toByteBuffer(), CounterContext.instance().total(cell.value()));
     }
 
-    private List<ColumnOrSuperColumn> thriftifySuperColumns(Collection<org.apache.cassandra.db.Column> columns,
+    private List<ColumnOrSuperColumn> thriftifySuperColumns(Collection<Cell> cells,
                                                             boolean reverseOrder,
                                                             long now,
                                                             boolean subcolumnsOnly,
@@ -194,13 +194,13 @@ public class CassandraServer implements Cassandra.Iface
     {
         if (subcolumnsOnly)
         {
-            ArrayList<ColumnOrSuperColumn> thriftSuperColumns = new ArrayList<ColumnOrSuperColumn>(columns.size());
-            for (org.apache.cassandra.db.Column column : columns)
+            ArrayList<ColumnOrSuperColumn> thriftSuperColumns = new ArrayList<ColumnOrSuperColumn>(cells.size());
+            for (Cell cell : cells)
             {
-                if (column.isMarkedForDelete(now))
+                if (cell.isMarkedForDelete(now))
                     continue;
 
-                thriftSuperColumns.add(thriftifyColumnWithName(column, SuperColumns.subName(column.name())));
+                thriftSuperColumns.add(thriftifyColumnWithName(cell, SuperColumns.subName(cell.name())));
             }
             if (reverseOrder)
                 Collections.reverse(thriftSuperColumns);
@@ -209,28 +209,28 @@ public class CassandraServer implements Cassandra.Iface
         else
         {
             if (isCounterCF)
-                return thriftifyCounterSuperColumns(columns, reverseOrder, now);
+                return thriftifyCounterSuperColumns(cells, reverseOrder, now);
             else
-                return thriftifySuperColumns(columns, reverseOrder, now);
+                return thriftifySuperColumns(cells, reverseOrder, now);
         }
     }
 
-    private List<ColumnOrSuperColumn> thriftifySuperColumns(Collection<org.apache.cassandra.db.Column> columns, boolean reverseOrder, long now)
+    private List<ColumnOrSuperColumn> thriftifySuperColumns(Collection<Cell> cells, boolean reverseOrder, long now)
     {
-        ArrayList<ColumnOrSuperColumn> thriftSuperColumns = new ArrayList<ColumnOrSuperColumn>(columns.size());
+        ArrayList<ColumnOrSuperColumn> thriftSuperColumns = new ArrayList<ColumnOrSuperColumn>(cells.size());
         SuperColumn current = null;
-        for (org.apache.cassandra.db.Column column : columns)
+        for (Cell cell : cells)
         {
-            if (column.isMarkedForDelete(now))
+            if (cell.isMarkedForDelete(now))
                 continue;
 
-            ByteBuffer scName = SuperColumns.scName(column.name());
+            ByteBuffer scName = SuperColumns.scName(cell.name());
             if (current == null || !scName.equals(current.bufferForName()))
             {
                 current = new SuperColumn(scName, new ArrayList<Column>());
                 thriftSuperColumns.add(new ColumnOrSuperColumn().setSuper_column(current));
             }
-            current.getColumns().add(thriftifySubColumn(column).setName(SuperColumns.subName(column.name())));
+            current.getColumns().add(thriftifySubColumn(cell).setName(SuperColumns.subName(cell.name())));
         }
 
         if (reverseOrder)
@@ -239,22 +239,22 @@ public class CassandraServer implements Cassandra.Iface
         return thriftSuperColumns;
     }
 
-    private List<ColumnOrSuperColumn> thriftifyCounterSuperColumns(Collection<org.apache.cassandra.db.Column> columns, boolean reverseOrder, long now)
+    private List<ColumnOrSuperColumn> thriftifyCounterSuperColumns(Collection<Cell> cells, boolean reverseOrder, long now)
     {
-        ArrayList<ColumnOrSuperColumn> thriftSuperColumns = new ArrayList<ColumnOrSuperColumn>(columns.size());
+        ArrayList<ColumnOrSuperColumn> thriftSuperColumns = new ArrayList<ColumnOrSuperColumn>(cells.size());
         CounterSuperColumn current = null;
-        for (org.apache.cassandra.db.Column column : columns)
+        for (Cell cell : cells)
         {
-            if (column.isMarkedForDelete(now))
+            if (cell.isMarkedForDelete(now))
                 continue;
 
-            ByteBuffer scName = SuperColumns.scName(column.name());
+            ByteBuffer scName = SuperColumns.scName(cell.name());
             if (current == null || !scName.equals(current.bufferForName()))
             {
                 current = new CounterSuperColumn(scName, new ArrayList<CounterColumn>());
                 thriftSuperColumns.add(new ColumnOrSuperColumn().setCounter_super_column(current));
             }
-            current.getColumns().add(thriftifySubCounter(column).setName(SuperColumns.subName(column.name())));
+            current.getColumns().add(thriftifySubCounter(cell).setName(SuperColumns.subName(cell.name())));
         }
 
         if (reverseOrder)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/thrift/ThriftValidation.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/thrift/ThriftValidation.java b/src/java/org/apache/cassandra/thrift/ThriftValidation.java
index b1eb50b..c0c22f1 100644
--- a/src/java/org/apache/cassandra/thrift/ThriftValidation.java
+++ b/src/java/org/apache/cassandra/thrift/ThriftValidation.java
@@ -198,7 +198,7 @@ public class ThriftValidation
     private static void validateColumnNames(CFMetaData metadata, ByteBuffer superColumnName, Iterable<ByteBuffer> column_names)
     throws org.apache.cassandra.exceptions.InvalidRequestException
     {
-        int maxNameLength = org.apache.cassandra.db.Column.MAX_NAME_LENGTH;
+        int maxNameLength = Cell.MAX_NAME_LENGTH;
 
         if (superColumnName != null)
         {
@@ -349,8 +349,8 @@ public class ThriftValidation
             if (column.ttl <= 0)
                 throw new org.apache.cassandra.exceptions.InvalidRequestException("ttl must be positive");
 
-            if (column.ttl > ExpiringColumn.MAX_TTL)
-                throw new org.apache.cassandra.exceptions.InvalidRequestException(String.format("ttl is too large. requested (%d) maximum (%d)", column.ttl, ExpiringColumn.MAX_TTL));
+            if (column.ttl > ExpiringCell.MAX_TTL)
+                throw new org.apache.cassandra.exceptions.InvalidRequestException(String.format("ttl is too large. requested (%d) maximum (%d)", column.ttl, ExpiringCell.MAX_TTL));
         }
         else
         {
@@ -423,7 +423,7 @@ public class ThriftValidation
     }
 
     /**
-     * Validates the data part of the column (everything in the Column object but the name, which is assumed to be valid)
+     * Validates the data part of the column (everything in the column object but the name, which is assumed to be valid)
      */
     public static void validateColumnData(CFMetaData metadata, ByteBuffer scName, Column column) throws org.apache.cassandra.exceptions.InvalidRequestException
     {
@@ -464,12 +464,12 @@ public class ThriftValidation
                                                                               metadata.ksName));
     }
 
-    private static org.apache.cassandra.db.Column asDBColumn(CellName name, Column column)
+    private static Cell asDBColumn(CellName name, Column column)
     {
         if (column.ttl <= 0)
-            return new org.apache.cassandra.db.Column(name, column.value, column.timestamp);
+            return new Cell(name, column.value, column.timestamp);
         else
-            return new org.apache.cassandra.db.ExpiringColumn(name, column.value, column.timestamp, column.ttl);
+            return new ExpiringCell(name, column.value, column.timestamp, column.ttl);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/tools/SSTableExport.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/SSTableExport.java b/src/java/org/apache/cassandra/tools/SSTableExport.java
index c5fc970..6cce81d 100644
--- a/src/java/org/apache/cassandra/tools/SSTableExport.java
+++ b/src/java/org/apache/cassandra/tools/SSTableExport.java
@@ -147,9 +147,9 @@ public class SSTableExport
 
     private static List<Object> serializeAtom(OnDiskAtom atom, CFMetaData cfMetaData)
     {
-        if (atom instanceof Column)
+        if (atom instanceof Cell)
         {
-            return serializeColumn((Column) atom, cfMetaData);
+            return serializeColumn((Cell) atom, cfMetaData);
         }
         else
         {
@@ -166,46 +166,46 @@ public class SSTableExport
     }
 
     /**
-     * Serialize a given column to the JSON format
+     * Serialize a given cell to the JSON format
      *
-     * @param column     column presentation
+     * @param cell     cell presentation
      * @param comparator columns comparator
      * @param cfMetaData Column Family metadata (to get validator)
-     * @return column as serialized list
+     * @return cell as serialized list
      */
-    private static List<Object> serializeColumn(Column column, CFMetaData cfMetaData)
+    private static List<Object> serializeColumn(Cell cell, CFMetaData cfMetaData)
     {
         CellNameType comparator = cfMetaData.comparator;
         ArrayList<Object> serializedColumn = new ArrayList<Object>();
 
-        ByteBuffer value = ByteBufferUtil.clone(column.value());
+        ByteBuffer value = ByteBufferUtil.clone(cell.value());
 
-        serializedColumn.add(comparator.getString(column.name()));
-        if (column instanceof DeletedColumn)
+        serializedColumn.add(comparator.getString(cell.name()));
+        if (cell instanceof DeletedCell)
         {
             serializedColumn.add(ByteBufferUtil.bytesToHex(value));
         }
         else
         {
-            AbstractType<?> validator = cfMetaData.getValueValidator(column.name());
+            AbstractType<?> validator = cfMetaData.getValueValidator(cell.name());
             serializedColumn.add(validator.getString(value));
         }
-        serializedColumn.add(column.timestamp());
+        serializedColumn.add(cell.timestamp());
 
-        if (column instanceof DeletedColumn)
+        if (cell instanceof DeletedCell)
         {
             serializedColumn.add("d");
         }
-        else if (column instanceof ExpiringColumn)
+        else if (cell instanceof ExpiringCell)
         {
             serializedColumn.add("e");
-            serializedColumn.add(((ExpiringColumn) column).getTimeToLive());
-            serializedColumn.add(column.getLocalDeletionTime());
+            serializedColumn.add(((ExpiringCell) cell).getTimeToLive());
+            serializedColumn.add(cell.getLocalDeletionTime());
         }
-        else if (column instanceof CounterColumn)
+        else if (cell instanceof CounterCell)
         {
             serializedColumn.add("c");
-            serializedColumn.add(((CounterColumn) column).timestampOfLastDelete());
+            serializedColumn.add(((CounterCell) cell).timestampOfLastDelete());
         }
 
         return serializedColumn;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/tools/SSTableImport.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/SSTableImport.java b/src/java/org/apache/cassandra/tools/SSTableImport.java
index 0a6ef6b..71b687b 100644
--- a/src/java/org/apache/cassandra/tools/SSTableImport.java
+++ b/src/java/org/apache/cassandra/tools/SSTableImport.java
@@ -110,7 +110,7 @@ public class SSTableImport
                 CellNameType comparator = oldSCFormat ? new SimpleDenseCellNameType(SuperColumns.getComparatorFor(meta, isSubColumn)) : meta.comparator;
                 List fields = (List<?>) json;
 
-                assert fields.size() >= 3 : "Column definition should have at least 3";
+                assert fields.size() >= 3 : "Cell definition should have at least 3";
 
                 name  = stringAsType((String) fields.get(0), comparator.asAbstractType());
                 timestamp = (Long) fields.get(2);
@@ -257,11 +257,11 @@ public class SSTableImport
 
             if (col.isExpiring())
             {
-                cfamily.addColumn(new ExpiringColumn(cname, col.getValue(), col.timestamp, col.ttl, col.localExpirationTime));
+                cfamily.addColumn(new ExpiringCell(cname, col.getValue(), col.timestamp, col.ttl, col.localExpirationTime));
             }
             else if (col.isCounter())
             {
-                cfamily.addColumn(new CounterColumn(cname, col.getValue(), col.timestamp, col.timestampOfLastDelete));
+                cfamily.addColumn(new CounterCell(cname, col.getValue(), col.timestamp, col.timestampOfLastDelete));
             }
             else if (col.isDeleted())
             {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/tools/SSTableMetadataViewer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/SSTableMetadataViewer.java b/src/java/org/apache/cassandra/tools/SSTableMetadataViewer.java
index aeec284..d4b0b77 100644
--- a/src/java/org/apache/cassandra/tools/SSTableMetadataViewer.java
+++ b/src/java/org/apache/cassandra/tools/SSTableMetadataViewer.java
@@ -76,7 +76,7 @@ public class SSTableMetadataViewer
         long[] ecch = metadata.estimatedColumnCount.getBuckets(false);
 
         out.println(String.format("%-10s%18s%18s",
-                                  "Count", "Row Size", "Column Count"));
+                                  "Count", "Row Size", "Cell Count"));
 
         for (int i = 0; i < offsets.length; i++)
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/tracing/Tracing.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tracing/Tracing.java b/src/java/org/apache/cassandra/tracing/Tracing.java
index b1f618d..963ee0c 100644
--- a/src/java/org/apache/cassandra/tracing/Tracing.java
+++ b/src/java/org/apache/cassandra/tracing/Tracing.java
@@ -93,14 +93,14 @@ public class Tracing
 
     private static void addColumn(ColumnFamily cf, CellName name, ByteBuffer value)
     {
-        cf.addColumn(new ExpiringColumn(name, value, System.currentTimeMillis(), TTL));
+        cf.addColumn(new ExpiringCell(name, value, System.currentTimeMillis(), TTL));
     }
 
     public void addParameterColumns(ColumnFamily cf, Map<String, String> rawPayload)
     {
         for (Map.Entry<String, String> entry : rawPayload.entrySet())
         {
-            cf.addColumn(new ExpiringColumn(buildName(cf.metadata(), "parameters", entry.getKey()),
+            cf.addColumn(new ExpiringCell(buildName(cf.metadata(), "parameters", entry.getKey()),
                                             bytes(entry.getValue()), System.currentTimeMillis(), TTL));
         }
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/utils/BloomCalculations.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/BloomCalculations.java b/src/java/org/apache/cassandra/utils/BloomCalculations.java
index 17966e8..b73f531 100644
--- a/src/java/org/apache/cassandra/utils/BloomCalculations.java
+++ b/src/java/org/apache/cassandra/utils/BloomCalculations.java
@@ -35,7 +35,7 @@ class BloomCalculations {
 
     /**
      * In the following keyspaceName, the row 'i' shows false positive rates if i buckets
-     * per element are used.  Column 'j' shows false positive rates if j hash
+     * per element are used.  Cell 'j' shows false positive rates if j hash
      * functions are used.  The first row is 'i=0', the first column is 'j=0'.
      * Each cell (i,j) the false positive rate determined by using i buckets per
      * element and j hash functions.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/utils/CounterId.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/CounterId.java b/src/java/org/apache/cassandra/utils/CounterId.java
index 4b6fd46..a8545c0 100644
--- a/src/java/org/apache/cassandra/utils/CounterId.java
+++ b/src/java/org/apache/cassandra/utils/CounterId.java
@@ -26,7 +26,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import com.google.common.base.Objects;
 
-import org.apache.cassandra.db.CounterColumn;
+import org.apache.cassandra.db.CounterCell;
 import org.apache.cassandra.db.SystemKeyspace;
 
 public class CounterId implements Comparable<CounterId>
@@ -174,7 +174,7 @@ public class CounterId implements Comparable<CounterId>
             initialId = getLocalId();
         }
 
-        public void maybeRenew(CounterColumn column)
+        public void maybeRenew(CounterCell column)
         {
             if (!renewed && column.hasCounterId(initialId))
             {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/test/long/org/apache/cassandra/db/MeteredFlusherTest.java
----------------------------------------------------------------------
diff --git a/test/long/org/apache/cassandra/db/MeteredFlusherTest.java b/test/long/org/apache/cassandra/db/MeteredFlusherTest.java
index 99ae65a..07ddbd6 100644
--- a/test/long/org/apache/cassandra/db/MeteredFlusherTest.java
+++ b/test/long/org/apache/cassandra/db/MeteredFlusherTest.java
@@ -54,7 +54,7 @@ public class MeteredFlusherTest extends SchemaLoader
                 ColumnFamily cf = TreeMapBackedSortedColumns.factory.create("Keyspace1", "_CF" + i);
                 // don't cheat by allocating this outside of the loop; that defeats the purpose of deliberately using lots of memory
                 ByteBuffer value = ByteBuffer.allocate(100000);
-                cf.addColumn(new Column(Util.cellname("c"), value));
+                cf.addColumn(new Cell(Util.cellname("c"), value));
                 rm.add(cf);
                 rm.applyUnsafe();
             }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/test/long/org/apache/cassandra/db/compaction/LongCompactionsTest.java
----------------------------------------------------------------------
diff --git a/test/long/org/apache/cassandra/db/compaction/LongCompactionsTest.java b/test/long/org/apache/cassandra/db/compaction/LongCompactionsTest.java
index 68a2489..5d68a3b 100644
--- a/test/long/org/apache/cassandra/db/compaction/LongCompactionsTest.java
+++ b/test/long/org/apache/cassandra/db/compaction/LongCompactionsTest.java
@@ -81,7 +81,7 @@ public class LongCompactionsTest extends SchemaLoader
             for (int j = 0; j < rowsPerSSTable; j++)
             {
                 String key = String.valueOf(j);
-                Column[] cols = new Column[colsPerRow];
+                Cell[] cols = new Cell[colsPerRow];
                 for (int i = 0; i < colsPerRow; i++)
                 {
                     // last sstable has highest timestamps


[5/6] Rename Column to Cell

Posted by sl...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/db/ColumnFamily.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ColumnFamily.java b/src/java/org/apache/cassandra/db/ColumnFamily.java
index c2134c2..2ea60f1 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamily.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamily.java
@@ -54,7 +54,7 @@ import org.apache.cassandra.utils.*;
  * Whether the implementation is thread safe or not is left to the
  * implementing classes.
  */
-public abstract class ColumnFamily implements Iterable<Column>, IRowCacheEntry
+public abstract class ColumnFamily implements Iterable<Cell>, IRowCacheEntry
 {
     /* The column serializer for this Column Family. Create based on config. */
     public static final ColumnFamilySerializer serializer = new ColumnFamilySerializer();
@@ -102,20 +102,20 @@ public abstract class ColumnFamily implements Iterable<Column>, IRowCacheEntry
         return metadata;
     }
 
-    public void addIfRelevant(Column column, DeletionInfo.InOrderTester tester, int gcBefore)
+    public void addIfRelevant(Cell cell, DeletionInfo.InOrderTester tester, int gcBefore)
     {
-        // the column itself must be not gc-able (it is live, or a still relevant tombstone), (1)
-        // and if its container is deleted, the column must be changed more recently than the container tombstone (2)
-        if ((column.getLocalDeletionTime() >= gcBefore) // (1)
-            && (!tester.isDeleted(column.name(), column.timestamp())))                                // (2)
+        // the cell itself must be not gc-able (it is live, or a still relevant tombstone), (1)
+        // and if its container is deleted, the cell must be changed more recently than the container tombstone (2)
+        if ((cell.getLocalDeletionTime() >= gcBefore) // (1)
+            && (!tester.isDeleted(cell.name(), cell.timestamp())))                                // (2)
         {
-            addColumn(column);
+            addColumn(cell);
         }
     }
 
-    public void addColumn(Column column)
+    public void addColumn(Cell cell)
     {
-        addColumn(column, HeapAllocator.instance);
+        addColumn(cell, HeapAllocator.instance);
     }
 
     public void addColumn(CellName name, ByteBuffer value, long timestamp)
@@ -126,30 +126,30 @@ public abstract class ColumnFamily implements Iterable<Column>, IRowCacheEntry
     public void addColumn(CellName name, ByteBuffer value, long timestamp, int timeToLive)
     {
         assert !metadata().getDefaultValidator().isCommutative();
-        Column column = Column.create(name, value, timestamp, timeToLive, metadata());
-        addColumn(column);
+        Cell cell = Cell.create(name, value, timestamp, timeToLive, metadata());
+        addColumn(cell);
     }
 
     public void addCounter(CellName name, long value)
     {
-        addColumn(new CounterUpdateColumn(name, value, System.currentTimeMillis()));
+        addColumn(new CounterUpdateCell(name, value, System.currentTimeMillis()));
     }
 
     public void addTombstone(CellName name, ByteBuffer localDeletionTime, long timestamp)
     {
-        addColumn(new DeletedColumn(name, localDeletionTime, timestamp));
+        addColumn(new DeletedCell(name, localDeletionTime, timestamp));
     }
 
     public void addTombstone(CellName name, int localDeletionTime, long timestamp)
     {
-        addColumn(new DeletedColumn(name, localDeletionTime, timestamp));
+        addColumn(new DeletedCell(name, localDeletionTime, timestamp));
     }
 
     public void addAtom(OnDiskAtom atom)
     {
-        if (atom instanceof Column)
+        if (atom instanceof Cell)
         {
-            addColumn((Column)atom);
+            addColumn((Cell)atom);
         }
         else
         {
@@ -194,35 +194,35 @@ public abstract class ColumnFamily implements Iterable<Column>, IRowCacheEntry
     public abstract void purgeTombstones(int gcBefore);
 
     /**
-     * Adds a column to this column map.
-     * If a column with the same name is already present in the map, it will
-     * be replaced by the newly added column.
+     * Adds a cell to this cell map.
+     * If a cell with the same name is already present in the map, it will
+     * be replaced by the newly added cell.
      */
-    public abstract void addColumn(Column column, Allocator allocator);
+    public abstract void addColumn(Cell cell, Allocator allocator);
 
     /**
      * Adds all the columns of a given column map to this column map.
      * This is equivalent to:
      *   <code>
-     *   for (Column c : cm)
+     *   for (Cell c : cm)
      *      addColumn(c, ...);
      *   </code>
      *  but is potentially faster.
      */
-    public abstract void addAll(ColumnFamily cm, Allocator allocator, Function<Column, Column> transformation);
+    public abstract void addAll(ColumnFamily cm, Allocator allocator, Function<Cell, Cell> transformation);
 
     /**
-     * Replace oldColumn if present by newColumn.
-     * Returns true if oldColumn was present and thus replaced.
-     * oldColumn and newColumn should have the same name.
+     * Replace oldCell if present by newCell.
+     * Returns true if oldCell was present and thus replaced.
+     * oldCell and newCell should have the same name.
      */
-    public abstract boolean replace(Column oldColumn, Column newColumn);
+    public abstract boolean replace(Cell oldCell, Cell newCell);
 
     /**
      * Get a column given its name, returning null if the column is not
      * present.
      */
-    public abstract Column getColumn(CellName name);
+    public abstract Cell getColumn(CellName name);
 
     /**
      * Returns an iterable with the names of columns in this column map in the same order
@@ -235,14 +235,14 @@ public abstract class ColumnFamily implements Iterable<Column>, IRowCacheEntry
      * The columns in the returned collection should be sorted as the columns
      * in this map.
      */
-    public abstract Collection<Column> getSortedColumns();
+    public abstract Collection<Cell> getSortedColumns();
 
     /**
      * Returns the columns of this column map as a collection.
      * The columns in the returned collection should be sorted in reverse
      * order of the columns in this map.
      */
-    public abstract Collection<Column> getReverseSortedColumns();
+    public abstract Collection<Cell> getReverseSortedColumns();
 
     /**
      * Returns the number of columns in this map.
@@ -261,13 +261,13 @@ public abstract class ColumnFamily implements Iterable<Column>, IRowCacheEntry
      * Returns an iterator over the columns of this map that returns only the matching @param slices.
      * The provided slices must be in order and must be non-overlapping.
      */
-    public abstract Iterator<Column> iterator(ColumnSlice[] slices);
+    public abstract Iterator<Cell> iterator(ColumnSlice[] slices);
 
     /**
      * Returns a reversed iterator over the columns of this map that returns only the matching @param slices.
      * The provided slices must be in reversed order and must be non-overlapping.
      */
-    public abstract Iterator<Column> reverseIterator(ColumnSlice[] slices);
+    public abstract Iterator<Cell> reverseIterator(ColumnSlice[] slices);
 
     /**
      * Returns if this map only support inserts in reverse order.
@@ -284,7 +284,7 @@ public abstract class ColumnFamily implements Iterable<Column>, IRowCacheEntry
 
     public void addAll(ColumnFamily cf, Allocator allocator)
     {
-        addAll(cf, allocator, Functions.<Column>identity());
+        addAll(cf, allocator, Functions.<Cell>identity());
     }
 
     /*
@@ -300,20 +300,20 @@ public abstract class ColumnFamily implements Iterable<Column>, IRowCacheEntry
         // (don't need to worry about cfNew containing Columns that are shadowed by
         // the delete tombstone, since cfNew was generated by CF.resolve, which
         // takes care of those for us.)
-        for (Column columnExternal : cfComposite)
+        for (Cell cellExternal : cfComposite)
         {
-            CellName cName = columnExternal.name();
-            Column columnInternal = getColumn(cName);
-            if (columnInternal == null)
+            CellName cName = cellExternal.name();
+            Cell cellInternal = getColumn(cName);
+            if (cellInternal == null)
             {
-                cfDiff.addColumn(columnExternal);
+                cfDiff.addColumn(cellExternal);
             }
             else
             {
-                Column columnDiff = columnInternal.diff(columnExternal);
-                if (columnDiff != null)
+                Cell cellDiff = cellInternal.diff(cellExternal);
+                if (cellDiff != null)
                 {
-                    cfDiff.addColumn(columnDiff);
+                    cfDiff.addColumn(cellDiff);
                 }
             }
         }
@@ -326,16 +326,16 @@ public abstract class ColumnFamily implements Iterable<Column>, IRowCacheEntry
     public long dataSize()
     {
         long size = 0;
-        for (Column column : this)
-            size += column.dataSize();
+        for (Cell cell : this)
+            size += cell.dataSize();
         return size;
     }
 
     public long maxTimestamp()
     {
         long maxTimestamp = deletionInfo().maxTimestamp();
-        for (Column column : this)
-            maxTimestamp = Math.max(maxTimestamp, column.maxTimestamp());
+        for (Cell cell : this)
+            maxTimestamp = Math.max(maxTimestamp, cell.maxTimestamp());
         return maxTimestamp;
     }
 
@@ -345,8 +345,8 @@ public abstract class ColumnFamily implements Iterable<Column>, IRowCacheEntry
         HashCodeBuilder builder = new HashCodeBuilder(373, 75437)
                 .append(metadata)
                 .append(deletionInfo());
-        for (Column column : this)
-            builder.append(column);
+        for (Cell cell : this)
+            builder.append(cell);
         return builder.toHashCode();
     }
 
@@ -388,8 +388,8 @@ public abstract class ColumnFamily implements Iterable<Column>, IRowCacheEntry
 
     public void updateDigest(MessageDigest digest)
     {
-        for (Column column : this)
-            column.updateDigest(digest);
+        for (Cell cell : this)
+            cell.updateDigest(digest);
     }
 
     public static ColumnFamily diff(ColumnFamily cf1, ColumnFamily cf2)
@@ -420,16 +420,16 @@ public abstract class ColumnFamily implements Iterable<Column>, IRowCacheEntry
         int maxLocalDeletionTime = Integer.MIN_VALUE;
         List<ByteBuffer> minColumnNamesSeen = Collections.emptyList();
         List<ByteBuffer> maxColumnNamesSeen = Collections.emptyList();
-        for (Column column : this)
+        for (Cell cell : this)
         {
-            minTimestampSeen = Math.min(minTimestampSeen, column.minTimestamp());
-            maxTimestampSeen = Math.max(maxTimestampSeen, column.maxTimestamp());
-            maxLocalDeletionTime = Math.max(maxLocalDeletionTime, column.getLocalDeletionTime());
-            int deletionTime = column.getLocalDeletionTime();
+            minTimestampSeen = Math.min(minTimestampSeen, cell.minTimestamp());
+            maxTimestampSeen = Math.max(maxTimestampSeen, cell.maxTimestamp());
+            maxLocalDeletionTime = Math.max(maxLocalDeletionTime, cell.getLocalDeletionTime());
+            int deletionTime = cell.getLocalDeletionTime();
             if (deletionTime < Integer.MAX_VALUE)
                 tombstones.update(deletionTime);
-            minColumnNamesSeen = ColumnNameHelper.minComponents(minColumnNamesSeen, column.name, metadata.comparator);
-            maxColumnNamesSeen = ColumnNameHelper.maxComponents(maxColumnNamesSeen, column.name, metadata.comparator);
+            minColumnNamesSeen = ColumnNameHelper.minComponents(minColumnNamesSeen, cell.name, metadata.comparator);
+            maxColumnNamesSeen = ColumnNameHelper.maxComponents(maxColumnNamesSeen, cell.name, metadata.comparator);
         }
         return new ColumnStats(getColumnCount(), minTimestampSeen, maxTimestampSeen, maxLocalDeletionTime, tombstones, minColumnNamesSeen, maxColumnNamesSeen);
     }
@@ -449,18 +449,18 @@ public abstract class ColumnFamily implements Iterable<Column>, IRowCacheEntry
 
     public boolean hasOnlyTombstones(long now)
     {
-        for (Column column : this)
-            if (column.isLive(now))
+        for (Cell cell : this)
+            if (cell.isLive(now))
                 return false;
         return true;
     }
 
-    public Iterator<Column> iterator()
+    public Iterator<Cell> iterator()
     {
         return getSortedColumns().iterator();
     }
 
-    public Iterator<Column> reverseIterator()
+    public Iterator<Cell> reverseIterator()
     {
         return getReverseSortedColumns().iterator();
     }
@@ -473,8 +473,8 @@ public abstract class ColumnFamily implements Iterable<Column>, IRowCacheEntry
 
         // Do we have colums that are either deleted by the container or gcable tombstone?
         DeletionInfo.InOrderTester tester = inOrderDeletionTester();
-        for (Column column : this)
-            if (tester.isDeleted(column) || column.hasIrrelevantData(gcBefore))
+        for (Cell cell : this)
+            if (tester.isDeleted(cell) || cell.hasIrrelevantData(gcBefore))
                 return true;
 
         return false;
@@ -483,8 +483,8 @@ public abstract class ColumnFamily implements Iterable<Column>, IRowCacheEntry
     public Map<CellName, ByteBuffer> asMap()
     {
         ImmutableMap.Builder<CellName, ByteBuffer> builder = ImmutableMap.builder();
-        for (Column column : this)
-            builder.put(column.name, column.value);
+        for (Cell cell : this)
+            builder.put(cell.name, cell.value);
         return builder.build();
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/db/ColumnFamilySerializer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ColumnFamilySerializer.java b/src/java/org/apache/cassandra/db/ColumnFamilySerializer.java
index 13ec6fc..92aa955 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilySerializer.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilySerializer.java
@@ -72,9 +72,9 @@ public class ColumnFamilySerializer implements IVersionedSerializer<ColumnFamily
             int count = cf.getColumnCount();
             out.writeInt(count);
             int written = 0;
-            for (Column column : cf)
+            for (Cell cell : cf)
             {
-                columnSerializer.serialize(column, out);
+                columnSerializer.serialize(cell, out);
                 written++;
             }
             assert count == written: "Column family had " + count + " columns, but " + written + " written";
@@ -131,8 +131,8 @@ public class ColumnFamilySerializer implements IVersionedSerializer<ColumnFamily
             size += cf.getComparator().deletionInfoSerializer().serializedSize(cf.deletionInfo(), typeSizes, version);
             size += typeSizes.sizeof(cf.getColumnCount());
             ColumnSerializer columnSerializer = cf.getComparator().columnSerializer();
-            for (Column column : cf)
-                size += columnSerializer.serializedSize(column, typeSizes);
+            for (Cell cell : cf)
+                size += columnSerializer.serializedSize(cell, typeSizes);
         }
         return size;
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index f00e281..80f77bf 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@ -903,12 +903,12 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
      */
     public static ColumnFamily removeDeletedColumnsOnly(ColumnFamily cf, int gcBefore, SecondaryIndexManager.Updater indexer)
     {
-        Iterator<Column> iter = cf.iterator();
+        Iterator<Cell> iter = cf.iterator();
         DeletionInfo.InOrderTester tester = cf.inOrderDeletionTester();
         boolean hasDroppedColumns = !cf.metadata.getDroppedColumns().isEmpty();
         while (iter.hasNext())
         {
-            Column c = iter.next();
+            Cell c = iter.next();
             // remove columns if
             // (a) the column itself is gcable or
             // (b) the column is shadowed by a CF tombstone
@@ -926,7 +926,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
     // returns true if
     // 1. this column has been dropped from schema and
     // 2. if it has been re-added since then, this particular column was inserted before the last drop
-    private static boolean isDroppedColumn(Column c, CFMetaData meta)
+    private static boolean isDroppedColumn(Cell c, CFMetaData meta)
     {
         Long droppedAt = meta.getDroppedColumns().get(c.name().cql3ColumnName());
         return droppedAt != null && c.timestamp() <= droppedAt;
@@ -937,7 +937,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         if (cf == null || cf.metadata.getDroppedColumns().isEmpty())
             return;
 
-        Iterator<Column> iter = cf.iterator();
+        Iterator<Cell> iter = cf.iterator();
         while (iter.hasNext())
             if (isDroppedColumn(iter.next(), metadata))
                 iter.remove();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/db/ColumnIndex.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ColumnIndex.java b/src/java/org/apache/cassandra/db/ColumnIndex.java
index 3e6d55d..f00f958 100644
--- a/src/java/org/apache/cassandra/db/ColumnIndex.java
+++ b/src/java/org/apache/cassandra/db/ColumnIndex.java
@@ -126,7 +126,7 @@ public class ColumnIndex
             Iterator<RangeTombstone> rangeIter = cf.deletionInfo().rangeIterator();
             RangeTombstone tombstone = rangeIter.hasNext() ? rangeIter.next() : null;
 
-            for (Column c : cf)
+            for (Cell c : cf)
             {
                 while (tombstone != null && comparator.compare(c.name(), tombstone.min) >= 0)
                 {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/db/ColumnSerializer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ColumnSerializer.java b/src/java/org/apache/cassandra/db/ColumnSerializer.java
index a1c6ebd..3cfd900 100644
--- a/src/java/org/apache/cassandra/db/ColumnSerializer.java
+++ b/src/java/org/apache/cassandra/db/ColumnSerializer.java
@@ -30,7 +30,7 @@ import org.apache.cassandra.io.util.FileDataInput;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
-public class ColumnSerializer implements ISerializer<Column>
+public class ColumnSerializer implements ISerializer<Cell>
 {
     public final static int DELETION_MASK        = 0x01;
     public final static int EXPIRATION_MASK      = 0x02;
@@ -61,24 +61,24 @@ public class ColumnSerializer implements ISerializer<Column>
         this.type = type;
     }
 
-    public void serialize(Column column, DataOutput out) throws IOException
+    public void serialize(Cell cell, DataOutput out) throws IOException
     {
-        assert !column.name().isEmpty();
-        type.cellSerializer().serialize(column.name(), out);
+        assert !cell.name().isEmpty();
+        type.cellSerializer().serialize(cell.name(), out);
         try
         {
-            out.writeByte(column.serializationFlags());
-            if (column instanceof CounterColumn)
+            out.writeByte(cell.serializationFlags());
+            if (cell instanceof CounterCell)
             {
-                out.writeLong(((CounterColumn)column).timestampOfLastDelete());
+                out.writeLong(((CounterCell) cell).timestampOfLastDelete());
             }
-            else if (column instanceof ExpiringColumn)
+            else if (cell instanceof ExpiringCell)
             {
-                out.writeInt(((ExpiringColumn) column).getTimeToLive());
-                out.writeInt(column.getLocalDeletionTime());
+                out.writeInt(((ExpiringCell) cell).getTimeToLive());
+                out.writeInt(cell.getLocalDeletionTime());
             }
-            out.writeLong(column.timestamp());
-            ByteBufferUtil.writeWithLength(column.value(), out);
+            out.writeLong(cell.timestamp());
+            ByteBufferUtil.writeWithLength(cell.value(), out);
         }
         catch (IOException e)
         {
@@ -86,7 +86,7 @@ public class ColumnSerializer implements ISerializer<Column>
         }
     }
 
-    public Column deserialize(DataInput in) throws IOException
+    public Cell deserialize(DataInput in) throws IOException
     {
         return deserialize(in, Flag.LOCAL);
     }
@@ -96,12 +96,12 @@ public class ColumnSerializer implements ISerializer<Column>
      * deserialize comes from a remote host. If it does, then we must clear
      * the delta.
      */
-    public Column deserialize(DataInput in, ColumnSerializer.Flag flag) throws IOException
+    public Cell deserialize(DataInput in, ColumnSerializer.Flag flag) throws IOException
     {
         return deserialize(in, flag, Integer.MIN_VALUE);
     }
 
-    public Column deserialize(DataInput in, ColumnSerializer.Flag flag, int expireBefore) throws IOException
+    public Cell deserialize(DataInput in, ColumnSerializer.Flag flag, int expireBefore) throws IOException
     {
         CellName name = type.cellSerializer().deserialize(in);
 
@@ -109,14 +109,14 @@ public class ColumnSerializer implements ISerializer<Column>
         return deserializeColumnBody(in, name, b, flag, expireBefore);
     }
 
-    Column deserializeColumnBody(DataInput in, CellName name, int mask, ColumnSerializer.Flag flag, int expireBefore) throws IOException
+    Cell deserializeColumnBody(DataInput in, CellName name, int mask, ColumnSerializer.Flag flag, int expireBefore) throws IOException
     {
         if ((mask & COUNTER_MASK) != 0)
         {
             long timestampOfLastDelete = in.readLong();
             long ts = in.readLong();
             ByteBuffer value = ByteBufferUtil.readWithLength(in);
-            return CounterColumn.create(name, value, ts, timestampOfLastDelete, flag);
+            return CounterCell.create(name, value, ts, timestampOfLastDelete, flag);
         }
         else if ((mask & EXPIRATION_MASK) != 0)
         {
@@ -124,17 +124,17 @@ public class ColumnSerializer implements ISerializer<Column>
             int expiration = in.readInt();
             long ts = in.readLong();
             ByteBuffer value = ByteBufferUtil.readWithLength(in);
-            return ExpiringColumn.create(name, value, ts, ttl, expiration, expireBefore, flag);
+            return ExpiringCell.create(name, value, ts, ttl, expiration, expireBefore, flag);
         }
         else
         {
             long ts = in.readLong();
             ByteBuffer value = ByteBufferUtil.readWithLength(in);
             return (mask & COUNTER_UPDATE_MASK) != 0
-                   ? new CounterUpdateColumn(name, value, ts)
+                   ? new CounterUpdateCell(name, value, ts)
                    : ((mask & DELETION_MASK) == 0
-                      ? new Column(name, value, ts)
-                      : new DeletedColumn(name, value, ts));
+                      ? new Cell(name, value, ts)
+                      : new DeletedCell(name, value, ts));
         }
     }
 
@@ -151,9 +151,9 @@ public class ColumnSerializer implements ISerializer<Column>
         FileUtils.skipBytesFully(in, length);
     }
 
-    public long serializedSize(Column column, TypeSizes typeSizes)
+    public long serializedSize(Cell cell, TypeSizes typeSizes)
     {
-        return column.serializedSize(type, typeSizes);
+        return cell.serializedSize(type, typeSizes);
     }
 
     public static class CorruptColumnException extends IOException

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/db/CounterCell.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/CounterCell.java b/src/java/org/apache/cassandra/db/CounterCell.java
new file mode 100644
index 0000000..0391eb8
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/CounterCell.java
@@ -0,0 +1,372 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.db;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.net.InetAddress;
+import java.security.MessageDigest;
+import java.util.Set;
+
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Sets;
+import org.apache.cassandra.serializers.MarshalException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.composites.CellName;
+import org.apache.cassandra.db.composites.CellNameType;
+import org.apache.cassandra.db.context.CounterContext;
+import org.apache.cassandra.db.context.IContext.ContextRelationship;
+import org.apache.cassandra.exceptions.OverloadedException;
+import org.apache.cassandra.exceptions.RequestExecutionException;
+import org.apache.cassandra.io.util.DataOutputBuffer;
+import org.apache.cassandra.utils.Allocator;
+import org.apache.cassandra.service.AbstractWriteResponseHandler;
+import org.apache.cassandra.service.StorageProxy;
+import org.apache.cassandra.utils.*;
+
+/**
+ * A column that represents a partitioned counter.
+ */
+public class CounterCell extends Cell
+{
+    private static final Logger logger = LoggerFactory.getLogger(CounterCell.class);
+
+    protected static final CounterContext contextManager = CounterContext.instance();
+
+    private final long timestampOfLastDelete;
+
+    public CounterCell(CellName name, long value, long timestamp)
+    {
+        this(name, contextManager.create(value, HeapAllocator.instance), timestamp);
+    }
+
+    public CounterCell(CellName name, long value, long timestamp, long timestampOfLastDelete)
+    {
+        this(name, contextManager.create(value, HeapAllocator.instance), timestamp, timestampOfLastDelete);
+    }
+
+    public CounterCell(CellName name, ByteBuffer value, long timestamp)
+    {
+        this(name, value, timestamp, Long.MIN_VALUE);
+    }
+
+    public CounterCell(CellName name, ByteBuffer value, long timestamp, long timestampOfLastDelete)
+    {
+        super(name, value, timestamp);
+        this.timestampOfLastDelete = timestampOfLastDelete;
+    }
+
+    public static CounterCell create(CellName name, ByteBuffer value, long timestamp, long timestampOfLastDelete, ColumnSerializer.Flag flag)
+    {
+        // #elt being negative means we have to clean delta
+        short count = value.getShort(value.position());
+        if (flag == ColumnSerializer.Flag.FROM_REMOTE || (flag == ColumnSerializer.Flag.LOCAL && count < 0))
+            value = CounterContext.instance().clearAllDelta(value);
+        return new CounterCell(name, value, timestamp, timestampOfLastDelete);
+    }
+
+    @Override
+    public Cell withUpdatedName(CellName newName)
+    {
+        return new CounterCell(newName, value, timestamp, timestampOfLastDelete);
+    }
+
+    public long timestampOfLastDelete()
+    {
+        return timestampOfLastDelete;
+    }
+
+    public long total()
+    {
+        return contextManager.total(value);
+    }
+
+    @Override
+    public int dataSize()
+    {
+        /*
+         * A counter column adds to a Cell :
+         *  + 8 bytes for timestampOfLastDelete
+         */
+        return super.dataSize() + TypeSizes.NATIVE.sizeof(timestampOfLastDelete);
+    }
+
+    @Override
+    public int serializedSize(CellNameType type, TypeSizes typeSizes)
+    {
+        return super.serializedSize(type, typeSizes) + typeSizes.sizeof(timestampOfLastDelete);
+    }
+
+    @Override
+    public Cell diff(Cell cell)
+    {
+        assert (cell instanceof CounterCell) || (cell instanceof DeletedCell) : "Wrong class type: " + cell.getClass();
+
+        if (timestamp() < cell.timestamp())
+            return cell;
+
+        // Note that if at that point, cell can't be a tombstone. Indeed,
+        // cell is the result of merging us with other nodes results, and
+        // merging a CounterCell with a tombstone never return a tombstone
+        // unless that tombstone timestamp is greater that the CounterCell
+        // one.
+        assert !(cell instanceof DeletedCell) : "Wrong class type: " + cell.getClass();
+
+        if (timestampOfLastDelete() < ((CounterCell) cell).timestampOfLastDelete())
+            return cell;
+        ContextRelationship rel = contextManager.diff(cell.value(), value());
+        if (ContextRelationship.GREATER_THAN == rel || ContextRelationship.DISJOINT == rel)
+            return cell;
+        return null;
+    }
+
+    /*
+     * We have to special case digest creation for counter column because
+     * we don't want to include the information about which shard of the
+     * context is a delta or not, since this information differs from node to
+     * node.
+     */
+    @Override
+    public void updateDigest(MessageDigest digest)
+    {
+        digest.update(name.toByteBuffer().duplicate());
+        // We don't take the deltas into account in a digest
+        contextManager.updateDigest(digest, value);
+        DataOutputBuffer buffer = new DataOutputBuffer();
+        try
+        {
+            buffer.writeLong(timestamp);
+            buffer.writeByte(serializationFlags());
+            buffer.writeLong(timestampOfLastDelete);
+        }
+        catch (IOException e)
+        {
+            throw new RuntimeException(e);
+        }
+        digest.update(buffer.getData(), 0, buffer.getLength());
+    }
+
+    @Override
+    public Cell reconcile(Cell cell, Allocator allocator)
+    {
+        assert (cell instanceof CounterCell) || (cell instanceof DeletedCell) : "Wrong class type: " + cell.getClass();
+
+        // live + tombstone: track last tombstone
+        if (cell.isMarkedForDelete(Long.MIN_VALUE)) // cannot be an expired cell, so the current time is irrelevant
+        {
+            // live < tombstone
+            if (timestamp() < cell.timestamp())
+            {
+                return cell;
+            }
+            // live last delete >= tombstone
+            if (timestampOfLastDelete() >= cell.timestamp())
+            {
+                return this;
+            }
+            // live last delete < tombstone
+            return new CounterCell(name(), value(), timestamp(), cell.timestamp());
+        }
+        // live < live last delete
+        if (timestamp() < ((CounterCell) cell).timestampOfLastDelete())
+            return cell;
+        // live last delete > live
+        if (timestampOfLastDelete() > cell.timestamp())
+            return this;
+        // live + live: merge clocks; update value
+        return new CounterCell(
+            name(),
+            contextManager.merge(value(), cell.value(), allocator),
+            Math.max(timestamp(), cell.timestamp()),
+            Math.max(timestampOfLastDelete(), ((CounterCell) cell).timestampOfLastDelete()));
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+        // super.equals() returns false if o is not a CounterCell
+        return super.equals(o) && timestampOfLastDelete == ((CounterCell)o).timestampOfLastDelete;
+    }
+
+    @Override
+    public int hashCode()
+    {
+        int result = super.hashCode();
+        result = 31 * result + (int)(timestampOfLastDelete ^ (timestampOfLastDelete >>> 32));
+        return result;
+    }
+
+    @Override
+    public Cell localCopy(ColumnFamilyStore cfs)
+    {
+        return localCopy(cfs, HeapAllocator.instance);
+    }
+
+    @Override
+    public Cell localCopy(ColumnFamilyStore cfs, Allocator allocator)
+    {
+        return new CounterCell(name.copy(allocator), allocator.clone(value), timestamp, timestampOfLastDelete);
+    }
+
+    @Override
+    public String getString(CellNameType comparator)
+    {
+        StringBuilder sb = new StringBuilder();
+        sb.append(comparator.getString(name));
+        sb.append(":");
+        sb.append(false);
+        sb.append(":");
+        sb.append(contextManager.toString(value));
+        sb.append("@");
+        sb.append(timestamp());
+        sb.append("!");
+        sb.append(timestampOfLastDelete);
+        return sb.toString();
+    }
+
+    @Override
+    public int serializationFlags()
+    {
+        return ColumnSerializer.COUNTER_MASK;
+    }
+
+    @Override
+    public void validateFields(CFMetaData metadata) throws MarshalException
+    {
+        validateName(metadata);
+        // We cannot use the value validator as for other columns as the CounterColumnType validate a long,
+        // which is not the internal representation of counters
+        contextManager.validateContext(value());
+    }
+
+    /**
+     * Check if a given counterId is found in this CounterCell context.
+     */
+    public boolean hasCounterId(CounterId id)
+    {
+        return contextManager.hasCounterId(value(), id);
+    }
+
+    private CounterCell computeOldShardMerger(int mergeBefore)
+    {
+        ByteBuffer bb = contextManager.computeOldShardMerger(value(), CounterId.getOldLocalCounterIds(), mergeBefore);
+        if (bb == null)
+            return null;
+        else
+            return new CounterCell(name(), bb, timestamp(), timestampOfLastDelete);
+    }
+
+    private CounterCell removeOldShards(int gcBefore)
+    {
+        ByteBuffer bb = contextManager.removeOldShards(value(), gcBefore);
+        if (bb == value())
+            return this;
+        else
+        {
+            return new CounterCell(name(), bb, timestamp(), timestampOfLastDelete);
+        }
+    }
+
+    public static void mergeAndRemoveOldShards(DecoratedKey key, ColumnFamily cf, int gcBefore, int mergeBefore)
+    {
+        mergeAndRemoveOldShards(key, cf, gcBefore, mergeBefore, true);
+    }
+
+    /**
+     * There is two phase to the removal of old shards.
+     * First phase: we merge the old shard value to the current shard and
+     * 'nulify' the old one. We then send the counter context with the old
+     * shard nulified to all other replica.
+     * Second phase: once an old shard has been nulified for longer than
+     * gc_grace (to be sure all other replica had been aware of the merge), we
+     * simply remove that old shard from the context (it's value is 0).
+     * This method does both phases.
+     * (Note that the sendToOtherReplica flag is here only to facilitate
+     * testing. It should be true in real code so use the method above
+     * preferably)
+     */
+    public static void mergeAndRemoveOldShards(DecoratedKey key, ColumnFamily cf, int gcBefore, int mergeBefore, boolean sendToOtherReplica)
+    {
+        ColumnFamily remoteMerger = null;
+
+        for (Cell c : cf)
+        {
+            if (!(c instanceof CounterCell))
+                continue;
+            CounterCell cc = (CounterCell) c;
+            CounterCell shardMerger = cc.computeOldShardMerger(mergeBefore);
+            CounterCell merged = cc;
+            if (shardMerger != null)
+            {
+                merged = (CounterCell) cc.reconcile(shardMerger);
+                if (remoteMerger == null)
+                    remoteMerger = cf.cloneMeShallow();
+                remoteMerger.addColumn(merged);
+            }
+            CounterCell cleaned = merged.removeOldShards(gcBefore);
+            if (cleaned != cc)
+            {
+                cf.replace(cc, cleaned);
+            }
+        }
+
+        if (remoteMerger != null && sendToOtherReplica)
+        {
+            try
+            {
+                sendToOtherReplica(key, remoteMerger);
+            }
+            catch (Exception e)
+            {
+                logger.error("Error while sending shard merger mutation to remote endpoints", e);
+            }
+        }
+    }
+
+    public Cell markDeltaToBeCleared()
+    {
+        return new CounterCell(name, contextManager.markDeltaToBeCleared(value), timestamp, timestampOfLastDelete);
+    }
+
+    private static void sendToOtherReplica(DecoratedKey key, ColumnFamily cf) throws RequestExecutionException
+    {
+        RowMutation rm = new RowMutation(cf.metadata().ksName, key.key, cf);
+
+        final InetAddress local = FBUtilities.getBroadcastAddress();
+        String localDataCenter = DatabaseDescriptor.getEndpointSnitch().getDatacenter(local);
+
+        StorageProxy.performWrite(rm, ConsistencyLevel.ANY, localDataCenter, new StorageProxy.WritePerformer()
+        {
+            public void apply(IMutation mutation, Iterable<InetAddress> targets, AbstractWriteResponseHandler responseHandler, String localDataCenter, ConsistencyLevel consistency_level)
+            throws OverloadedException
+            {
+                // We should only send to the remote replica, not the local one
+                Set<InetAddress> remotes = Sets.difference(ImmutableSet.copyOf(targets), ImmutableSet.of(local));
+                // Fake local response to be a good lad but we won't wait on the responseHandler
+                responseHandler.response(null);
+                StorageProxy.sendToHintedEndpoints((RowMutation) mutation, remotes, responseHandler, localDataCenter);
+            }
+        }, null, WriteType.SIMPLE);
+
+        // we don't wait for answers
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/db/CounterColumn.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/CounterColumn.java b/src/java/org/apache/cassandra/db/CounterColumn.java
deleted file mode 100644
index ac2c88e..0000000
--- a/src/java/org/apache/cassandra/db/CounterColumn.java
+++ /dev/null
@@ -1,372 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.db;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.net.InetAddress;
-import java.security.MessageDigest;
-import java.util.Set;
-
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Sets;
-import org.apache.cassandra.serializers.MarshalException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.db.composites.CellName;
-import org.apache.cassandra.db.composites.CellNameType;
-import org.apache.cassandra.db.context.CounterContext;
-import org.apache.cassandra.db.context.IContext.ContextRelationship;
-import org.apache.cassandra.exceptions.OverloadedException;
-import org.apache.cassandra.exceptions.RequestExecutionException;
-import org.apache.cassandra.io.util.DataOutputBuffer;
-import org.apache.cassandra.utils.Allocator;
-import org.apache.cassandra.service.AbstractWriteResponseHandler;
-import org.apache.cassandra.service.StorageProxy;
-import org.apache.cassandra.utils.*;
-
-/**
- * A column that represents a partitioned counter.
- */
-public class CounterColumn extends Column
-{
-    private static final Logger logger = LoggerFactory.getLogger(CounterColumn.class);
-
-    protected static final CounterContext contextManager = CounterContext.instance();
-
-    private final long timestampOfLastDelete;
-
-    public CounterColumn(CellName name, long value, long timestamp)
-    {
-        this(name, contextManager.create(value, HeapAllocator.instance), timestamp);
-    }
-
-    public CounterColumn(CellName name, long value, long timestamp, long timestampOfLastDelete)
-    {
-        this(name, contextManager.create(value, HeapAllocator.instance), timestamp, timestampOfLastDelete);
-    }
-
-    public CounterColumn(CellName name, ByteBuffer value, long timestamp)
-    {
-        this(name, value, timestamp, Long.MIN_VALUE);
-    }
-
-    public CounterColumn(CellName name, ByteBuffer value, long timestamp, long timestampOfLastDelete)
-    {
-        super(name, value, timestamp);
-        this.timestampOfLastDelete = timestampOfLastDelete;
-    }
-
-    public static CounterColumn create(CellName name, ByteBuffer value, long timestamp, long timestampOfLastDelete, ColumnSerializer.Flag flag)
-    {
-        // #elt being negative means we have to clean delta
-        short count = value.getShort(value.position());
-        if (flag == ColumnSerializer.Flag.FROM_REMOTE || (flag == ColumnSerializer.Flag.LOCAL && count < 0))
-            value = CounterContext.instance().clearAllDelta(value);
-        return new CounterColumn(name, value, timestamp, timestampOfLastDelete);
-    }
-
-    @Override
-    public Column withUpdatedName(CellName newName)
-    {
-        return new CounterColumn(newName, value, timestamp, timestampOfLastDelete);
-    }
-
-    public long timestampOfLastDelete()
-    {
-        return timestampOfLastDelete;
-    }
-
-    public long total()
-    {
-        return contextManager.total(value);
-    }
-
-    @Override
-    public int dataSize()
-    {
-        /*
-         * A counter column adds to a Column :
-         *  + 8 bytes for timestampOfLastDelete
-         */
-        return super.dataSize() + TypeSizes.NATIVE.sizeof(timestampOfLastDelete);
-    }
-
-    @Override
-    public int serializedSize(CellNameType type, TypeSizes typeSizes)
-    {
-        return super.serializedSize(type, typeSizes) + typeSizes.sizeof(timestampOfLastDelete);
-    }
-
-    @Override
-    public Column diff(Column column)
-    {
-        assert (column instanceof CounterColumn) || (column instanceof DeletedColumn) : "Wrong class type: " + column.getClass();
-
-        if (timestamp() < column.timestamp())
-            return column;
-
-        // Note that if at that point, column can't be a tombstone. Indeed,
-        // column is the result of merging us with other nodes results, and
-        // merging a CounterColumn with a tombstone never return a tombstone
-        // unless that tombstone timestamp is greater that the CounterColumn
-        // one.
-        assert !(column instanceof DeletedColumn) : "Wrong class type: " + column.getClass();
-
-        if (timestampOfLastDelete() < ((CounterColumn)column).timestampOfLastDelete())
-            return column;
-        ContextRelationship rel = contextManager.diff(column.value(), value());
-        if (ContextRelationship.GREATER_THAN == rel || ContextRelationship.DISJOINT == rel)
-            return column;
-        return null;
-    }
-
-    /*
-     * We have to special case digest creation for counter column because
-     * we don't want to include the information about which shard of the
-     * context is a delta or not, since this information differs from node to
-     * node.
-     */
-    @Override
-    public void updateDigest(MessageDigest digest)
-    {
-        digest.update(name.toByteBuffer().duplicate());
-        // We don't take the deltas into account in a digest
-        contextManager.updateDigest(digest, value);
-        DataOutputBuffer buffer = new DataOutputBuffer();
-        try
-        {
-            buffer.writeLong(timestamp);
-            buffer.writeByte(serializationFlags());
-            buffer.writeLong(timestampOfLastDelete);
-        }
-        catch (IOException e)
-        {
-            throw new RuntimeException(e);
-        }
-        digest.update(buffer.getData(), 0, buffer.getLength());
-    }
-
-    @Override
-    public Column reconcile(Column column, Allocator allocator)
-    {
-        assert (column instanceof CounterColumn) || (column instanceof DeletedColumn) : "Wrong class type: " + column.getClass();
-
-        // live + tombstone: track last tombstone
-        if (column.isMarkedForDelete(Long.MIN_VALUE)) // cannot be an expired column, so the current time is irrelevant
-        {
-            // live < tombstone
-            if (timestamp() < column.timestamp())
-            {
-                return column;
-            }
-            // live last delete >= tombstone
-            if (timestampOfLastDelete() >= column.timestamp())
-            {
-                return this;
-            }
-            // live last delete < tombstone
-            return new CounterColumn(name(), value(), timestamp(), column.timestamp());
-        }
-        // live < live last delete
-        if (timestamp() < ((CounterColumn)column).timestampOfLastDelete())
-            return column;
-        // live last delete > live
-        if (timestampOfLastDelete() > column.timestamp())
-            return this;
-        // live + live: merge clocks; update value
-        return new CounterColumn(
-            name(),
-            contextManager.merge(value(), column.value(), allocator),
-            Math.max(timestamp(), column.timestamp()),
-            Math.max(timestampOfLastDelete(), ((CounterColumn)column).timestampOfLastDelete()));
-    }
-
-    @Override
-    public boolean equals(Object o)
-    {
-        // super.equals() returns false if o is not a CounterColumn
-        return super.equals(o) && timestampOfLastDelete == ((CounterColumn)o).timestampOfLastDelete;
-    }
-
-    @Override
-    public int hashCode()
-    {
-        int result = super.hashCode();
-        result = 31 * result + (int)(timestampOfLastDelete ^ (timestampOfLastDelete >>> 32));
-        return result;
-    }
-
-    @Override
-    public Column localCopy(ColumnFamilyStore cfs)
-    {
-        return localCopy(cfs, HeapAllocator.instance);
-    }
-
-    @Override
-    public Column localCopy(ColumnFamilyStore cfs, Allocator allocator)
-    {
-        return new CounterColumn(name.copy(allocator), allocator.clone(value), timestamp, timestampOfLastDelete);
-    }
-
-    @Override
-    public String getString(CellNameType comparator)
-    {
-        StringBuilder sb = new StringBuilder();
-        sb.append(comparator.getString(name));
-        sb.append(":");
-        sb.append(false);
-        sb.append(":");
-        sb.append(contextManager.toString(value));
-        sb.append("@");
-        sb.append(timestamp());
-        sb.append("!");
-        sb.append(timestampOfLastDelete);
-        return sb.toString();
-    }
-
-    @Override
-    public int serializationFlags()
-    {
-        return ColumnSerializer.COUNTER_MASK;
-    }
-
-    @Override
-    public void validateFields(CFMetaData metadata) throws MarshalException
-    {
-        validateName(metadata);
-        // We cannot use the value validator as for other columns as the CounterColumnType validate a long,
-        // which is not the internal representation of counters
-        contextManager.validateContext(value());
-    }
-
-    /**
-     * Check if a given counterId is found in this CounterColumn context.
-     */
-    public boolean hasCounterId(CounterId id)
-    {
-        return contextManager.hasCounterId(value(), id);
-    }
-
-    private CounterColumn computeOldShardMerger(int mergeBefore)
-    {
-        ByteBuffer bb = contextManager.computeOldShardMerger(value(), CounterId.getOldLocalCounterIds(), mergeBefore);
-        if (bb == null)
-            return null;
-        else
-            return new CounterColumn(name(), bb, timestamp(), timestampOfLastDelete);
-    }
-
-    private CounterColumn removeOldShards(int gcBefore)
-    {
-        ByteBuffer bb = contextManager.removeOldShards(value(), gcBefore);
-        if (bb == value())
-            return this;
-        else
-        {
-            return new CounterColumn(name(), bb, timestamp(), timestampOfLastDelete);
-        }
-    }
-
-    public static void mergeAndRemoveOldShards(DecoratedKey key, ColumnFamily cf, int gcBefore, int mergeBefore)
-    {
-        mergeAndRemoveOldShards(key, cf, gcBefore, mergeBefore, true);
-    }
-
-    /**
-     * There is two phase to the removal of old shards.
-     * First phase: we merge the old shard value to the current shard and
-     * 'nulify' the old one. We then send the counter context with the old
-     * shard nulified to all other replica.
-     * Second phase: once an old shard has been nulified for longer than
-     * gc_grace (to be sure all other replica had been aware of the merge), we
-     * simply remove that old shard from the context (it's value is 0).
-     * This method does both phases.
-     * (Note that the sendToOtherReplica flag is here only to facilitate
-     * testing. It should be true in real code so use the method above
-     * preferably)
-     */
-    public static void mergeAndRemoveOldShards(DecoratedKey key, ColumnFamily cf, int gcBefore, int mergeBefore, boolean sendToOtherReplica)
-    {
-        ColumnFamily remoteMerger = null;
-
-        for (Column c : cf)
-        {
-            if (!(c instanceof CounterColumn))
-                continue;
-            CounterColumn cc = (CounterColumn) c;
-            CounterColumn shardMerger = cc.computeOldShardMerger(mergeBefore);
-            CounterColumn merged = cc;
-            if (shardMerger != null)
-            {
-                merged = (CounterColumn) cc.reconcile(shardMerger);
-                if (remoteMerger == null)
-                    remoteMerger = cf.cloneMeShallow();
-                remoteMerger.addColumn(merged);
-            }
-            CounterColumn cleaned = merged.removeOldShards(gcBefore);
-            if (cleaned != cc)
-            {
-                cf.replace(cc, cleaned);
-            }
-        }
-
-        if (remoteMerger != null && sendToOtherReplica)
-        {
-            try
-            {
-                sendToOtherReplica(key, remoteMerger);
-            }
-            catch (Exception e)
-            {
-                logger.error("Error while sending shard merger mutation to remote endpoints", e);
-            }
-        }
-    }
-
-    public Column markDeltaToBeCleared()
-    {
-        return new CounterColumn(name, contextManager.markDeltaToBeCleared(value), timestamp, timestampOfLastDelete);
-    }
-
-    private static void sendToOtherReplica(DecoratedKey key, ColumnFamily cf) throws RequestExecutionException
-    {
-        RowMutation rm = new RowMutation(cf.metadata().ksName, key.key, cf);
-
-        final InetAddress local = FBUtilities.getBroadcastAddress();
-        String localDataCenter = DatabaseDescriptor.getEndpointSnitch().getDatacenter(local);
-
-        StorageProxy.performWrite(rm, ConsistencyLevel.ANY, localDataCenter, new StorageProxy.WritePerformer()
-        {
-            public void apply(IMutation mutation, Iterable<InetAddress> targets, AbstractWriteResponseHandler responseHandler, String localDataCenter, ConsistencyLevel consistency_level)
-            throws OverloadedException
-            {
-                // We should only send to the remote replica, not the local one
-                Set<InetAddress> remotes = Sets.difference(ImmutableSet.copyOf(targets), ImmutableSet.of(local));
-                // Fake local response to be a good lad but we won't wait on the responseHandler
-                responseHandler.response(null);
-                StorageProxy.sendToHintedEndpoints((RowMutation) mutation, remotes, responseHandler, localDataCenter);
-            }
-        }, null, WriteType.SIMPLE);
-
-        // we don't wait for answers
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/db/CounterMutation.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/CounterMutation.java b/src/java/org/apache/cassandra/db/CounterMutation.java
index eae8e12..f0942e2 100644
--- a/src/java/org/apache/cassandra/db/CounterMutation.java
+++ b/src/java/org/apache/cassandra/db/CounterMutation.java
@@ -129,7 +129,7 @@ public class CounterMutation implements IMutation
 
     public void apply()
     {
-        // transform all CounterUpdateColumn to CounterColumn: accomplished by localCopy
+        // transform all CounterUpdateCell to CounterCell: accomplished by localCopy
         RowMutation rm = new RowMutation(rowMutation.getKeyspaceName(), ByteBufferUtil.clone(rowMutation.key()));
         Keyspace keyspace = Keyspace.open(rm.getKeyspaceName());
 
@@ -137,9 +137,9 @@ public class CounterMutation implements IMutation
         {
             ColumnFamily cf = cf_.cloneMeShallow();
             ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cf.id());
-            for (Column column : cf_)
+            for (Cell cell : cf_)
             {
-                cf.addColumn(column.localCopy(cfs), HeapAllocator.instance);
+                cf.addColumn(cell.localCopy(cfs), HeapAllocator.instance);
             }
             rm.add(cf);
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/db/CounterUpdateCell.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/CounterUpdateCell.java b/src/java/org/apache/cassandra/db/CounterUpdateCell.java
new file mode 100644
index 0000000..8acb854
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/CounterUpdateCell.java
@@ -0,0 +1,99 @@
+/*
+ * 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 org.apache.cassandra.db.composites.CellName;
+import org.apache.cassandra.db.context.CounterContext;
+import org.apache.cassandra.utils.Allocator;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.HeapAllocator;
+
+/**
+ * A counter update while it hasn't been applied yet by the leader replica.
+ *
+ * Contains a single counter update. When applied by the leader replica, this
+ * is transformed to a relevant CounterCell. This Cell is a temporary data
+ * structure that should never be stored inside a memtable or an sstable.
+ */
+public class CounterUpdateCell extends Cell
+{
+    public CounterUpdateCell(CellName name, long value, long timestamp)
+    {
+        this(name, ByteBufferUtil.bytes(value), timestamp);
+    }
+
+    public CounterUpdateCell(CellName name, ByteBuffer value, long timestamp)
+    {
+        super(name, value, timestamp);
+    }
+
+    public long delta()
+    {
+        return value().getLong(value().position());
+    }
+
+    @Override
+    public Cell diff(Cell cell)
+    {
+        // Diff is used during reads, but we should never read those columns
+        throw new UnsupportedOperationException("This operation is unsupported on CounterUpdateCell.");
+    }
+
+    @Override
+    public Cell reconcile(Cell cell, Allocator allocator)
+    {
+        // The only time this could happen is if a batchAdd ships two
+        // increment for the same cell. Hence we simply sums the delta.
+
+        assert (cell instanceof CounterUpdateCell) || (cell instanceof DeletedCell) : "Wrong class type.";
+
+        // tombstones take precedence
+        if (cell.isMarkedForDelete(Long.MIN_VALUE)) // can't be an expired cell, so the current time is irrelevant
+            return timestamp() > cell.timestamp() ? this : cell;
+
+        // neither is tombstoned
+        CounterUpdateCell c = (CounterUpdateCell) cell;
+        return new CounterUpdateCell(name(), delta() + c.delta(), Math.max(timestamp(), c.timestamp()));
+    }
+
+    @Override
+    public int serializationFlags()
+    {
+        return ColumnSerializer.COUNTER_UPDATE_MASK;
+    }
+
+    @Override
+    public CounterCell localCopy(ColumnFamilyStore cfs)
+    {
+        return new CounterCell(name.copy(HeapAllocator.instance),
+                                 CounterContext.instance().create(delta(), HeapAllocator.instance),
+                                 timestamp(),
+                                 Long.MIN_VALUE);
+    }
+
+    @Override
+    public Cell localCopy(ColumnFamilyStore cfs, Allocator allocator)
+    {
+        return new CounterCell(name.copy(allocator),
+                                 CounterContext.instance().create(delta(), allocator),
+                                 timestamp(),
+                                 Long.MIN_VALUE);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/db/CounterUpdateColumn.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/CounterUpdateColumn.java b/src/java/org/apache/cassandra/db/CounterUpdateColumn.java
deleted file mode 100644
index aaf3307..0000000
--- a/src/java/org/apache/cassandra/db/CounterUpdateColumn.java
+++ /dev/null
@@ -1,99 +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.nio.ByteBuffer;
-
-import org.apache.cassandra.db.composites.CellName;
-import org.apache.cassandra.db.context.CounterContext;
-import org.apache.cassandra.utils.Allocator;
-import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.HeapAllocator;
-
-/**
- * A counter update while it hasn't been applied yet by the leader replica.
- *
- * Contains a single counter update. When applied by the leader replica, this
- * is transformed to a relevant CounterColumn. This Column is a temporary data
- * structure that should never be stored inside a memtable or an sstable.
- */
-public class CounterUpdateColumn extends Column
-{
-    public CounterUpdateColumn(CellName name, long value, long timestamp)
-    {
-        this(name, ByteBufferUtil.bytes(value), timestamp);
-    }
-
-    public CounterUpdateColumn(CellName name, ByteBuffer value, long timestamp)
-    {
-        super(name, value, timestamp);
-    }
-
-    public long delta()
-    {
-        return value().getLong(value().position());
-    }
-
-    @Override
-    public Column diff(Column column)
-    {
-        // Diff is used during reads, but we should never read those columns
-        throw new UnsupportedOperationException("This operation is unsupported on CounterUpdateColumn.");
-    }
-
-    @Override
-    public Column reconcile(Column column, Allocator allocator)
-    {
-        // The only time this could happen is if a batchAdd ships two
-        // increment for the same column. Hence we simply sums the delta.
-
-        assert (column instanceof CounterUpdateColumn) || (column instanceof DeletedColumn) : "Wrong class type.";
-
-        // tombstones take precedence
-        if (column.isMarkedForDelete(Long.MIN_VALUE)) // can't be an expired column, so the current time is irrelevant
-            return timestamp() > column.timestamp() ? this : column;
-
-        // neither is tombstoned
-        CounterUpdateColumn c = (CounterUpdateColumn)column;
-        return new CounterUpdateColumn(name(), delta() + c.delta(), Math.max(timestamp(), c.timestamp()));
-    }
-
-    @Override
-    public int serializationFlags()
-    {
-        return ColumnSerializer.COUNTER_UPDATE_MASK;
-    }
-
-    @Override
-    public CounterColumn localCopy(ColumnFamilyStore cfs)
-    {
-        return new CounterColumn(name.copy(HeapAllocator.instance),
-                                 CounterContext.instance().create(delta(), HeapAllocator.instance),
-                                 timestamp(),
-                                 Long.MIN_VALUE);
-    }
-
-    @Override
-    public Column localCopy(ColumnFamilyStore cfs, Allocator allocator)
-    {
-        return new CounterColumn(name.copy(allocator),
-                                 CounterContext.instance().create(delta(), allocator),
-                                 timestamp(),
-                                 Long.MIN_VALUE);
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/db/DefsTables.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/DefsTables.java b/src/java/org/apache/cassandra/db/DefsTables.java
index f30e256..693ef97 100644
--- a/src/java/org/apache/cassandra/db/DefsTables.java
+++ b/src/java/org/apache/cassandra/db/DefsTables.java
@@ -74,7 +74,7 @@ import org.apache.cassandra.utils.ByteBufferUtil;
  *
  * Where <key> is a name of keyspace e.g. "ks".
  *
- * Column names where made composite to support 3-level nesting which represents following structure:
+ * Cell names where made composite to support 3-level nesting which represents following structure:
  * "ColumnFamily name":"column name":"column attribute" => "value"
  *
  * Example of schema (using CLI):

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/db/DeletedCell.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/DeletedCell.java b/src/java/org/apache/cassandra/db/DeletedCell.java
new file mode 100644
index 0000000..5b89e1d
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/DeletedCell.java
@@ -0,0 +1,127 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.db;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.security.MessageDigest;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.db.composites.CellName;
+import org.apache.cassandra.io.util.DataOutputBuffer;
+import org.apache.cassandra.serializers.MarshalException;
+import org.apache.cassandra.utils.Allocator;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.HeapAllocator;
+
+public class DeletedCell extends Cell
+{
+    public DeletedCell(CellName name, int localDeletionTime, long timestamp)
+    {
+        this(name, ByteBufferUtil.bytes(localDeletionTime), timestamp);
+    }
+
+    public DeletedCell(CellName name, ByteBuffer value, long timestamp)
+    {
+        super(name, value, timestamp);
+    }
+
+    @Override
+    public Cell withUpdatedName(CellName newName)
+    {
+        return new DeletedCell(newName, value, timestamp);
+    }
+
+    @Override
+    public Cell withUpdatedTimestamp(long newTimestamp)
+    {
+        return new DeletedCell(name, value, newTimestamp);
+    }
+
+    @Override
+    public boolean isMarkedForDelete(long now)
+    {
+        return true;
+    }
+
+    @Override
+    public long getMarkedForDeleteAt()
+    {
+        return timestamp;
+    }
+
+    @Override
+    public void updateDigest(MessageDigest digest)
+    {
+        digest.update(name.toByteBuffer().duplicate());
+
+        DataOutputBuffer buffer = new DataOutputBuffer();
+        try
+        {
+            buffer.writeLong(timestamp);
+            buffer.writeByte(serializationFlags());
+        }
+        catch (IOException e)
+        {
+            throw new RuntimeException(e);
+        }
+        digest.update(buffer.getData(), 0, buffer.getLength());
+    }
+
+    @Override
+    public int getLocalDeletionTime()
+    {
+       return value.getInt(value.position());
+    }
+
+    @Override
+    public Cell reconcile(Cell cell, Allocator allocator)
+    {
+        if (cell instanceof DeletedCell)
+            return super.reconcile(cell, allocator);
+        return cell.reconcile(this, allocator);
+    }
+
+    @Override
+    public Cell localCopy(ColumnFamilyStore cfs)
+    {
+        return new DeletedCell(name.copy(HeapAllocator.instance), ByteBufferUtil.clone(value), timestamp);
+    }
+
+    @Override
+    public Cell localCopy(ColumnFamilyStore cfs, Allocator allocator)
+    {
+        return new DeletedCell(name.copy(allocator), allocator.clone(value), timestamp);
+    }
+
+    @Override
+    public int serializationFlags()
+    {
+        return ColumnSerializer.DELETION_MASK;
+    }
+
+    @Override
+    public void validateFields(CFMetaData metadata) throws MarshalException
+    {
+        validateName(metadata);
+        if (value().remaining() != 4)
+            throw new MarshalException("A tombstone value should be 4 bytes long");
+        if (getLocalDeletionTime() < 0)
+            throw new MarshalException("The local deletion time should not be negative");
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/db/DeletedColumn.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/DeletedColumn.java b/src/java/org/apache/cassandra/db/DeletedColumn.java
deleted file mode 100644
index ec88015..0000000
--- a/src/java/org/apache/cassandra/db/DeletedColumn.java
+++ /dev/null
@@ -1,127 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.db;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.security.MessageDigest;
-
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.db.composites.CellName;
-import org.apache.cassandra.io.util.DataOutputBuffer;
-import org.apache.cassandra.serializers.MarshalException;
-import org.apache.cassandra.utils.Allocator;
-import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.HeapAllocator;
-
-public class DeletedColumn extends Column
-{
-    public DeletedColumn(CellName name, int localDeletionTime, long timestamp)
-    {
-        this(name, ByteBufferUtil.bytes(localDeletionTime), timestamp);
-    }
-
-    public DeletedColumn(CellName name, ByteBuffer value, long timestamp)
-    {
-        super(name, value, timestamp);
-    }
-
-    @Override
-    public Column withUpdatedName(CellName newName)
-    {
-        return new DeletedColumn(newName, value, timestamp);
-    }
-
-    @Override
-    public Column withUpdatedTimestamp(long newTimestamp)
-    {
-        return new DeletedColumn(name, value, newTimestamp);
-    }
-
-    @Override
-    public boolean isMarkedForDelete(long now)
-    {
-        return true;
-    }
-
-    @Override
-    public long getMarkedForDeleteAt()
-    {
-        return timestamp;
-    }
-
-    @Override
-    public void updateDigest(MessageDigest digest)
-    {
-        digest.update(name.toByteBuffer().duplicate());
-
-        DataOutputBuffer buffer = new DataOutputBuffer();
-        try
-        {
-            buffer.writeLong(timestamp);
-            buffer.writeByte(serializationFlags());
-        }
-        catch (IOException e)
-        {
-            throw new RuntimeException(e);
-        }
-        digest.update(buffer.getData(), 0, buffer.getLength());
-    }
-
-    @Override
-    public int getLocalDeletionTime()
-    {
-       return value.getInt(value.position());
-    }
-
-    @Override
-    public Column reconcile(Column column, Allocator allocator)
-    {
-        if (column instanceof DeletedColumn)
-            return super.reconcile(column, allocator);
-        return column.reconcile(this, allocator);
-    }
-
-    @Override
-    public Column localCopy(ColumnFamilyStore cfs)
-    {
-        return new DeletedColumn(name.copy(HeapAllocator.instance), ByteBufferUtil.clone(value), timestamp);
-    }
-
-    @Override
-    public Column localCopy(ColumnFamilyStore cfs, Allocator allocator)
-    {
-        return new DeletedColumn(name.copy(allocator), allocator.clone(value), timestamp);
-    }
-
-    @Override
-    public int serializationFlags()
-    {
-        return ColumnSerializer.DELETION_MASK;
-    }
-
-    @Override
-    public void validateFields(CFMetaData metadata) throws MarshalException
-    {
-        validateName(metadata);
-        if (value().remaining() != 4)
-            throw new MarshalException("A tombstone value should be 4 bytes long");
-        if (getLocalDeletionTime() < 0)
-            throw new MarshalException("The local deletion time should not be negative");
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/db/DeletionInfo.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/DeletionInfo.java b/src/java/org/apache/cassandra/db/DeletionInfo.java
index 0bd0635..5c62132 100644
--- a/src/java/org/apache/cassandra/db/DeletionInfo.java
+++ b/src/java/org/apache/cassandra/db/DeletionInfo.java
@@ -106,14 +106,14 @@ public class DeletionInfo
     }
 
     /**
-     * Return whether a given column is deleted by the container having this deletion info.
+     * Return whether a given cell is deleted by the container having this deletion info.
      *
-     * @param column the column to check.
-     * @return true if the column is deleted, false otherwise
+     * @param cell the cell to check.
+     * @return true if the cell is deleted, false otherwise
      */
-    public boolean isDeleted(Column column)
+    public boolean isDeleted(Cell cell)
     {
-        return isDeleted(column.name(), column.timestamp());
+        return isDeleted(cell.name(), cell.timestamp());
     }
 
     public boolean isDeleted(Composite name, long timestamp)
@@ -375,9 +375,9 @@ public class DeletionInfo
             this.reversed = reversed;
         }
 
-        public boolean isDeleted(Column column)
+        public boolean isDeleted(Cell cell)
         {
-            return isDeleted(column.name(), column.timestamp());
+            return isDeleted(cell.name(), cell.timestamp());
         }
 
         public boolean isDeleted(Composite name, long timestamp)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/db/EmptyColumns.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/EmptyColumns.java b/src/java/org/apache/cassandra/db/EmptyColumns.java
index 782ffc9..5498353 100644
--- a/src/java/org/apache/cassandra/db/EmptyColumns.java
+++ b/src/java/org/apache/cassandra/db/EmptyColumns.java
@@ -63,22 +63,22 @@ public class EmptyColumns extends AbstractThreadUnsafeSortedColumns
         return factory;
     }
 
-    public void addColumn(Column column, Allocator allocator)
+    public void addColumn(Cell cell, Allocator allocator)
     {
         throw new UnsupportedOperationException();
     }
 
-    public void addAll(ColumnFamily cm, Allocator allocator, Function<Column, Column> transformation)
+    public void addAll(ColumnFamily cm, Allocator allocator, Function<Cell, Cell> transformation)
     {
         throw new UnsupportedOperationException();
     }
 
-    public boolean replace(Column oldColumn, Column newColumn)
+    public boolean replace(Cell oldCell, Cell newCell)
     {
         throw new UnsupportedOperationException();
     }
 
-    public Column getColumn(CellName name)
+    public Cell getColumn(CellName name)
     {
         throw new UnsupportedOperationException();
     }
@@ -88,12 +88,12 @@ public class EmptyColumns extends AbstractThreadUnsafeSortedColumns
         return Collections.emptyList();
     }
 
-    public Collection<Column> getSortedColumns()
+    public Collection<Cell> getSortedColumns()
     {
         return Collections.emptyList();
     }
 
-    public Collection<Column> getReverseSortedColumns()
+    public Collection<Cell> getReverseSortedColumns()
     {
         return Collections.emptyList();
     }
@@ -103,12 +103,12 @@ public class EmptyColumns extends AbstractThreadUnsafeSortedColumns
         return 0;
     }
 
-    public Iterator<Column> iterator(ColumnSlice[] slices)
+    public Iterator<Cell> iterator(ColumnSlice[] slices)
     {
         return Iterators.emptyIterator();
     }
 
-    public Iterator<Column> reverseIterator(ColumnSlice[] slices)
+    public Iterator<Cell> reverseIterator(ColumnSlice[] slices)
     {
         return Iterators.emptyIterator();
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/db/ExpiringCell.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ExpiringCell.java b/src/java/org/apache/cassandra/db/ExpiringCell.java
new file mode 100644
index 0000000..2b9541c
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/ExpiringCell.java
@@ -0,0 +1,202 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.db;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.security.MessageDigest;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.db.composites.CellName;
+import org.apache.cassandra.db.composites.CellNameType;
+import org.apache.cassandra.io.util.DataOutputBuffer;
+import org.apache.cassandra.serializers.MarshalException;
+import org.apache.cassandra.utils.Allocator;
+import org.apache.cassandra.utils.HeapAllocator;
+
+/**
+ * Alternative to Cell that have an expiring time.
+ * ExpiringCell is immutable (as Cell is).
+ *
+ * Note that ExpiringCell does not override Cell.getMarkedForDeleteAt,
+ * which means that it's in the somewhat unintuitive position of being deleted (after its expiration)
+ * without having a time-at-which-it-became-deleted.  (Because ttl is a server-side measurement,
+ * we can't mix it with the timestamp field, which is client-supplied and whose resolution we
+ * can't assume anything about.)
+ */
+public class ExpiringCell extends Cell
+{
+    public static final int MAX_TTL = 20 * 365 * 24 * 60 * 60; // 20 years in seconds
+
+    private final int localExpirationTime;
+    private final int timeToLive;
+
+    public ExpiringCell(CellName name, ByteBuffer value, long timestamp, int timeToLive)
+    {
+      this(name, value, timestamp, timeToLive, (int) (System.currentTimeMillis() / 1000) + timeToLive);
+    }
+
+    public ExpiringCell(CellName name, ByteBuffer value, long timestamp, int timeToLive, int localExpirationTime)
+    {
+        super(name, value, timestamp);
+        assert timeToLive > 0 : timeToLive;
+        assert localExpirationTime > 0 : localExpirationTime;
+        this.timeToLive = timeToLive;
+        this.localExpirationTime = localExpirationTime;
+    }
+
+    /** @return Either a DeletedCell, or an ExpiringCell. */
+    public static Cell create(CellName name, ByteBuffer value, long timestamp, int timeToLive, int localExpirationTime, int expireBefore, ColumnSerializer.Flag flag)
+    {
+        if (localExpirationTime >= expireBefore || flag == ColumnSerializer.Flag.PRESERVE_SIZE)
+            return new ExpiringCell(name, value, timestamp, timeToLive, localExpirationTime);
+        // The column is now expired, we can safely return a simple tombstone. Note that
+        // as long as the expiring column and the tombstone put together live longer than GC grace seconds,
+        // we'll fulfil our responsibility to repair.  See discussion at
+        // http://cassandra-user-incubator-apache-org.3065146.n2.nabble.com/repair-compaction-and-tombstone-rows-td7583481.html
+        return new DeletedCell(name, localExpirationTime - timeToLive, timestamp);
+    }
+
+    public int getTimeToLive()
+    {
+        return timeToLive;
+    }
+
+    @Override
+    public Cell withUpdatedName(CellName newName)
+    {
+        return new ExpiringCell(newName, value, timestamp, timeToLive, localExpirationTime);
+    }
+
+    @Override
+    public Cell withUpdatedTimestamp(long newTimestamp)
+    {
+        return new ExpiringCell(name, value, newTimestamp, timeToLive, localExpirationTime);
+    }
+
+    @Override
+    public int dataSize()
+    {
+        return super.dataSize() + TypeSizes.NATIVE.sizeof(localExpirationTime) + TypeSizes.NATIVE.sizeof(timeToLive);
+    }
+
+    @Override
+    public int serializedSize(CellNameType type, TypeSizes typeSizes)
+    {
+        /*
+         * An expired column adds to a Cell :
+         *    4 bytes for the localExpirationTime
+         *  + 4 bytes for the timeToLive
+        */
+        return super.serializedSize(type, typeSizes) + typeSizes.sizeof(localExpirationTime) + typeSizes.sizeof(timeToLive);
+    }
+
+    @Override
+    public void updateDigest(MessageDigest digest)
+    {
+        digest.update(name.toByteBuffer().duplicate());
+        digest.update(value.duplicate());
+
+        DataOutputBuffer buffer = new DataOutputBuffer();
+        try
+        {
+            buffer.writeLong(timestamp);
+            buffer.writeByte(serializationFlags());
+            buffer.writeInt(timeToLive);
+        }
+        catch (IOException e)
+        {
+            throw new RuntimeException(e);
+        }
+        digest.update(buffer.getData(), 0, buffer.getLength());
+    }
+
+    @Override
+    public int getLocalDeletionTime()
+    {
+        return localExpirationTime;
+    }
+
+    @Override
+    public Cell localCopy(ColumnFamilyStore cfs)
+    {
+        return localCopy(cfs, HeapAllocator.instance);
+    }
+
+    @Override
+    public Cell localCopy(ColumnFamilyStore cfs, Allocator allocator)
+    {
+        return new ExpiringCell(name.copy(allocator), allocator.clone(value), timestamp, timeToLive, localExpirationTime);
+    }
+
+    @Override
+    public String getString(CellNameType comparator)
+    {
+        StringBuilder sb = new StringBuilder();
+        sb.append(super.getString(comparator));
+        sb.append("!");
+        sb.append(timeToLive);
+        return sb.toString();
+    }
+
+    @Override
+    public boolean isMarkedForDelete(long now)
+    {
+        return (int) (now / 1000) >= getLocalDeletionTime();
+    }
+
+    @Override
+    public long getMarkedForDeleteAt()
+    {
+        return timestamp;
+    }
+
+    @Override
+    public int serializationFlags()
+    {
+        return ColumnSerializer.EXPIRATION_MASK;
+    }
+
+    @Override
+    public void validateFields(CFMetaData metadata) throws MarshalException
+    {
+        super.validateFields(metadata);
+        if (timeToLive <= 0)
+            throw new MarshalException("A column TTL should be > 0");
+        if (localExpirationTime < 0)
+            throw new MarshalException("The local expiration time should not be negative");
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+        // super.equals() returns false if o is not a CounterCell
+        return super.equals(o)
+            && localExpirationTime == ((ExpiringCell)o).localExpirationTime
+            && timeToLive == ((ExpiringCell)o).timeToLive;
+    }
+
+    @Override
+    public int hashCode()
+    {
+        int result = super.hashCode();
+        result = 31 * result + localExpirationTime;
+        result = 31 * result + timeToLive;
+        return result;
+    }
+}


[6/6] git commit: Rename Column to Cell

Posted by sl...@apache.org.
Rename Column to Cell


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/e50d6af1
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/e50d6af1
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/e50d6af1

Branch: refs/heads/trunk
Commit: e50d6af12fabac98e3cb4bd589e2b92212b041f4
Parents: 362cc05
Author: Sylvain Lebresne <sy...@datastax.com>
Authored: Tue Dec 17 17:51:55 2013 +0100
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Tue Dec 17 17:52:20 2013 +0100

----------------------------------------------------------------------
 examples/hadoop_word_count/src/WordCount.java   |  16 +-
 .../src/WordCountCounters.java                  |  14 +-
 .../cassandra/triggers/InvertedIndex.java       |   4 +-
 .../org/apache/cassandra/config/CFMetaData.java |   8 +-
 .../apache/cassandra/cql/QueryProcessor.java    |  14 +-
 .../org/apache/cassandra/cql3/Attributes.java   |   6 +-
 src/java/org/apache/cassandra/cql3/CQL3Row.java |   8 +-
 src/java/org/apache/cassandra/cql3/Lists.java   |  10 +-
 .../apache/cassandra/cql3/QueryProcessor.java   |   4 +-
 .../apache/cassandra/cql3/UpdateParameters.java |  12 +-
 .../cql3/statements/AlterTableStatement.java    |   4 +-
 .../cql3/statements/SelectStatement.java        |  14 +-
 .../cassandra/cql3/statements/Selection.java    |  16 +-
 .../cassandra/db/ArrayBackedSortedColumns.java  | 160 ++++----
 .../cassandra/db/AtomicSortedColumns.java       |  70 ++--
 .../apache/cassandra/db/BatchlogManager.java    |   6 +-
 .../org/apache/cassandra/db/CFRowAdder.java     |   6 +-
 src/java/org/apache/cassandra/db/Cell.java      | 310 ++++++++++++++++
 .../cassandra/db/CollationController.java       |   4 +-
 src/java/org/apache/cassandra/db/Column.java    | 310 ----------------
 .../org/apache/cassandra/db/ColumnFamily.java   | 126 +++----
 .../cassandra/db/ColumnFamilySerializer.java    |   8 +-
 .../apache/cassandra/db/ColumnFamilyStore.java  |   8 +-
 .../org/apache/cassandra/db/ColumnIndex.java    |   2 +-
 .../apache/cassandra/db/ColumnSerializer.java   |  46 +--
 .../org/apache/cassandra/db/CounterCell.java    | 372 +++++++++++++++++++
 .../org/apache/cassandra/db/CounterColumn.java  | 372 -------------------
 .../apache/cassandra/db/CounterMutation.java    |   6 +-
 .../apache/cassandra/db/CounterUpdateCell.java  |  99 +++++
 .../cassandra/db/CounterUpdateColumn.java       |  99 -----
 .../org/apache/cassandra/db/DefsTables.java     |   2 +-
 .../org/apache/cassandra/db/DeletedCell.java    | 127 +++++++
 .../org/apache/cassandra/db/DeletedColumn.java  | 127 -------
 .../org/apache/cassandra/db/DeletionInfo.java   |  14 +-
 .../org/apache/cassandra/db/EmptyColumns.java   |  16 +-
 .../org/apache/cassandra/db/ExpiringCell.java   | 202 ++++++++++
 .../org/apache/cassandra/db/ExpiringColumn.java | 202 ----------
 .../cassandra/db/HintedHandOffManager.java      |   2 +-
 src/java/org/apache/cassandra/db/Keyspace.java  |   6 +-
 src/java/org/apache/cassandra/db/Memtable.java  |   5 +-
 .../org/apache/cassandra/db/OnDiskAtom.java     |   8 +-
 .../org/apache/cassandra/db/RangeTombstone.java |  12 +-
 .../org/apache/cassandra/db/SuperColumns.java   |  46 +--
 .../org/apache/cassandra/db/SystemKeyspace.java |   6 +-
 .../db/TreeMapBackedSortedColumns.java          |  54 +--
 .../apache/cassandra/db/UnsortedColumns.java    |  46 +--
 .../db/columniterator/SSTableNamesIterator.java |   2 +-
 .../db/columniterator/SSTableSliceIterator.java |   2 +-
 .../db/commitlog/CommitLogReplayer.java         |   2 +-
 .../db/compaction/CompactionManager.java        |  10 +-
 .../db/compaction/LazilyCompactedRow.java       |  16 +-
 .../db/composites/AbstractCellNameType.java     |  42 +--
 .../cassandra/db/composites/CellNameType.java   |   6 +-
 .../cassandra/db/composites/CellNames.java      |   8 +-
 .../cassandra/db/filter/ColumnCounter.java      |  18 +-
 .../apache/cassandra/db/filter/ColumnSlice.java |  10 +-
 .../cassandra/db/filter/ExtendedFilter.java     |  12 +-
 .../cassandra/db/filter/IDiskAtomFilter.java    |   4 +-
 .../cassandra/db/filter/NamesQueryFilter.java   |  14 +-
 .../apache/cassandra/db/filter/QueryFilter.java |  36 +-
 .../cassandra/db/filter/SliceQueryFilter.java   |  22 +-
 .../AbstractSimplePerColumnSecondaryIndex.java  |  28 +-
 .../db/index/PerColumnSecondaryIndex.java       |  12 +-
 .../db/index/PerRowSecondaryIndex.java          |   4 +-
 .../cassandra/db/index/SecondaryIndex.java      |   4 +-
 .../db/index/SecondaryIndexManager.java         |  64 ++--
 .../db/index/composites/CompositesIndex.java    |   6 +-
 .../CompositesIndexOnClusteringKey.java         |   6 +-
 .../CompositesIndexOnCollectionKey.java         |  10 +-
 .../CompositesIndexOnCollectionValue.java       |  12 +-
 .../CompositesIndexOnPartitionKey.java          |   4 +-
 .../composites/CompositesIndexOnRegular.java    |  12 +-
 .../db/index/composites/CompositesSearcher.java |  36 +-
 .../cassandra/db/index/keys/KeysIndex.java      |  14 +-
 .../cassandra/db/index/keys/KeysSearcher.java   |  30 +-
 .../db/marshal/AbstractCommutativeType.java     |   5 +-
 .../cassandra/db/marshal/CollectionType.java    |  15 +-
 .../cassandra/db/marshal/CounterColumnType.java |   4 +-
 .../apache/cassandra/db/marshal/ListType.java   |  13 +-
 .../apache/cassandra/db/marshal/MapType.java    |  12 +-
 .../apache/cassandra/db/marshal/SetType.java    |  13 +-
 .../hadoop/ColumnFamilyInputFormat.java         |   8 +-
 .../hadoop/ColumnFamilyRecordReader.java        |  92 ++---
 .../hadoop/pig/AbstractCassandraStorage.java    |   5 +-
 .../cassandra/hadoop/pig/CassandraStorage.java  |  24 +-
 .../apache/cassandra/hadoop/pig/CqlStorage.java |   8 +-
 .../io/sstable/AbstractSSTableSimpleWriter.java |  14 +-
 .../cassandra/io/sstable/SSTableWriter.java     |   4 +-
 .../cassandra/service/RowDataResolver.java      |   2 +-
 .../apache/cassandra/service/StorageProxy.java  |   4 +-
 .../service/pager/AbstractQueryPager.java       |   8 +-
 .../apache/cassandra/service/paxos/Commit.java  |   6 +-
 .../cassandra/thrift/CassandraServer.java       |  84 ++---
 .../cassandra/thrift/ThriftValidation.java      |  14 +-
 .../apache/cassandra/tools/SSTableExport.java   |  34 +-
 .../apache/cassandra/tools/SSTableImport.java   |   6 +-
 .../cassandra/tools/SSTableMetadataViewer.java  |   2 +-
 .../org/apache/cassandra/tracing/Tracing.java   |   4 +-
 .../cassandra/utils/BloomCalculations.java      |   2 +-
 .../org/apache/cassandra/utils/CounterId.java   |   4 +-
 .../apache/cassandra/db/MeteredFlusherTest.java |   2 +-
 .../db/compaction/LongCompactionsTest.java      |   2 +-
 test/unit/org/apache/cassandra/Util.java        |  14 +-
 .../org/apache/cassandra/config/DefsTest.java   |   7 +-
 .../db/ArrayBackedSortedColumnsTest.java        |  24 +-
 .../cassandra/db/ColumnFamilyStoreTest.java     | 108 +++---
 .../apache/cassandra/db/CounterCellTest.java    | 323 ++++++++++++++++
 .../apache/cassandra/db/CounterColumnTest.java  | 323 ----------------
 .../cassandra/db/CounterMutationTest.java       |  14 +-
 .../org/apache/cassandra/db/KeyspaceTest.java   |  41 +-
 .../org/apache/cassandra/db/NameSortTest.java   |  10 +-
 .../apache/cassandra/db/RangeTombstoneTest.java |  34 +-
 .../apache/cassandra/db/ReadMessageTest.java    |   2 +-
 .../cassandra/db/RecoveryManagerTest.java       |   7 +-
 .../db/RecoveryManagerTruncateTest.java         |   3 +-
 .../org/apache/cassandra/db/RemoveCellTest.java |  86 +++++
 .../apache/cassandra/db/RemoveColumnTest.java   |  86 -----
 .../apache/cassandra/db/RemoveSubCellTest.java  | 100 +++++
 .../cassandra/db/RemoveSubColumnTest.java       | 100 -----
 .../org/apache/cassandra/db/RowCacheTest.java   |  24 +-
 test/unit/org/apache/cassandra/db/RowTest.java  |   2 +-
 .../unit/org/apache/cassandra/db/ScrubTest.java |   2 +-
 .../db/SecondaryIndexCellSizeTest.java          | 230 ++++++++++++
 .../db/SecondaryIndexColumnSizeTest.java        | 230 ------------
 .../apache/cassandra/db/SerializationsTest.java |  31 +-
 .../org/apache/cassandra/db/TimeSortTest.java   |  24 +-
 .../db/compaction/CompactionsPurgeTest.java     |   6 +-
 .../db/index/PerRowSecondaryIndexTest.java      |   8 +-
 .../cassandra/db/marshal/CompositeTypeTest.java |   2 +-
 .../db/marshal/DynamicCompositeTypeTest.java    |   2 +-
 .../io/sstable/IndexSummaryManagerTest.java     |   7 +-
 .../io/sstable/SSTableSimpleWriterTest.java     |   4 +-
 .../cassandra/io/sstable/SSTableUtils.java      |  10 +-
 .../cassandra/service/QueryPagerTest.java       |   4 +-
 .../streaming/StreamingTransferTest.java        |   8 +-
 .../cassandra/tools/SSTableExportTest.java      |   6 +-
 .../cassandra/tools/SSTableImportTest.java      |  43 ++-
 tools/stress/README.txt                         |   4 +-
 .../org/apache/cassandra/stress/Session.java    |   6 +-
 .../stress/operations/CqlInserter.java          |   2 +-
 140 files changed, 2880 insertions(+), 2903 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/examples/hadoop_word_count/src/WordCount.java
----------------------------------------------------------------------
diff --git a/examples/hadoop_word_count/src/WordCount.java b/examples/hadoop_word_count/src/WordCount.java
index 398a7cb..ce9565d 100644
--- a/examples/hadoop_word_count/src/WordCount.java
+++ b/examples/hadoop_word_count/src/WordCount.java
@@ -18,14 +18,12 @@
 
 import java.io.IOException;
 import java.nio.ByteBuffer;
-import java.util.*;
 
-import org.apache.cassandra.thrift.*;
+import org.apache.cassandra.db.Cell;
 import org.apache.cassandra.hadoop.ColumnFamilyOutputFormat;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.db.Column;
 import org.apache.cassandra.hadoop.ColumnFamilyInputFormat;
 import org.apache.cassandra.hadoop.ConfigHelper;
 import org.apache.cassandra.utils.ByteBufferUtil;
@@ -70,7 +68,7 @@ public class WordCount extends Configured implements Tool
         System.exit(0);
     }
 
-    public static class TokenizerMapper extends Mapper<ByteBuffer, SortedMap<ByteBuffer, Column>, Text, IntWritable>
+    public static class TokenizerMapper extends Mapper<ByteBuffer, SortedMap<ByteBuffer, Cell>, Text, IntWritable>
     {
         private final static IntWritable one = new IntWritable(1);
         private Text word = new Text();
@@ -81,17 +79,17 @@ public class WordCount extends Configured implements Tool
         {
         }
 
-        public void map(ByteBuffer key, SortedMap<ByteBuffer, Column> columns, Context context) throws IOException, InterruptedException
+        public void map(ByteBuffer key, SortedMap<ByteBuffer, Cell> columns, Context context) throws IOException, InterruptedException
         {
-            for (Column column : columns.values())
+            for (Cell cell : columns.values())
             {
-                String name  = ByteBufferUtil.string(column.name());
+                String name  = ByteBufferUtil.string(cell.name());
                 String value = null;
                 
                 if (name.contains("int"))
-                    value = String.valueOf(ByteBufferUtil.toInt(column.value()));
+                    value = String.valueOf(ByteBufferUtil.toInt(cell.value()));
                 else
-                    value = ByteBufferUtil.string(column.value());
+                    value = ByteBufferUtil.string(cell.value());
                                
                 logger.debug("read {}:{}={} from {}",
                              new Object[] {ByteBufferUtil.string(key), name, value, context.getInputSplit()});

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/examples/hadoop_word_count/src/WordCountCounters.java
----------------------------------------------------------------------
diff --git a/examples/hadoop_word_count/src/WordCountCounters.java b/examples/hadoop_word_count/src/WordCountCounters.java
index 55d0889..4ea063b 100644
--- a/examples/hadoop_word_count/src/WordCountCounters.java
+++ b/examples/hadoop_word_count/src/WordCountCounters.java
@@ -18,8 +18,8 @@
 
 import java.io.IOException;
 import java.nio.ByteBuffer;
-import java.util.*;
 
+import org.apache.cassandra.db.Cell;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -34,10 +34,8 @@ import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
 
-import org.apache.cassandra.db.Column;
 import org.apache.cassandra.hadoop.ColumnFamilyInputFormat;
 import org.apache.cassandra.hadoop.ConfigHelper;
-import org.apache.cassandra.thrift.*;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 /**
@@ -60,15 +58,15 @@ public class WordCountCounters extends Configured implements Tool
         System.exit(0);
     }
 
-    public static class SumMapper extends Mapper<ByteBuffer, SortedMap<ByteBuffer, Column>, Text, LongWritable>
+    public static class SumMapper extends Mapper<ByteBuffer, SortedMap<ByteBuffer, Cell>, Text, LongWritable>
     {
-        public void map(ByteBuffer key, SortedMap<ByteBuffer, Column> columns, Context context) throws IOException, InterruptedException
+        public void map(ByteBuffer key, SortedMap<ByteBuffer, Cell> columns, Context context) throws IOException, InterruptedException
         {
             long sum = 0;
-            for (Column column : columns.values())
+            for (Cell cell : columns.values())
             {
-                logger.debug("read " + key + ":" + column.name() + " from " + context.getInputSplit());
-                sum += ByteBufferUtil.toLong(column.value());
+                logger.debug("read " + key + ":" + cell.name() + " from " + context.getInputSplit());
+                sum += ByteBufferUtil.toLong(cell.value());
             }
             context.write(new Text(ByteBufferUtil.string(key)), new LongWritable(sum));
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/examples/triggers/src/org/apache/cassandra/triggers/InvertedIndex.java
----------------------------------------------------------------------
diff --git a/examples/triggers/src/org/apache/cassandra/triggers/InvertedIndex.java b/examples/triggers/src/org/apache/cassandra/triggers/InvertedIndex.java
index a2d7644..2e1d4cc 100644
--- a/examples/triggers/src/org/apache/cassandra/triggers/InvertedIndex.java
+++ b/examples/triggers/src/org/apache/cassandra/triggers/InvertedIndex.java
@@ -24,10 +24,10 @@ import java.util.Collection;
 import java.util.List;
 import java.util.Properties;
 
+import org.apache.cassandra.db.Cell;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.db.Column;
 import org.apache.cassandra.db.ColumnFamily;
 import org.apache.cassandra.db.RowMutation;
 import org.apache.cassandra.io.util.FileUtils;
@@ -40,7 +40,7 @@ public class InvertedIndex implements ITrigger
     public Collection<RowMutation> augment(ByteBuffer key, ColumnFamily update)
     {
         List<RowMutation> mutations = new ArrayList<>();
-        for (Column cell : update)
+        for (Cell cell : update)
         {
             RowMutation mutation = new RowMutation(properties.getProperty("keyspace"), cell.value());
             mutation.add(properties.getProperty("columnfamily"), cell.name(), key, System.currentTimeMillis());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/config/CFMetaData.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/CFMetaData.java b/src/java/org/apache/cassandra/config/CFMetaData.java
index e56c1dd..097f150 100644
--- a/src/java/org/apache/cassandra/config/CFMetaData.java
+++ b/src/java/org/apache/cassandra/config/CFMetaData.java
@@ -1347,7 +1347,7 @@ public final class CFMetaData
     {
         if (version.hasSuperColumns && cfType == ColumnFamilyType.Super)
             return SuperColumns.onDiskIterator(in, count, flag, expireBefore, comparator);
-        return Column.onDiskIterator(in, count, flag, expireBefore, version, comparator);
+        return Cell.onDiskIterator(in, count, flag, expireBefore, version, comparator);
     }
 
     public AtomDeserializer getOnDiskDeserializer(DataInput in, Descriptor.Version version)
@@ -2101,10 +2101,10 @@ public final class CFMetaData
         return true;
     }
 
-    public void validateColumns(Iterable<Column> columns)
+    public void validateColumns(Iterable<Cell> columns)
     {
-        for (Column column : columns)
-            column.validateFields(this);
+        for (Cell cell : columns)
+            cell.validateFields(this);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/cql/QueryProcessor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql/QueryProcessor.java b/src/java/org/apache/cassandra/cql/QueryProcessor.java
index 12a0c22..3d5c3f1 100644
--- a/src/java/org/apache/cassandra/cql/QueryProcessor.java
+++ b/src/java/org/apache/cassandra/cql/QueryProcessor.java
@@ -33,7 +33,7 @@ import org.apache.cassandra.cql.hooks.ExecutionContext;
 import org.apache.cassandra.cql.hooks.PostPreparationHook;
 import org.apache.cassandra.cql.hooks.PreExecutionHook;
 import org.apache.cassandra.cql.hooks.PreparationContext;
-import org.apache.cassandra.db.CounterColumn;
+import org.apache.cassandra.db.CounterCell;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.composites.CellName;
 import org.apache.cassandra.db.composites.CellNameType;
@@ -310,10 +310,10 @@ public class QueryProcessor
     {
         for (CellName name : columns)
         {
-            if (name.dataSize() > org.apache.cassandra.db.Column.MAX_NAME_LENGTH)
+            if (name.dataSize() > org.apache.cassandra.db.Cell.MAX_NAME_LENGTH)
                 throw new InvalidRequestException(String.format("column name is too long (%s > %s)",
                                                                 name.dataSize(),
-                                                                org.apache.cassandra.db.Column.MAX_NAME_LENGTH));
+                                                                org.apache.cassandra.db.Cell.MAX_NAME_LENGTH));
             if (name.isEmpty())
                 throw new InvalidRequestException("zero-length column name");
         }
@@ -467,7 +467,7 @@ public class QueryProcessor
                         // preserve comparator order
                         if (row.cf != null)
                         {
-                            for (org.apache.cassandra.db.Column c : row.cf.getSortedColumns())
+                            for (org.apache.cassandra.db.Cell c : row.cf.getSortedColumns())
                             {
                                 if (c.isMarkedForDelete(now))
                                     continue;
@@ -514,7 +514,7 @@ public class QueryProcessor
                             ColumnDefinition cd = metadata.getColumnDefinition(name);
                             if (cd != null)
                                 result.schema.value_types.put(nameBytes, TypeParser.getShortName(cd.type));
-                            org.apache.cassandra.db.Column c = row.cf.getColumn(name);
+                            org.apache.cassandra.db.Cell c = row.cf.getColumn(name);
                             if (c == null || c.isMarkedForDelete(System.currentTimeMillis()))
                                 thriftColumns.add(new Column().setName(nameBytes));
                             else
@@ -854,9 +854,9 @@ public class QueryProcessor
         return cql.hashCode();
     }
 
-    private static Column thriftify(org.apache.cassandra.db.Column c)
+    private static Column thriftify(org.apache.cassandra.db.Cell c)
     {
-        ByteBuffer value = (c instanceof CounterColumn)
+        ByteBuffer value = (c instanceof CounterCell)
                            ? ByteBufferUtil.bytes(CounterContext.instance().total(c.value()))
                            : c.value();
         return new Column(c.name().toByteBuffer()).setValue(value).setTimestamp(c.timestamp());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/cql3/Attributes.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Attributes.java b/src/java/org/apache/cassandra/cql3/Attributes.java
index a92cc80..97ce31a 100644
--- a/src/java/org/apache/cassandra/cql3/Attributes.java
+++ b/src/java/org/apache/cassandra/cql3/Attributes.java
@@ -20,7 +20,7 @@ package org.apache.cassandra.cql3;
 import java.nio.ByteBuffer;
 import java.util.List;
 
-import org.apache.cassandra.db.ExpiringColumn;
+import org.apache.cassandra.db.ExpiringCell;
 import org.apache.cassandra.db.marshal.Int32Type;
 import org.apache.cassandra.db.marshal.LongType;
 import org.apache.cassandra.exceptions.InvalidRequestException;
@@ -99,8 +99,8 @@ public class Attributes
         if (ttl < 0)
             throw new InvalidRequestException("A TTL must be greater or equal to 0");
 
-        if (ttl > ExpiringColumn.MAX_TTL)
-            throw new InvalidRequestException(String.format("ttl is too large. requested (%d) maximum (%d)", ttl, ExpiringColumn.MAX_TTL));
+        if (ttl > ExpiringCell.MAX_TTL)
+            throw new InvalidRequestException(String.format("ttl is too large. requested (%d) maximum (%d)", ttl, ExpiringCell.MAX_TTL));
 
         return ttl;
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/cql3/CQL3Row.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/CQL3Row.java b/src/java/org/apache/cassandra/cql3/CQL3Row.java
index bc6bb2c..afeb095 100644
--- a/src/java/org/apache/cassandra/cql3/CQL3Row.java
+++ b/src/java/org/apache/cassandra/cql3/CQL3Row.java
@@ -21,16 +21,16 @@ import java.nio.ByteBuffer;
 import java.util.Iterator;
 import java.util.List;
 
-import org.apache.cassandra.db.Column;
+import org.apache.cassandra.db.Cell;
 
 public interface CQL3Row
 {
     public ByteBuffer getClusteringColumn(int i);
-    public Column getColumn(ColumnIdentifier name);
-    public List<Column> getCollection(ColumnIdentifier name);
+    public Cell getColumn(ColumnIdentifier name);
+    public List<Cell> getCollection(ColumnIdentifier name);
 
     public interface Builder
     {
-        Iterator<CQL3Row> group(Iterator<Column> cells);
+        Iterator<CQL3Row> group(Iterator<Cell> cells);
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/cql3/Lists.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Lists.java b/src/java/org/apache/cassandra/cql3/Lists.java
index 6bcdfcd..ab21a64 100644
--- a/src/java/org/apache/cassandra/cql3/Lists.java
+++ b/src/java/org/apache/cassandra/cql3/Lists.java
@@ -24,8 +24,8 @@ import java.util.List;
 import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.db.Cell;
 import org.apache.cassandra.db.ColumnFamily;
-import org.apache.cassandra.db.Column;
 import org.apache.cassandra.db.composites.CellName;
 import org.apache.cassandra.db.composites.Composite;
 import org.apache.cassandra.db.marshal.CollectionType;
@@ -305,7 +305,7 @@ public abstract class Lists
             if (index == null)
                 throw new InvalidRequestException("Invalid null value for list index");
 
-            List<Column> existingList = params.getPrefetchedList(rowKey, column.name);
+            List<Cell> existingList = params.getPrefetchedList(rowKey, column.name);
             int idx = ByteBufferUtil.toInt(index);
             if (idx < 0 || idx >= existingList.size())
                 throw new InvalidRequestException(String.format("List index %d out of bound, list has size %d", idx, existingList.size()));
@@ -399,7 +399,7 @@ public abstract class Lists
 
         public void execute(ByteBuffer rowKey, ColumnFamily cf, Composite prefix, UpdateParameters params) throws InvalidRequestException
         {
-            List<Column> existingList = params.getPrefetchedList(rowKey, column.name);
+            List<Cell> existingList = params.getPrefetchedList(rowKey, column.name);
             if (existingList.isEmpty())
                 return;
 
@@ -414,7 +414,7 @@ public abstract class Lists
             // the read-before-write this operation requires limits its usefulness on big lists, so in practice
             // toDiscard will be small and keeping a list will be more efficient.
             List<ByteBuffer> toDiscard = ((Lists.Value)value).elements;
-            for (Column cell : existingList)
+            for (Cell cell : existingList)
             {
                 if (toDiscard.contains(cell.value()))
                     cf.addColumn(params.makeTombstone(cell.name()));
@@ -443,7 +443,7 @@ public abstract class Lists
 
             assert index instanceof Constants.Value;
 
-            List<Column> existingList = params.getPrefetchedList(rowKey, column.name);
+            List<Cell> existingList = params.getPrefetchedList(rowKey, column.name);
             int idx = ByteBufferUtil.toInt(((Constants.Value)index).bytes);
             if (idx < 0 || idx >= existingList.size())
                 throw new InvalidRequestException(String.format("List index %d out of bound, list has size %d", idx, existingList.size()));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/cql3/QueryProcessor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/QueryProcessor.java b/src/java/org/apache/cassandra/cql3/QueryProcessor.java
index 772c103..77542cd 100644
--- a/src/java/org/apache/cassandra/cql3/QueryProcessor.java
+++ b/src/java/org/apache/cassandra/cql3/QueryProcessor.java
@@ -174,10 +174,10 @@ public class QueryProcessor
 
     public static void validateComposite(Composite name) throws InvalidRequestException
     {
-        if (name.dataSize() > Column.MAX_NAME_LENGTH)
+        if (name.dataSize() > Cell.MAX_NAME_LENGTH)
             throw new InvalidRequestException(String.format("The sum of all clustering columns is too long (%s > %s)",
                                                             name.dataSize(),
-                                                            Column.MAX_NAME_LENGTH));
+                                                            Cell.MAX_NAME_LENGTH));
     }
 
     private static ResultMessage processStatement(CQLStatement statement,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/cql3/UpdateParameters.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/UpdateParameters.java b/src/java/org/apache/cassandra/cql3/UpdateParameters.java
index cc3fe4d..a3553f4 100644
--- a/src/java/org/apache/cassandra/cql3/UpdateParameters.java
+++ b/src/java/org/apache/cassandra/cql3/UpdateParameters.java
@@ -52,16 +52,16 @@ public class UpdateParameters
         this.prefetchedLists = prefetchedLists;
     }
 
-    public Column makeColumn(CellName name, ByteBuffer value) throws InvalidRequestException
+    public Cell makeColumn(CellName name, ByteBuffer value) throws InvalidRequestException
     {
         QueryProcessor.validateCellName(name);
-        return Column.create(name, value, timestamp, ttl, metadata);
+        return Cell.create(name, value, timestamp, ttl, metadata);
     }
 
-    public Column makeTombstone(CellName name) throws InvalidRequestException
+    public Cell makeTombstone(CellName name) throws InvalidRequestException
     {
         QueryProcessor.validateCellName(name);
-        return new DeletedColumn(name, localDeletionTime, timestamp);
+        return new DeletedCell(name, localDeletionTime, timestamp);
     }
 
     public RangeTombstone makeRangeTombstone(ColumnSlice slice) throws InvalidRequestException
@@ -78,12 +78,12 @@ public class UpdateParameters
         return new RangeTombstone(slice.start, slice.finish, timestamp - 1, localDeletionTime);
     }
 
-    public List<Column> getPrefetchedList(ByteBuffer rowKey, ColumnIdentifier cql3ColumnName)
+    public List<Cell> getPrefetchedList(ByteBuffer rowKey, ColumnIdentifier cql3ColumnName)
     {
         if (prefetchedLists == null)
             return Collections.emptyList();
 
         CQL3Row row = prefetchedLists.get(rowKey);
-        return row == null ? Collections.<Column>emptyList() : row.getCollection(cql3ColumnName);
+        return row == null ? Collections.<Cell>emptyList() : row.getCollection(cql3ColumnName);
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java b/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
index d1452d2..eb52a13 100644
--- a/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
@@ -106,7 +106,7 @@ public class AlterTableStatement extends SchemaAlteringStatement
 
             case ALTER:
                 if (def == null)
-                    throw new InvalidRequestException(String.format("Column %s was not found in table %s", columnName, columnFamily()));
+                    throw new InvalidRequestException(String.format("Cell %s was not found in table %s", columnName, columnFamily()));
 
                 switch (def.kind)
                 {
@@ -181,7 +181,7 @@ public class AlterTableStatement extends SchemaAlteringStatement
                 if (!cfm.isCQL3Table())
                     throw new InvalidRequestException("Cannot drop columns from a non-CQL3 CF");
                 if (def == null)
-                    throw new InvalidRequestException(String.format("Column %s was not found in table %s", columnName, columnFamily()));
+                    throw new InvalidRequestException(String.format("Cell %s was not found in table %s", columnName, columnFamily()));
 
                 switch (def.kind)
                 {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
index d9b4e04..b29a044 100644
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@ -808,7 +808,7 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
         return value;
     }
 
-    private Iterator<Column> applySliceRestriction(final Iterator<Column> cells, final List<ByteBuffer> variables) throws InvalidRequestException
+    private Iterator<Cell> applySliceRestriction(final Iterator<Cell> cells, final List<ByteBuffer> variables) throws InvalidRequestException
     {
         assert sliceRestriction != null;
 
@@ -816,14 +816,14 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
         final CellName excludedStart = sliceRestriction.isInclusive(Bound.START) ? null : type.makeCellName(sliceRestriction.bound(Bound.START, variables));
         final CellName excludedEnd = sliceRestriction.isInclusive(Bound.END) ? null : type.makeCellName(sliceRestriction.bound(Bound.END, variables));
 
-        return new AbstractIterator<Column>()
+        return new AbstractIterator<Cell>()
         {
-            protected Column computeNext()
+            protected Cell computeNext()
             {
                 if (!cells.hasNext())
                     return endOfData();
 
-                Column c = cells.next();
+                Cell c = cells.next();
 
                 // For dynamic CF, the column could be out of the requested bounds (because we don't support strict bounds internally (unless
                 // the comparator is composite that is)), filter here
@@ -876,7 +876,7 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
             keyComponents = new ByteBuffer[]{ key };
         }
 
-        Iterator<Column> cells = cf.getSortedColumns().iterator();
+        Iterator<Cell> cells = cf.getSortedColumns().iterator();
         if (sliceRestriction != null)
             cells = applySliceRestriction(cells, variables);
 
@@ -903,7 +903,7 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
                     case REGULAR:
                         if (def.type.isCollection())
                         {
-                            List<Column> collection = cql3Row.getCollection(def.name);
+                            List<Cell> collection = cql3Row.getCollection(def.name);
                             ByteBuffer value = collection == null
                                              ? null
                                              : ((CollectionType)def.type).serialize(collection);
@@ -1325,7 +1325,7 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
                 Boolean isReversed = null;
                 for (Boolean b : reversedMap)
                 {
-                    // Column on which order is specified can be in any order
+                    // Cell on which order is specified can be in any order
                     if (b == null)
                         continue;
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/cql3/statements/Selection.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/Selection.java b/src/java/org/apache/cassandra/cql3/statements/Selection.java
index 6a8cfe6..f8b3f52 100644
--- a/src/java/org/apache/cassandra/cql3/statements/Selection.java
+++ b/src/java/org/apache/cassandra/cql3/statements/Selection.java
@@ -28,9 +28,9 @@ import org.apache.cassandra.cql3.functions.Function;
 import org.apache.cassandra.cql3.functions.Functions;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.db.CounterColumn;
-import org.apache.cassandra.db.ExpiringColumn;
-import org.apache.cassandra.db.Column;
+import org.apache.cassandra.db.Cell;
+import org.apache.cassandra.db.CounterCell;
+import org.apache.cassandra.db.ExpiringCell;
 import org.apache.cassandra.db.context.CounterContext;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.marshal.Int32Type;
@@ -247,9 +247,9 @@ public abstract class Selection
         return new ResultSetBuilder(now);
     }
 
-    private static ByteBuffer value(Column c)
+    private static ByteBuffer value(Cell c)
     {
-        return (c instanceof CounterColumn)
+        return (c instanceof CounterCell)
             ? ByteBufferUtil.bytes(CounterContext.instance().total(c.value()))
             : c.value();
     }
@@ -284,7 +284,7 @@ public abstract class Selection
             current.add(v);
         }
 
-        public void add(Column c)
+        public void add(Cell c)
         {
             current.add(isDead(c) ? null : value(c));
             if (timestamps != null)
@@ -294,13 +294,13 @@ public abstract class Selection
             if (ttls != null)
             {
                 int ttl = -1;
-                if (!isDead(c) && c instanceof ExpiringColumn)
+                if (!isDead(c) && c instanceof ExpiringCell)
                     ttl = c.getLocalDeletionTime() - (int) (now / 1000);
                 ttls[current.size() - 1] = ttl;
             }
         }
 
-        private boolean isDead(Column c)
+        private boolean isDead(Cell c)
         {
             return c == null || c.isMarkedForDelete(now);
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/db/ArrayBackedSortedColumns.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ArrayBackedSortedColumns.java b/src/java/org/apache/cassandra/db/ArrayBackedSortedColumns.java
index 432b47e..5d5178c 100644
--- a/src/java/org/apache/cassandra/db/ArrayBackedSortedColumns.java
+++ b/src/java/org/apache/cassandra/db/ArrayBackedSortedColumns.java
@@ -35,13 +35,13 @@ import org.apache.cassandra.utils.Allocator;
  * A ColumnFamily backed by an ArrayList.
  * This implementation is not synchronized and should only be used when
  * thread-safety is not required. This implementation makes sense when the
- * main operations performed are iterating over the map and adding columns
+ * main operations performed are iterating over the map and adding cells
  * (especially if insertion is in sorted order).
  */
 public class ArrayBackedSortedColumns extends AbstractThreadUnsafeSortedColumns
 {
     private final boolean reversed;
-    private final ArrayList<Column> columns;
+    private final ArrayList<Cell> cells;
 
     public static final ColumnFamily.Factory<ArrayBackedSortedColumns> factory = new Factory<ArrayBackedSortedColumns>()
     {
@@ -55,14 +55,14 @@ public class ArrayBackedSortedColumns extends AbstractThreadUnsafeSortedColumns
     {
         super(metadata);
         this.reversed = reversed;
-        this.columns = new ArrayList<Column>();
+        this.cells = new ArrayList<Cell>();
     }
 
-    private ArrayBackedSortedColumns(Collection<Column> columns, CFMetaData metadata, boolean reversed)
+    private ArrayBackedSortedColumns(Collection<Cell> cells, CFMetaData metadata, boolean reversed)
     {
         super(metadata);
         this.reversed = reversed;
-        this.columns = new ArrayList<Column>(columns);
+        this.cells = new ArrayList<Cell>(cells);
     }
 
     public ColumnFamily.Factory getFactory()
@@ -72,7 +72,7 @@ public class ArrayBackedSortedColumns extends AbstractThreadUnsafeSortedColumns
 
     public ColumnFamily cloneMe()
     {
-        return new ArrayBackedSortedColumns(columns, metadata, reversed);
+        return new ArrayBackedSortedColumns(cells, metadata, reversed);
     }
 
     public boolean isInsertReversed()
@@ -85,15 +85,15 @@ public class ArrayBackedSortedColumns extends AbstractThreadUnsafeSortedColumns
         return reversed ? getComparator().reverseComparator() : getComparator();
     }
 
-    public Column getColumn(CellName name)
+    public Cell getColumn(CellName name)
     {
         int pos = binarySearch(name);
-        return pos >= 0 ? columns.get(pos) : null;
+        return pos >= 0 ? cells.get(pos) : null;
     }
 
     /**
-     * AddColumn throws an exception if the column added does not sort after
-     * the last column in the map.
+     * AddColumn throws an exception if the cell added does not sort after
+     * the last cell in the map.
      * The reasoning is that this implementation can get slower if too much
      * insertions are done in unsorted order and right now we only use it when
      * *all* insertion (with this method) are done in sorted order. The
@@ -101,37 +101,37 @@ public class ArrayBackedSortedColumns extends AbstractThreadUnsafeSortedColumns
      * without knowing about (we can revisit that decision later if we have
      * use cases where most insert are in sorted order but a few are not).
      */
-    public void addColumn(Column column, Allocator allocator)
+    public void addColumn(Cell cell, Allocator allocator)
     {
-        if (columns.isEmpty())
+        if (cells.isEmpty())
         {
-            columns.add(column);
+            cells.add(cell);
             return;
         }
 
         // Fast path if inserting at the tail
-        int c = internalComparator().compare(columns.get(getColumnCount() - 1).name(), column.name());
+        int c = internalComparator().compare(cells.get(getColumnCount() - 1).name(), cell.name());
         // note that we want an assertion here (see addColumn javadoc), but we also want that if
         // assertion are disabled, addColumn works correctly with unsorted input
-        assert c <= 0 : "Added column does not sort as the " + (reversed ? "first" : "last") + " column";
+        assert c <= 0 : "Added cell does not sort as the " + (reversed ? "first" : "last") + " cell";
 
         if (c < 0)
         {
             // Insert as last
-            columns.add(column);
+            cells.add(cell);
         }
         else if (c == 0)
         {
             // Resolve against last
-            resolveAgainst(getColumnCount() - 1, column, allocator);
+            resolveAgainst(getColumnCount() - 1, cell, allocator);
         }
         else
         {
-            int pos = binarySearch(column.name());
+            int pos = binarySearch(cell.name());
             if (pos >= 0)
-                resolveAgainst(pos, column, allocator);
+                resolveAgainst(pos, cell, allocator);
             else
-                columns.add(-pos-1, column);
+                cells.add(-pos-1, cell);
         }
     }
 
@@ -139,36 +139,36 @@ public class ArrayBackedSortedColumns extends AbstractThreadUnsafeSortedColumns
      * Resolve against element at position i.
      * Assume that i is a valid position.
      */
-    private void resolveAgainst(int i, Column column, Allocator allocator)
+    private void resolveAgainst(int i, Cell cell, Allocator allocator)
     {
-        Column oldColumn = columns.get(i);
+        Cell oldCell = cells.get(i);
 
         // calculate reconciled col from old (existing) col and new col
-        Column reconciledColumn = column.reconcile(oldColumn, allocator);
-        columns.set(i, reconciledColumn);
+        Cell reconciledCell = cell.reconcile(oldCell, allocator);
+        cells.set(i, reconciledCell);
     }
 
     private int binarySearch(CellName name)
     {
-        return binarySearch(columns, internalComparator(), name, 0);
+        return binarySearch(cells, internalComparator(), name, 0);
     }
 
     /**
      * Simple binary search for a given column name.
      * The return value has the exact same meaning that the one of Collections.binarySearch().
      * (We don't use Collections.binarySearch() directly because it would require us to create
-     * a fake Column (as well as an Column comparator) to do the search, which is ugly.
+     * a fake Cell (as well as an Cell comparator) to do the search, which is ugly.
      */
-    private static int binarySearch(List<Column> columns, Comparator<Composite> comparator, Composite name, int start)
+    private static int binarySearch(List<Cell> cells, Comparator<Composite> comparator, Composite name, int start)
     {
         int low = start;
-        int mid = columns.size();
+        int mid = cells.size();
         int high = mid - 1;
         int result = -1;
         while (low <= high)
         {
             mid = (low + high) >> 1;
-            if ((result = comparator.compare(name, columns.get(mid).name())) > 0)
+            if ((result = comparator.compare(name, cells.get(mid).name())) > 0)
             {
                 low = mid + 1;
             }
@@ -184,71 +184,71 @@ public class ArrayBackedSortedColumns extends AbstractThreadUnsafeSortedColumns
         return -mid - (result < 0 ? 1 : 2);
     }
 
-    public void addAll(ColumnFamily cm, Allocator allocator, Function<Column, Column> transformation)
+    public void addAll(ColumnFamily cm, Allocator allocator, Function<Cell, Cell> transformation)
     {
         delete(cm.deletionInfo());
         if (cm.getColumnCount() == 0)
             return;
 
-        Column[] copy = columns.toArray(new Column[getColumnCount()]);
+        Cell[] copy = cells.toArray(new Cell[getColumnCount()]);
         int idx = 0;
-        Iterator<Column> other = reversed ? cm.reverseIterator(ColumnSlice.ALL_COLUMNS_ARRAY) : cm.iterator();
-        Column otherColumn = other.next();
+        Iterator<Cell> other = reversed ? cm.reverseIterator(ColumnSlice.ALL_COLUMNS_ARRAY) : cm.iterator();
+        Cell otherCell = other.next();
 
-        columns.clear();
+        cells.clear();
 
-        while (idx < copy.length && otherColumn != null)
+        while (idx < copy.length && otherCell != null)
         {
-            int c = internalComparator().compare(copy[idx].name(), otherColumn.name());
+            int c = internalComparator().compare(copy[idx].name(), otherCell.name());
             if (c < 0)
             {
-                columns.add(copy[idx]);
+                cells.add(copy[idx]);
                 idx++;
             }
             else if (c > 0)
             {
-                columns.add(transformation.apply(otherColumn));
-                otherColumn = other.hasNext() ? other.next() : null;
+                cells.add(transformation.apply(otherCell));
+                otherCell = other.hasNext() ? other.next() : null;
             }
             else // c == 0
             {
-                columns.add(copy[idx]);
-                resolveAgainst(getColumnCount() - 1, transformation.apply(otherColumn), allocator);
+                cells.add(copy[idx]);
+                resolveAgainst(getColumnCount() - 1, transformation.apply(otherCell), allocator);
                 idx++;
-                otherColumn = other.hasNext() ? other.next() : null;
+                otherCell = other.hasNext() ? other.next() : null;
             }
         }
         while (idx < copy.length)
         {
-            columns.add(copy[idx++]);
+            cells.add(copy[idx++]);
         }
-        while (otherColumn != null)
+        while (otherCell != null)
         {
-            columns.add(transformation.apply(otherColumn));
-            otherColumn = other.hasNext() ? other.next() : null;
+            cells.add(transformation.apply(otherCell));
+            otherCell = other.hasNext() ? other.next() : null;
         }
     }
 
-    public boolean replace(Column oldColumn, Column newColumn)
+    public boolean replace(Cell oldCell, Cell newCell)
     {
-        if (!oldColumn.name().equals(newColumn.name()))
+        if (!oldCell.name().equals(newCell.name()))
             throw new IllegalArgumentException();
 
-        int pos = binarySearch(oldColumn.name());
+        int pos = binarySearch(oldCell.name());
         if (pos >= 0)
         {
-            columns.set(pos, newColumn);
+            cells.set(pos, newCell);
         }
 
         return pos >= 0;
     }
 
-    public Collection<Column> getSortedColumns()
+    public Collection<Cell> getSortedColumns()
     {
-        return reversed ? new ReverseSortedCollection() : columns;
+        return reversed ? new ReverseSortedCollection() : cells;
     }
 
-    public Collection<Column> getReverseSortedColumns()
+    public Collection<Cell> getReverseSortedColumns()
     {
         // If reversed, the element are sorted reversely, so we could expect
         // to return *this*, but *this* redefine the iterator to be in sorted
@@ -258,59 +258,59 @@ public class ArrayBackedSortedColumns extends AbstractThreadUnsafeSortedColumns
 
     public int getColumnCount()
     {
-        return columns.size();
+        return cells.size();
     }
 
     public void clear()
     {
         setDeletionInfo(DeletionInfo.live());
-        columns.clear();
+        cells.clear();
     }
 
     public Iterable<CellName> getColumnNames()
     {
-        return Iterables.transform(columns, new Function<Column, CellName>()
+        return Iterables.transform(cells, new Function<Cell, CellName>()
         {
-            public CellName apply(Column column)
+            public CellName apply(Cell cell)
             {
-                return column.name;
+                return cell.name;
             }
         });
     }
 
-    public Iterator<Column> iterator()
+    public Iterator<Cell> iterator()
     {
-        return reversed ? Lists.reverse(columns).iterator() : columns.iterator();
+        return reversed ? Lists.reverse(cells).iterator() : cells.iterator();
     }
 
-    public Iterator<Column> iterator(ColumnSlice[] slices)
+    public Iterator<Cell> iterator(ColumnSlice[] slices)
     {
-        return new SlicesIterator(columns, getComparator(), slices, reversed);
+        return new SlicesIterator(cells, getComparator(), slices, reversed);
     }
 
-    public Iterator<Column> reverseIterator(ColumnSlice[] slices)
+    public Iterator<Cell> reverseIterator(ColumnSlice[] slices)
     {
-        return new SlicesIterator(columns, getComparator(), slices, !reversed);
+        return new SlicesIterator(cells, getComparator(), slices, !reversed);
     }
 
-    private static class SlicesIterator extends AbstractIterator<Column>
+    private static class SlicesIterator extends AbstractIterator<Cell>
     {
-        private final List<Column> list;
+        private final List<Cell> list;
         private final ColumnSlice[] slices;
         private final Comparator<Composite> comparator;
 
         private int idx = 0;
         private int previousSliceEnd = 0;
-        private Iterator<Column> currentSlice;
+        private Iterator<Cell> currentSlice;
 
-        public SlicesIterator(List<Column> list, CellNameType comparator, ColumnSlice[] slices, boolean reversed)
+        public SlicesIterator(List<Cell> list, CellNameType comparator, ColumnSlice[] slices, boolean reversed)
         {
             this.list = reversed ? Lists.reverse(list) : list;
             this.slices = slices;
             this.comparator = reversed ? comparator.reverseComparator() : comparator;
         }
 
-        protected Column computeNext()
+        protected Cell computeNext()
         {
             if (currentSlice == null)
             {
@@ -346,16 +346,16 @@ public class ArrayBackedSortedColumns extends AbstractThreadUnsafeSortedColumns
         }
     }
 
-    private class ReverseSortedCollection extends AbstractCollection<Column>
+    private class ReverseSortedCollection extends AbstractCollection<Cell>
     {
         public int size()
         {
-            return columns.size();
+            return cells.size();
         }
 
-        public Iterator<Column> iterator()
+        public Iterator<Cell> iterator()
         {
-            return new Iterator<Column>()
+            return new Iterator<Cell>()
             {
                 int idx = size() - 1;
 
@@ -364,29 +364,29 @@ public class ArrayBackedSortedColumns extends AbstractThreadUnsafeSortedColumns
                     return idx >= 0;
                 }
 
-                public Column next()
+                public Cell next()
                 {
-                    return columns.get(idx--);
+                    return cells.get(idx--);
                 }
 
                 public void remove()
                 {
-                    columns.remove(idx--);
+                    cells.remove(idx--);
                 }
             };
         }
     }
 
-    private class ForwardSortedCollection extends AbstractCollection<Column>
+    private class ForwardSortedCollection extends AbstractCollection<Cell>
     {
         public int size()
         {
-            return columns.size();
+            return cells.size();
         }
 
-        public Iterator<Column> iterator()
+        public Iterator<Cell> iterator()
         {
-            return columns.iterator();
+            return cells.iterator();
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/db/AtomicSortedColumns.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/AtomicSortedColumns.java b/src/java/org/apache/cassandra/db/AtomicSortedColumns.java
index 5056c26..6e4fd01 100644
--- a/src/java/org/apache/cassandra/db/AtomicSortedColumns.java
+++ b/src/java/org/apache/cassandra/db/AtomicSortedColumns.java
@@ -135,19 +135,19 @@ public class AtomicSortedColumns extends ColumnFamily
         }
     }
 
-    public void addColumn(Column column, Allocator allocator)
+    public void addColumn(Cell cell, Allocator allocator)
     {
         Holder current, modified;
         do
         {
             current = ref.get();
             modified = current.cloneMe();
-            modified.addColumn(column, allocator, SecondaryIndexManager.nullUpdater);
+            modified.addColumn(cell, allocator, SecondaryIndexManager.nullUpdater);
         }
         while (!ref.compareAndSet(current, modified));
     }
 
-    public void addAll(ColumnFamily cm, Allocator allocator, Function<Column, Column> transformation)
+    public void addAll(ColumnFamily cm, Allocator allocator, Function<Cell, Cell> transformation)
     {
         addAllWithSizeDelta(cm, allocator, transformation, SecondaryIndexManager.nullUpdater);
     }
@@ -157,7 +157,7 @@ public class AtomicSortedColumns extends ColumnFamily
      *
      *  @return the difference in size seen after merging the given columns
      */
-    public long addAllWithSizeDelta(ColumnFamily cm, Allocator allocator, Function<Column, Column> transformation, SecondaryIndexManager.Updater indexer)
+    public long addAllWithSizeDelta(ColumnFamily cm, Allocator allocator, Function<Cell, Cell> transformation, SecondaryIndexManager.Updater indexer)
     {
         /*
          * This operation needs to atomicity and isolation. To that end, we
@@ -183,16 +183,16 @@ public class AtomicSortedColumns extends ColumnFamily
 
             if (cm.deletionInfo().hasRanges())
             {
-                for (Column currentColumn : Iterables.concat(current.map.values(), cm))
+                for (Cell currentCell : Iterables.concat(current.map.values(), cm))
                 {
-                    if (cm.deletionInfo().isDeleted(currentColumn))
-                        indexer.remove(currentColumn);
+                    if (cm.deletionInfo().isDeleted(currentCell))
+                        indexer.remove(currentCell);
                 }
             }
 
-            for (Column column : cm)
+            for (Cell cell : cm)
             {
-                sizeDelta += modified.addColumn(transformation.apply(column), allocator, indexer);
+                sizeDelta += modified.addColumn(transformation.apply(cell), allocator, indexer);
                 // bail early if we know we've been beaten
                 if (ref.get() != current)
                     continue main_loop;
@@ -205,9 +205,9 @@ public class AtomicSortedColumns extends ColumnFamily
         return sizeDelta;
     }
 
-    public boolean replace(Column oldColumn, Column newColumn)
+    public boolean replace(Cell oldCell, Cell newCell)
     {
-        if (!oldColumn.name().equals(newColumn.name()))
+        if (!oldCell.name().equals(newCell.name()))
             throw new IllegalArgumentException();
 
         Holder current, modified;
@@ -216,7 +216,7 @@ public class AtomicSortedColumns extends ColumnFamily
         {
             current = ref.get();
             modified = current.cloneMe();
-            replaced = modified.map.replace(oldColumn.name(), oldColumn, newColumn);
+            replaced = modified.map.replace(oldCell.name(), oldCell, newCell);
         }
         while (!ref.compareAndSet(current, modified));
         return replaced;
@@ -233,7 +233,7 @@ public class AtomicSortedColumns extends ColumnFamily
         while (!ref.compareAndSet(current, modified));
     }
 
-    public Column getColumn(CellName name)
+    public Cell getColumn(CellName name)
     {
         return ref.get().map.get(name);
     }
@@ -243,12 +243,12 @@ public class AtomicSortedColumns extends ColumnFamily
         return ref.get().map.keySet();
     }
 
-    public Collection<Column> getSortedColumns()
+    public Collection<Cell> getSortedColumns()
     {
         return ref.get().map.values();
     }
 
-    public Collection<Column> getReverseSortedColumns()
+    public Collection<Cell> getReverseSortedColumns()
     {
         return ref.get().map.descendingMap().values();
     }
@@ -258,12 +258,12 @@ public class AtomicSortedColumns extends ColumnFamily
         return ref.get().map.size();
     }
 
-    public Iterator<Column> iterator(ColumnSlice[] slices)
+    public Iterator<Cell> iterator(ColumnSlice[] slices)
     {
         return new ColumnSlice.NavigableMapIterator(ref.get().map, slices);
     }
 
-    public Iterator<Column> reverseIterator(ColumnSlice[] slices)
+    public Iterator<Cell> reverseIterator(ColumnSlice[] slices)
     {
         return new ColumnSlice.NavigableMapIterator(ref.get().map.descendingMap(), slices);
     }
@@ -279,15 +279,15 @@ public class AtomicSortedColumns extends ColumnFamily
         // so we can safely alias one DeletionInfo.live() reference and avoid some allocations.
         private static final DeletionInfo LIVE = DeletionInfo.live();
 
-        final SnapTreeMap<CellName, Column> map;
+        final SnapTreeMap<CellName, Cell> map;
         final DeletionInfo deletionInfo;
 
         Holder(CellNameType comparator)
         {
-            this(new SnapTreeMap<CellName, Column>(comparator), LIVE);
+            this(new SnapTreeMap<CellName, Cell>(comparator), LIVE);
         }
 
-        Holder(SnapTreeMap<CellName, Column> map, DeletionInfo deletionInfo)
+        Holder(SnapTreeMap<CellName, Cell> map, DeletionInfo deletionInfo)
         {
             this.map = map;
             this.deletionInfo = deletionInfo;
@@ -303,7 +303,7 @@ public class AtomicSortedColumns extends ColumnFamily
             return new Holder(map, info);
         }
 
-        Holder with(SnapTreeMap<CellName, Column> newMap)
+        Holder with(SnapTreeMap<CellName, Cell> newMap)
         {
             return new Holder(newMap, deletionInfo);
         }
@@ -312,33 +312,33 @@ public class AtomicSortedColumns extends ColumnFamily
         // afterwards.
         Holder clear()
         {
-            return new Holder(new SnapTreeMap<CellName, Column>(map.comparator()), LIVE);
+            return new Holder(new SnapTreeMap<CellName, Cell>(map.comparator()), LIVE);
         }
 
-        long addColumn(Column column, Allocator allocator, SecondaryIndexManager.Updater indexer)
+        long addColumn(Cell cell, Allocator allocator, SecondaryIndexManager.Updater indexer)
         {
-            CellName name = column.name();
+            CellName name = cell.name();
             while (true)
             {
-                Column oldColumn = map.putIfAbsent(name, column);
-                if (oldColumn == null)
+                Cell oldCell = map.putIfAbsent(name, cell);
+                if (oldCell == null)
                 {
-                    indexer.insert(column);
-                    return column.dataSize();
+                    indexer.insert(cell);
+                    return cell.dataSize();
                 }
 
-                Column reconciledColumn = column.reconcile(oldColumn, allocator);
-                if (map.replace(name, oldColumn, reconciledColumn))
+                Cell reconciledCell = cell.reconcile(oldCell, allocator);
+                if (map.replace(name, oldCell, reconciledCell))
                 {
                     // for memtable updates we only care about oldcolumn, reconciledcolumn, but when compacting
                     // we need to make sure we update indexes no matter the order we merge
-                    if (reconciledColumn == column)
-                        indexer.update(oldColumn, reconciledColumn);
+                    if (reconciledCell == cell)
+                        indexer.update(oldCell, reconciledCell);
                     else
-                        indexer.update(column, reconciledColumn);
-                    return reconciledColumn.dataSize() - oldColumn.dataSize();
+                        indexer.update(cell, reconciledCell);
+                    return reconciledCell.dataSize() - oldCell.dataSize();
                 }
-                // We failed to replace column due to a concurrent update or a concurrent removal. Keep trying.
+                // We failed to replace cell due to a concurrent update or a concurrent removal. Keep trying.
                 // (Currently, concurrent removal should not happen (only updates), but let us support that anyway.)
             }
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/db/BatchlogManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/BatchlogManager.java b/src/java/org/apache/cassandra/db/BatchlogManager.java
index 9bc857b..24fb0c5 100644
--- a/src/java/org/apache/cassandra/db/BatchlogManager.java
+++ b/src/java/org/apache/cassandra/db/BatchlogManager.java
@@ -128,9 +128,9 @@ public class BatchlogManager implements BatchlogManagerMBean
         ByteBuffer data = serializeRowMutations(mutations);
 
         ColumnFamily cf = ArrayBackedSortedColumns.factory.create(CFMetaData.BatchlogCf);
-        cf.addColumn(new Column(cellName(""), ByteBufferUtil.EMPTY_BYTE_BUFFER, timestamp));
-        cf.addColumn(new Column(cellName("data"), data, timestamp));
-        cf.addColumn(new Column(cellName("written_at"), writtenAt, timestamp));
+        cf.addColumn(new Cell(cellName(""), ByteBufferUtil.EMPTY_BYTE_BUFFER, timestamp));
+        cf.addColumn(new Cell(cellName("data"), data, timestamp));
+        cf.addColumn(new Cell(cellName("written_at"), writtenAt, timestamp));
 
         return new RowMutation(Keyspace.SYSTEM_KS, UUIDType.instance.decompose(uuid), cf);
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/db/CFRowAdder.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/CFRowAdder.java b/src/java/org/apache/cassandra/db/CFRowAdder.java
index 7c70cf6..4247822 100644
--- a/src/java/org/apache/cassandra/db/CFRowAdder.java
+++ b/src/java/org/apache/cassandra/db/CFRowAdder.java
@@ -47,7 +47,7 @@ public class CFRowAdder
 
         // If a CQL3 table, add the row marker
         if (cf.metadata().isCQL3Table())
-            cf.addColumn(new Column(cf.getComparator().rowMarker(prefix), ByteBufferUtil.EMPTY_BYTE_BUFFER, timestamp));
+            cf.addColumn(new Cell(cf.getComparator().rowMarker(prefix), ByteBufferUtil.EMPTY_BYTE_BUFFER, timestamp));
     }
 
     public CFRowAdder add(String cql3ColumnName, Object value)
@@ -73,9 +73,9 @@ public class CFRowAdder
     private CFRowAdder add(CellName name, ColumnDefinition def, Object value)
     {
         if (value == null)
-            cf.addColumn(new DeletedColumn(name, ldt, timestamp));
+            cf.addColumn(new DeletedCell(name, ldt, timestamp));
         else
-            cf.addColumn(new Column(name, ((AbstractType)def.type).decompose(value), timestamp));
+            cf.addColumn(new Cell(name, ((AbstractType)def.type).decompose(value), timestamp));
         return this;
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/db/Cell.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Cell.java b/src/java/org/apache/cassandra/db/Cell.java
new file mode 100644
index 0000000..537a744
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/Cell.java
@@ -0,0 +1,310 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.db;
+
+import java.io.DataInput;
+import java.io.IOError;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.security.MessageDigest;
+import java.util.Iterator;
+
+import com.google.common.collect.AbstractIterator;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.db.composites.CellName;
+import org.apache.cassandra.db.composites.CellNameType;
+import org.apache.cassandra.db.marshal.*;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.io.util.DataOutputBuffer;
+import org.apache.cassandra.serializers.MarshalException;
+import org.apache.cassandra.utils.Allocator;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.HeapAllocator;
+
+/**
+ * Cell is immutable, which prevents all kinds of confusion in a multithreaded environment.
+ */
+public class Cell implements OnDiskAtom
+{
+    public static final int MAX_NAME_LENGTH = FBUtilities.MAX_UNSIGNED_SHORT;
+
+    /**
+     * For 2.0-formatted sstables (where column count is not stored), @param count should be Integer.MAX_VALUE,
+     * and we will look for the end-of-row column name marker instead of relying on that.
+     */
+    public static Iterator<OnDiskAtom> onDiskIterator(final DataInput in,
+                                                      final int count,
+                                                      final ColumnSerializer.Flag flag,
+                                                      final int expireBefore,
+                                                      final Descriptor.Version version,
+                                                      final CellNameType type)
+    {
+        return new AbstractIterator<OnDiskAtom>()
+        {
+            int i = 0;
+
+            protected OnDiskAtom computeNext()
+            {
+                if (i++ >= count)
+                    return endOfData();
+
+                OnDiskAtom atom;
+                try
+                {
+                    atom = type.onDiskAtomSerializer().deserializeFromSSTable(in, flag, expireBefore, version);
+                }
+                catch (IOException e)
+                {
+                    throw new IOError(e);
+                }
+                if (atom == null)
+                    return endOfData();
+
+                return atom;
+            }
+        };
+    }
+
+    protected final CellName name;
+    protected final ByteBuffer value;
+    protected final long timestamp;
+
+    Cell(CellName name)
+    {
+        this(name, ByteBufferUtil.EMPTY_BYTE_BUFFER);
+    }
+
+    public Cell(CellName name, ByteBuffer value)
+    {
+        this(name, value, 0);
+    }
+
+    public Cell(CellName name, ByteBuffer value, long timestamp)
+    {
+        assert name != null;
+        assert value != null;
+        this.name = name;
+        this.value = value;
+        this.timestamp = timestamp;
+    }
+
+    public Cell withUpdatedName(CellName newName)
+    {
+        return new Cell(newName, value, timestamp);
+    }
+
+    public Cell withUpdatedTimestamp(long newTimestamp)
+    {
+        return new Cell(name, value, newTimestamp);
+    }
+
+    public CellName name()
+    {
+        return name;
+    }
+
+    public ByteBuffer value()
+    {
+        return value;
+    }
+
+    public long timestamp()
+    {
+        return timestamp;
+    }
+
+    public long minTimestamp()
+    {
+        return timestamp;
+    }
+
+    public long maxTimestamp()
+    {
+        return timestamp;
+    }
+
+    public boolean isMarkedForDelete(long now)
+    {
+        return false;
+    }
+
+    public boolean isLive(long now)
+    {
+        return !isMarkedForDelete(now);
+    }
+
+    // Don't call unless the column is actually marked for delete.
+    public long getMarkedForDeleteAt()
+    {
+        return Long.MAX_VALUE;
+    }
+
+    public int dataSize()
+    {
+        return name().dataSize() + value.remaining() + TypeSizes.NATIVE.sizeof(timestamp);
+    }
+
+    public int serializedSize(CellNameType type, TypeSizes typeSizes)
+    {
+        /*
+         * Size of a column is =
+         *   size of a name (short + length of the string)
+         * + 1 byte to indicate if the column has been deleted
+         * + 8 bytes for timestamp
+         * + 4 bytes which basically indicates the size of the byte array
+         * + entire byte array.
+        */
+        int valueSize = value.remaining();
+        return ((int)type.cellSerializer().serializedSize(name, typeSizes)) + 1 + typeSizes.sizeof(timestamp) + typeSizes.sizeof(valueSize) + valueSize;
+    }
+
+    public int serializationFlags()
+    {
+        return 0;
+    }
+
+    public Cell diff(Cell cell)
+    {
+        if (timestamp() < cell.timestamp())
+            return cell;
+        return null;
+    }
+
+    public void updateDigest(MessageDigest digest)
+    {
+        digest.update(name.toByteBuffer().duplicate());
+        digest.update(value.duplicate());
+
+        DataOutputBuffer buffer = new DataOutputBuffer();
+        try
+        {
+            buffer.writeLong(timestamp);
+            buffer.writeByte(serializationFlags());
+        }
+        catch (IOException e)
+        {
+            throw new RuntimeException(e);
+        }
+        digest.update(buffer.getData(), 0, buffer.getLength());
+    }
+
+    public int getLocalDeletionTime()
+    {
+        return Integer.MAX_VALUE;
+    }
+
+    public Cell reconcile(Cell cell)
+    {
+        return reconcile(cell, HeapAllocator.instance);
+    }
+
+    public Cell reconcile(Cell cell, Allocator allocator)
+    {
+        // tombstones take precedence.  (if both are tombstones, then it doesn't matter which one we use.)
+        if (isMarkedForDelete(System.currentTimeMillis()))
+            return timestamp() < cell.timestamp() ? cell : this;
+        if (cell.isMarkedForDelete(System.currentTimeMillis()))
+            return timestamp() > cell.timestamp() ? this : cell;
+        // break ties by comparing values.
+        if (timestamp() == cell.timestamp())
+            return value().compareTo(cell.value()) < 0 ? cell : this;
+        // neither is tombstoned and timestamps are different
+        return timestamp() < cell.timestamp() ? cell : this;
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+        if (this == o)
+            return true;
+        if (o == null || getClass() != o.getClass())
+            return false;
+
+        Cell cell = (Cell)o;
+
+        if (timestamp != cell.timestamp)
+            return false;
+        if (!name.equals(cell.name))
+            return false;
+
+        return value.equals(cell.value);
+    }
+
+    @Override
+    public int hashCode()
+    {
+        int result = name != null ? name.hashCode() : 0;
+        result = 31 * result + (value != null ? value.hashCode() : 0);
+        result = 31 * result + (int)(timestamp ^ (timestamp >>> 32));
+        return result;
+    }
+
+    public Cell localCopy(ColumnFamilyStore cfs)
+    {
+        return localCopy(cfs, HeapAllocator.instance);
+    }
+
+    public Cell localCopy(ColumnFamilyStore cfs, Allocator allocator)
+    {
+        return new Cell(name.copy(allocator), allocator.clone(value), timestamp);
+    }
+
+    public String getString(CellNameType comparator)
+    {
+        StringBuilder sb = new StringBuilder();
+        sb.append(comparator.getString(name));
+        sb.append(":");
+        sb.append(isMarkedForDelete(System.currentTimeMillis()));
+        sb.append(":");
+        sb.append(value.remaining());
+        sb.append("@");
+        sb.append(timestamp());
+        return sb.toString();
+    }
+
+    protected void validateName(CFMetaData metadata) throws MarshalException
+    {
+        metadata.comparator.validate(name());
+    }
+
+    public void validateFields(CFMetaData metadata) throws MarshalException
+    {
+        validateName(metadata);
+
+        AbstractType<?> valueValidator = metadata.getValueValidator(name());
+        if (valueValidator != null)
+            valueValidator.validate(value());
+    }
+
+    public boolean hasIrrelevantData(int gcBefore)
+    {
+        return getLocalDeletionTime() < gcBefore;
+    }
+
+    public static Cell create(CellName name, ByteBuffer value, long timestamp, int ttl, CFMetaData metadata)
+    {
+        if (ttl <= 0)
+            ttl = metadata.getDefaultTimeToLive();
+
+        return ttl > 0
+               ? new ExpiringCell(name, value, timestamp, ttl)
+               : new Cell(name, value, timestamp);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/db/CollationController.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/CollationController.java b/src/java/org/apache/cassandra/db/CollationController.java
index 12f53db..c7c09bf 100644
--- a/src/java/org/apache/cassandra/db/CollationController.java
+++ b/src/java/org/apache/cassandra/db/CollationController.java
@@ -176,8 +176,8 @@ public class CollationController
         for (Iterator<CellName> iterator = ((NamesQueryFilter) filter.filter).columns.iterator(); iterator.hasNext(); )
         {
             CellName filterColumn = iterator.next();
-            Column column = container.getColumn(filterColumn);
-            if (column != null && column.timestamp() > sstableTimestamp)
+            Cell cell = container.getColumn(filterColumn);
+            if (cell != null && cell.timestamp() > sstableTimestamp)
                 iterator.remove();
         }
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/db/Column.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Column.java b/src/java/org/apache/cassandra/db/Column.java
deleted file mode 100644
index 4be1eeb..0000000
--- a/src/java/org/apache/cassandra/db/Column.java
+++ /dev/null
@@ -1,310 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.db;
-
-import java.io.DataInput;
-import java.io.IOError;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.security.MessageDigest;
-import java.util.Iterator;
-
-import com.google.common.collect.AbstractIterator;
-
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.db.composites.CellName;
-import org.apache.cassandra.db.composites.CellNameType;
-import org.apache.cassandra.db.marshal.*;
-import org.apache.cassandra.io.sstable.Descriptor;
-import org.apache.cassandra.io.util.DataOutputBuffer;
-import org.apache.cassandra.serializers.MarshalException;
-import org.apache.cassandra.utils.Allocator;
-import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.FBUtilities;
-import org.apache.cassandra.utils.HeapAllocator;
-
-/**
- * Column is immutable, which prevents all kinds of confusion in a multithreaded environment.
- */
-public class Column implements OnDiskAtom
-{
-    public static final int MAX_NAME_LENGTH = FBUtilities.MAX_UNSIGNED_SHORT;
-
-    /**
-     * For 2.0-formatted sstables (where column count is not stored), @param count should be Integer.MAX_VALUE,
-     * and we will look for the end-of-row column name marker instead of relying on that.
-     */
-    public static Iterator<OnDiskAtom> onDiskIterator(final DataInput in,
-                                                      final int count,
-                                                      final ColumnSerializer.Flag flag,
-                                                      final int expireBefore,
-                                                      final Descriptor.Version version,
-                                                      final CellNameType type)
-    {
-        return new AbstractIterator<OnDiskAtom>()
-        {
-            int i = 0;
-
-            protected OnDiskAtom computeNext()
-            {
-                if (i++ >= count)
-                    return endOfData();
-
-                OnDiskAtom atom;
-                try
-                {
-                    atom = type.onDiskAtomSerializer().deserializeFromSSTable(in, flag, expireBefore, version);
-                }
-                catch (IOException e)
-                {
-                    throw new IOError(e);
-                }
-                if (atom == null)
-                    return endOfData();
-
-                return atom;
-            }
-        };
-    }
-
-    protected final CellName name;
-    protected final ByteBuffer value;
-    protected final long timestamp;
-
-    Column(CellName name)
-    {
-        this(name, ByteBufferUtil.EMPTY_BYTE_BUFFER);
-    }
-
-    public Column(CellName name, ByteBuffer value)
-    {
-        this(name, value, 0);
-    }
-
-    public Column(CellName name, ByteBuffer value, long timestamp)
-    {
-        assert name != null;
-        assert value != null;
-        this.name = name;
-        this.value = value;
-        this.timestamp = timestamp;
-    }
-
-    public Column withUpdatedName(CellName newName)
-    {
-        return new Column(newName, value, timestamp);
-    }
-
-    public Column withUpdatedTimestamp(long newTimestamp)
-    {
-        return new Column(name, value, newTimestamp);
-    }
-
-    public CellName name()
-    {
-        return name;
-    }
-
-    public ByteBuffer value()
-    {
-        return value;
-    }
-
-    public long timestamp()
-    {
-        return timestamp;
-    }
-
-    public long minTimestamp()
-    {
-        return timestamp;
-    }
-
-    public long maxTimestamp()
-    {
-        return timestamp;
-    }
-
-    public boolean isMarkedForDelete(long now)
-    {
-        return false;
-    }
-
-    public boolean isLive(long now)
-    {
-        return !isMarkedForDelete(now);
-    }
-
-    // Don't call unless the column is actually marked for delete.
-    public long getMarkedForDeleteAt()
-    {
-        return Long.MAX_VALUE;
-    }
-
-    public int dataSize()
-    {
-        return name().dataSize() + value.remaining() + TypeSizes.NATIVE.sizeof(timestamp);
-    }
-
-    public int serializedSize(CellNameType type, TypeSizes typeSizes)
-    {
-        /*
-         * Size of a column is =
-         *   size of a name (short + length of the string)
-         * + 1 byte to indicate if the column has been deleted
-         * + 8 bytes for timestamp
-         * + 4 bytes which basically indicates the size of the byte array
-         * + entire byte array.
-        */
-        int valueSize = value.remaining();
-        return ((int)type.cellSerializer().serializedSize(name, typeSizes)) + 1 + typeSizes.sizeof(timestamp) + typeSizes.sizeof(valueSize) + valueSize;
-    }
-
-    public int serializationFlags()
-    {
-        return 0;
-    }
-
-    public Column diff(Column column)
-    {
-        if (timestamp() < column.timestamp())
-            return column;
-        return null;
-    }
-
-    public void updateDigest(MessageDigest digest)
-    {
-        digest.update(name.toByteBuffer().duplicate());
-        digest.update(value.duplicate());
-
-        DataOutputBuffer buffer = new DataOutputBuffer();
-        try
-        {
-            buffer.writeLong(timestamp);
-            buffer.writeByte(serializationFlags());
-        }
-        catch (IOException e)
-        {
-            throw new RuntimeException(e);
-        }
-        digest.update(buffer.getData(), 0, buffer.getLength());
-    }
-
-    public int getLocalDeletionTime()
-    {
-        return Integer.MAX_VALUE;
-    }
-
-    public Column reconcile(Column column)
-    {
-        return reconcile(column, HeapAllocator.instance);
-    }
-
-    public Column reconcile(Column column, Allocator allocator)
-    {
-        // tombstones take precedence.  (if both are tombstones, then it doesn't matter which one we use.)
-        if (isMarkedForDelete(System.currentTimeMillis()))
-            return timestamp() < column.timestamp() ? column : this;
-        if (column.isMarkedForDelete(System.currentTimeMillis()))
-            return timestamp() > column.timestamp() ? this : column;
-        // break ties by comparing values.
-        if (timestamp() == column.timestamp())
-            return value().compareTo(column.value()) < 0 ? column : this;
-        // neither is tombstoned and timestamps are different
-        return timestamp() < column.timestamp() ? column : this;
-    }
-
-    @Override
-    public boolean equals(Object o)
-    {
-        if (this == o)
-            return true;
-        if (o == null || getClass() != o.getClass())
-            return false;
-
-        Column column = (Column)o;
-
-        if (timestamp != column.timestamp)
-            return false;
-        if (!name.equals(column.name))
-            return false;
-
-        return value.equals(column.value);
-    }
-
-    @Override
-    public int hashCode()
-    {
-        int result = name != null ? name.hashCode() : 0;
-        result = 31 * result + (value != null ? value.hashCode() : 0);
-        result = 31 * result + (int)(timestamp ^ (timestamp >>> 32));
-        return result;
-    }
-
-    public Column localCopy(ColumnFamilyStore cfs)
-    {
-        return localCopy(cfs, HeapAllocator.instance);
-    }
-
-    public Column localCopy(ColumnFamilyStore cfs, Allocator allocator)
-    {
-        return new Column(name.copy(allocator), allocator.clone(value), timestamp);
-    }
-
-    public String getString(CellNameType comparator)
-    {
-        StringBuilder sb = new StringBuilder();
-        sb.append(comparator.getString(name));
-        sb.append(":");
-        sb.append(isMarkedForDelete(System.currentTimeMillis()));
-        sb.append(":");
-        sb.append(value.remaining());
-        sb.append("@");
-        sb.append(timestamp());
-        return sb.toString();
-    }
-
-    protected void validateName(CFMetaData metadata) throws MarshalException
-    {
-        metadata.comparator.validate(name());
-    }
-
-    public void validateFields(CFMetaData metadata) throws MarshalException
-    {
-        validateName(metadata);
-
-        AbstractType<?> valueValidator = metadata.getValueValidator(name());
-        if (valueValidator != null)
-            valueValidator.validate(value());
-    }
-
-    public boolean hasIrrelevantData(int gcBefore)
-    {
-        return getLocalDeletionTime() < gcBefore;
-    }
-
-    public static Column create(CellName name, ByteBuffer value, long timestamp, int ttl, CFMetaData metadata)
-    {
-        if (ttl <= 0)
-            ttl = metadata.getDefaultTimeToLive();
-
-        return ttl > 0
-               ? new ExpiringColumn(name, value, timestamp, ttl)
-               : new Column(name, value, timestamp);
-    }
-}


[4/6] Rename Column to Cell

Posted by sl...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/db/ExpiringColumn.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ExpiringColumn.java b/src/java/org/apache/cassandra/db/ExpiringColumn.java
deleted file mode 100644
index 92c8306..0000000
--- a/src/java/org/apache/cassandra/db/ExpiringColumn.java
+++ /dev/null
@@ -1,202 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.db;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.security.MessageDigest;
-
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.db.composites.CellName;
-import org.apache.cassandra.db.composites.CellNameType;
-import org.apache.cassandra.io.util.DataOutputBuffer;
-import org.apache.cassandra.serializers.MarshalException;
-import org.apache.cassandra.utils.Allocator;
-import org.apache.cassandra.utils.HeapAllocator;
-
-/**
- * Alternative to Column that have an expiring time.
- * ExpiringColumn is immutable (as Column is).
- *
- * Note that ExpiringColumn does not override Column.getMarkedForDeleteAt,
- * which means that it's in the somewhat unintuitive position of being deleted (after its expiration)
- * without having a time-at-which-it-became-deleted.  (Because ttl is a server-side measurement,
- * we can't mix it with the timestamp field, which is client-supplied and whose resolution we
- * can't assume anything about.)
- */
-public class ExpiringColumn extends Column
-{
-    public static final int MAX_TTL = 20 * 365 * 24 * 60 * 60; // 20 years in seconds
-
-    private final int localExpirationTime;
-    private final int timeToLive;
-
-    public ExpiringColumn(CellName name, ByteBuffer value, long timestamp, int timeToLive)
-    {
-      this(name, value, timestamp, timeToLive, (int) (System.currentTimeMillis() / 1000) + timeToLive);
-    }
-
-    public ExpiringColumn(CellName name, ByteBuffer value, long timestamp, int timeToLive, int localExpirationTime)
-    {
-        super(name, value, timestamp);
-        assert timeToLive > 0 : timeToLive;
-        assert localExpirationTime > 0 : localExpirationTime;
-        this.timeToLive = timeToLive;
-        this.localExpirationTime = localExpirationTime;
-    }
-
-    /** @return Either a DeletedColumn, or an ExpiringColumn. */
-    public static Column create(CellName name, ByteBuffer value, long timestamp, int timeToLive, int localExpirationTime, int expireBefore, ColumnSerializer.Flag flag)
-    {
-        if (localExpirationTime >= expireBefore || flag == ColumnSerializer.Flag.PRESERVE_SIZE)
-            return new ExpiringColumn(name, value, timestamp, timeToLive, localExpirationTime);
-        // The column is now expired, we can safely return a simple tombstone. Note that
-        // as long as the expiring column and the tombstone put together live longer than GC grace seconds,
-        // we'll fulfil our responsibility to repair.  See discussion at
-        // http://cassandra-user-incubator-apache-org.3065146.n2.nabble.com/repair-compaction-and-tombstone-rows-td7583481.html
-        return new DeletedColumn(name, localExpirationTime - timeToLive, timestamp);
-    }
-
-    public int getTimeToLive()
-    {
-        return timeToLive;
-    }
-
-    @Override
-    public Column withUpdatedName(CellName newName)
-    {
-        return new ExpiringColumn(newName, value, timestamp, timeToLive, localExpirationTime);
-    }
-
-    @Override
-    public Column withUpdatedTimestamp(long newTimestamp)
-    {
-        return new ExpiringColumn(name, value, newTimestamp, timeToLive, localExpirationTime);
-    }
-
-    @Override
-    public int dataSize()
-    {
-        return super.dataSize() + TypeSizes.NATIVE.sizeof(localExpirationTime) + TypeSizes.NATIVE.sizeof(timeToLive);
-    }
-
-    @Override
-    public int serializedSize(CellNameType type, TypeSizes typeSizes)
-    {
-        /*
-         * An expired column adds to a Column :
-         *    4 bytes for the localExpirationTime
-         *  + 4 bytes for the timeToLive
-        */
-        return super.serializedSize(type, typeSizes) + typeSizes.sizeof(localExpirationTime) + typeSizes.sizeof(timeToLive);
-    }
-
-    @Override
-    public void updateDigest(MessageDigest digest)
-    {
-        digest.update(name.toByteBuffer().duplicate());
-        digest.update(value.duplicate());
-
-        DataOutputBuffer buffer = new DataOutputBuffer();
-        try
-        {
-            buffer.writeLong(timestamp);
-            buffer.writeByte(serializationFlags());
-            buffer.writeInt(timeToLive);
-        }
-        catch (IOException e)
-        {
-            throw new RuntimeException(e);
-        }
-        digest.update(buffer.getData(), 0, buffer.getLength());
-    }
-
-    @Override
-    public int getLocalDeletionTime()
-    {
-        return localExpirationTime;
-    }
-
-    @Override
-    public Column localCopy(ColumnFamilyStore cfs)
-    {
-        return localCopy(cfs, HeapAllocator.instance);
-    }
-
-    @Override
-    public Column localCopy(ColumnFamilyStore cfs, Allocator allocator)
-    {
-        return new ExpiringColumn(name.copy(allocator), allocator.clone(value), timestamp, timeToLive, localExpirationTime);
-    }
-
-    @Override
-    public String getString(CellNameType comparator)
-    {
-        StringBuilder sb = new StringBuilder();
-        sb.append(super.getString(comparator));
-        sb.append("!");
-        sb.append(timeToLive);
-        return sb.toString();
-    }
-
-    @Override
-    public boolean isMarkedForDelete(long now)
-    {
-        return (int) (now / 1000) >= getLocalDeletionTime();
-    }
-
-    @Override
-    public long getMarkedForDeleteAt()
-    {
-        return timestamp;
-    }
-
-    @Override
-    public int serializationFlags()
-    {
-        return ColumnSerializer.EXPIRATION_MASK;
-    }
-
-    @Override
-    public void validateFields(CFMetaData metadata) throws MarshalException
-    {
-        super.validateFields(metadata);
-        if (timeToLive <= 0)
-            throw new MarshalException("A column TTL should be > 0");
-        if (localExpirationTime < 0)
-            throw new MarshalException("The local expiration time should not be negative");
-    }
-
-    @Override
-    public boolean equals(Object o)
-    {
-        // super.equals() returns false if o is not a CounterColumn
-        return super.equals(o)
-            && localExpirationTime == ((ExpiringColumn)o).localExpirationTime
-            && timeToLive == ((ExpiringColumn)o).timeToLive;
-    }
-
-    @Override
-    public int hashCode()
-    {
-        int result = super.hashCode();
-        result = 31 * result + localExpirationTime;
-        result = 31 * result + timeToLive;
-        return result;
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/db/HintedHandOffManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/HintedHandOffManager.java b/src/java/org/apache/cassandra/db/HintedHandOffManager.java
index 8868286..9b470e4 100644
--- a/src/java/org/apache/cassandra/db/HintedHandOffManager.java
+++ b/src/java/org/apache/cassandra/db/HintedHandOffManager.java
@@ -363,7 +363,7 @@ public class HintedHandOffManager implements HintedHandOffManagerMBean
 
             List<WriteResponseHandler> responseHandlers = Lists.newArrayList();
             Map<UUID, Long> truncationTimesCache = new HashMap<UUID, Long>();
-            for (final Column hint : hintsPage)
+            for (final Cell hint : hintsPage)
             {
                 // check if hints delivery has been paused during the process
                 if (hintedHandOffPaused)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/db/Keyspace.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Keyspace.java b/src/java/org/apache/cassandra/db/Keyspace.java
index 42f3de4..27939b6 100644
--- a/src/java/org/apache/cassandra/db/Keyspace.java
+++ b/src/java/org/apache/cassandra/db/Keyspace.java
@@ -401,10 +401,10 @@ public class Keyspace
             {
                 ColumnFamily cf = pager.next();
                 ColumnFamily cf2 = cf.cloneMeShallow();
-                for (Column column : cf)
+                for (Cell cell : cf)
                 {
-                    if (cfs.indexManager.indexes(column.name(), indexes))
-                        cf2.addColumn(column);
+                    if (cfs.indexManager.indexes(cell.name(), indexes))
+                        cf2.addColumn(cell);
                 }
                 cfs.indexManager.indexRow(key.key, cf2);
             }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/db/Memtable.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Memtable.java b/src/java/org/apache/cassandra/db/Memtable.java
index ffbdd37..11f11f8 100644
--- a/src/java/org/apache/cassandra/db/Memtable.java
+++ b/src/java/org/apache/cassandra/db/Memtable.java
@@ -35,7 +35,6 @@ import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.composites.CellNameType;
 import org.apache.cassandra.db.commitlog.ReplayPosition;
 import org.apache.cassandra.db.index.SecondaryIndexManager;
-import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.io.sstable.SSTableReader;
 import org.apache.cassandra.io.sstable.SSTableWriter;
 import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
@@ -101,9 +100,9 @@ public class Memtable
 
     private final Allocator allocator = DatabaseDescriptor.getMemtableAllocator();
     // We really only need one column by allocator but one by memtable is not a big waste and avoids needing allocators to know about CFS
-    private final Function<Column, Column> localCopyFunction = new Function<Column, Column>()
+    private final Function<Cell, Cell> localCopyFunction = new Function<Cell, Cell>()
     {
-        public Column apply(Column c)
+        public Cell apply(Cell c)
         {
             return c.localCopy(cfs, allocator);
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/db/OnDiskAtom.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/OnDiskAtom.java b/src/java/org/apache/cassandra/db/OnDiskAtom.java
index bda46d5..0115fd5 100644
--- a/src/java/org/apache/cassandra/db/OnDiskAtom.java
+++ b/src/java/org/apache/cassandra/db/OnDiskAtom.java
@@ -54,9 +54,9 @@ public interface OnDiskAtom
 
         public void serializeForSSTable(OnDiskAtom atom, DataOutput out) throws IOException
         {
-            if (atom instanceof Column)
+            if (atom instanceof Cell)
             {
-                type.columnSerializer().serialize((Column)atom, out);
+                type.columnSerializer().serialize((Cell)atom, out);
             }
             else
             {
@@ -88,9 +88,9 @@ public interface OnDiskAtom
 
         public long serializedSizeForSSTable(OnDiskAtom atom)
         {
-            if (atom instanceof Column)
+            if (atom instanceof Cell)
             {
-                return type.columnSerializer().serializedSize((Column)atom, TypeSizes.NATIVE);
+                return type.columnSerializer().serializedSize((Cell)atom, TypeSizes.NATIVE);
             }
             else
             {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/db/RangeTombstone.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/RangeTombstone.java b/src/java/org/apache/cassandra/db/RangeTombstone.java
index 9a60209..c10349a 100644
--- a/src/java/org/apache/cassandra/db/RangeTombstone.java
+++ b/src/java/org/apache/cassandra/db/RangeTombstone.java
@@ -173,7 +173,7 @@ public class RangeTombstone extends Interval<Composite, DeletionTime> implements
 
         /**
          * Update this tracker given an {@code atom}.
-         * If column is a Column, check if any tracked range is useless and
+         * If column is a Cell, check if any tracked range is useless and
          * can be removed. If it is a RangeTombstone, add it to this tracker.
          */
         public void update(OnDiskAtom atom)
@@ -200,7 +200,7 @@ public class RangeTombstone extends Interval<Composite, DeletionTime> implements
             }
             else
             {
-                assert atom instanceof Column;
+                assert atom instanceof Cell;
                 Iterator<RangeTombstone> iter = maxOrderingSet.iterator();
                 while (iter.hasNext())
                 {
@@ -221,13 +221,13 @@ public class RangeTombstone extends Interval<Composite, DeletionTime> implements
             }
         }
 
-        public boolean isDeleted(Column column)
+        public boolean isDeleted(Cell cell)
         {
             for (RangeTombstone tombstone : ranges)
             {
-                if (comparator.compare(column.name(), tombstone.min) >= 0
-                    && comparator.compare(column.name(), tombstone.max) <= 0
-                    && tombstone.maxTimestamp() >= column.timestamp())
+                if (comparator.compare(cell.name(), tombstone.min) >= 0
+                    && comparator.compare(cell.name(), tombstone.max) <= 0
+                    && tombstone.maxTimestamp() >= cell.timestamp())
                 {
                     return true;
                 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/db/SuperColumns.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/SuperColumns.java b/src/java/org/apache/cassandra/db/SuperColumns.java
index 402dc7e..a7d44dc 100644
--- a/src/java/org/apache/cassandra/db/SuperColumns.java
+++ b/src/java/org/apache/cassandra/db/SuperColumns.java
@@ -55,14 +55,14 @@ public class SuperColumns
          *   subcolumns range deletions).
          */
         DeletionInfo delInfo = scf.deletionInfo();
-        Map<CellName, List<Column>> scMap = groupSuperColumns(scf);
+        Map<CellName, List<Cell>> scMap = groupSuperColumns(scf);
 
         // Actually Serialize
         scf.getComparator().deletionInfoSerializer().serialize(new DeletionInfo(delInfo.getTopLevelDeletion()), out, version);
         out.writeInt(scMap.size());
 
         CellNameType subComparator = subType(scf.getComparator());
-        for (Map.Entry<CellName, List<Column>> entry : scMap.entrySet())
+        for (Map.Entry<CellName, List<Cell>> entry : scMap.entrySet())
         {
             scf.getComparator().cellSerializer().serialize(entry.getKey(), out);
 
@@ -72,35 +72,35 @@ public class SuperColumns
 
             out.writeInt(entry.getValue().size());
             ColumnSerializer serializer = subComparator.columnSerializer();
-            for (Column subColumn : entry.getValue())
-                serializer.serialize(subColumn, out);
+            for (Cell subCell : entry.getValue())
+                serializer.serialize(subCell, out);
         }
     }
 
-    private static Map<CellName, List<Column>> groupSuperColumns(ColumnFamily scf)
+    private static Map<CellName, List<Cell>> groupSuperColumns(ColumnFamily scf)
     {
         CellNameType type = scf.getComparator();
         // The order of insertion matters!
-        Map<CellName, List<Column>> scMap = new LinkedHashMap<>();
+        Map<CellName, List<Cell>> scMap = new LinkedHashMap<>();
 
         CellName scName = null;
-        List<Column> subColumns = null;
+        List<Cell> subCells = null;
         CellNameType scType = scType(type);
         CellNameType subType = subType(type);
-        for (Column column : scf)
+        for (Cell cell : scf)
         {
-            CellName newScName = scType.makeCellName(scName(column.name()));
-            CellName newSubName = subType.makeCellName(subName(column.name()));
+            CellName newScName = scType.makeCellName(scName(cell.name()));
+            CellName newSubName = subType.makeCellName(subName(cell.name()));
 
             if (scName == null || scType.compare(scName, newScName) != 0)
             {
-                // new super column
+                // new super cell
                 scName = newScName;
-                subColumns = new ArrayList<>();
-                scMap.put(scName, subColumns);
+                subCells = new ArrayList<>();
+                scMap.put(scName, subCells);
             }
 
-            subColumns.add(((Column)column).withUpdatedName(newSubName));
+            subCells.add(((Cell) cell).withUpdatedName(newSubName));
         }
         return scMap;
     }
@@ -118,7 +118,7 @@ public class SuperColumns
 
     public static long serializedSize(ColumnFamily scf, TypeSizes typeSizes, int version)
     {
-        Map<CellName, List<Column>> scMap = groupSuperColumns(scf);
+        Map<CellName, List<Cell>> scMap = groupSuperColumns(scf);
         DeletionInfo delInfo = scf.deletionInfo();
 
         // Actually Serialize
@@ -127,7 +127,7 @@ public class SuperColumns
         CellNameType scType = scType(scf.getComparator());
         CellNameType subType = subType(scf.getComparator());
         ColumnSerializer colSer = subType.columnSerializer();
-        for (Map.Entry<CellName, List<Column>> entry : scMap.entrySet())
+        for (Map.Entry<CellName, List<Cell>> entry : scMap.entrySet())
         {
             size += scType.cellSerializer().serializedSize(entry.getKey(), typeSizes);
 
@@ -136,8 +136,8 @@ public class SuperColumns
             size += DeletionTime.serializer.serializedSize(scDelInfo.getTopLevelDeletion(), TypeSizes.NATIVE);
 
             size += typeSizes.sizeof(entry.getValue().size());
-            for (Column subColumn : entry.getValue())
-                size += colSer.serializedSize(subColumn, typeSizes);
+            for (Cell subCell : entry.getValue())
+                size += colSer.serializedSize(subCell, typeSizes);
         }
         return size;
     }
@@ -154,7 +154,7 @@ public class SuperColumns
 
         private int read;
         private ByteBuffer scName;
-        private Iterator<Column> subColumnsIterator;
+        private Iterator<Cell> subColumnsIterator;
 
         private SCIterator(DataInput in, int superColumnCount, ColumnSerializer.Flag flag, int expireBefore, CellNameType type)
         {
@@ -176,7 +176,7 @@ public class SuperColumns
             {
                 if (subColumnsIterator != null && subColumnsIterator.hasNext())
                 {
-                    Column c = subColumnsIterator.next();
+                    Cell c = subColumnsIterator.next();
                     return c.withUpdatedName(type.makeCellName(scName, c.name().toByteBuffer()));
                 }
 
@@ -188,13 +188,13 @@ public class SuperColumns
 
                 /* read the number of columns */
                 int size = in.readInt();
-                List<Column> subColumns = new ArrayList<>(size);
+                List<Cell> subCells = new ArrayList<>(size);
 
                 ColumnSerializer colSer = subType(type).columnSerializer();
                 for (int i = 0; i < size; ++i)
-                    subColumns.add(colSer.deserialize(in, flag, expireBefore));
+                    subCells.add(colSer.deserialize(in, flag, expireBefore));
 
-                subColumnsIterator = subColumns.iterator();
+                subColumnsIterator = subCells.iterator();
 
                 // If the SC was deleted, return that first, otherwise return the first subcolumn
                 DeletionTime dtime = delInfo.getTopLevelDeletion();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/db/SystemKeyspace.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/SystemKeyspace.java b/src/java/org/apache/cassandra/db/SystemKeyspace.java
index e650d57..57ae146 100644
--- a/src/java/org/apache/cassandra/db/SystemKeyspace.java
+++ b/src/java/org/apache/cassandra/db/SystemKeyspace.java
@@ -596,7 +596,7 @@ public class SystemKeyspace
     public static void setIndexBuilt(String keyspaceName, String indexName)
     {
         ColumnFamily cf = ArrayBackedSortedColumns.factory.create(Keyspace.SYSTEM_KS, INDEX_CF);
-        cf.addColumn(new Column(cf.getComparator().makeCellName(indexName), ByteBufferUtil.EMPTY_BYTE_BUFFER, FBUtilities.timestampMicros()));
+        cf.addColumn(new Cell(cf.getComparator().makeCellName(indexName), ByteBufferUtil.EMPTY_BYTE_BUFFER, FBUtilities.timestampMicros()));
         RowMutation rm = new RowMutation(Keyspace.SYSTEM_KS, ByteBufferUtil.bytes(keyspaceName), cf);
         rm.apply();
     }
@@ -675,7 +675,7 @@ public class SystemKeyspace
         ByteBuffer ip = ByteBuffer.wrap(FBUtilities.getBroadcastAddress().getAddress());
 
         ColumnFamily cf = ArrayBackedSortedColumns.factory.create(Keyspace.SYSTEM_KS, COUNTER_ID_CF);
-        cf.addColumn(new Column(cf.getComparator().makeCellName(newCounterId.bytes()), ip, now));
+        cf.addColumn(new Cell(cf.getComparator().makeCellName(newCounterId.bytes()), ip, now));
         RowMutation rm = new RowMutation(Keyspace.SYSTEM_KS, ALL_LOCAL_NODE_ID_KEY, cf);
         rm.apply();
         forceBlockingFlush(COUNTER_ID_CF);
@@ -690,7 +690,7 @@ public class SystemKeyspace
         ColumnFamily cf = keyspace.getColumnFamilyStore(COUNTER_ID_CF).getColumnFamily(filter);
 
         CounterId previous = null;
-        for (Column c : cf)
+        for (Cell c : cf)
         {
             if (previous != null)
                 l.add(new CounterId.CounterIdRecord(previous, c.timestamp()));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/db/TreeMapBackedSortedColumns.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/TreeMapBackedSortedColumns.java b/src/java/org/apache/cassandra/db/TreeMapBackedSortedColumns.java
index 597cae6..252f243 100644
--- a/src/java/org/apache/cassandra/db/TreeMapBackedSortedColumns.java
+++ b/src/java/org/apache/cassandra/db/TreeMapBackedSortedColumns.java
@@ -33,7 +33,7 @@ import org.apache.cassandra.utils.Allocator;
 
 public class TreeMapBackedSortedColumns extends AbstractThreadUnsafeSortedColumns
 {
-    private final TreeMap<CellName, Column> map;
+    private final TreeMap<CellName, Cell> map;
 
     public static final ColumnFamily.Factory<TreeMapBackedSortedColumns> factory = new Factory<TreeMapBackedSortedColumns>()
     {
@@ -55,7 +55,7 @@ public class TreeMapBackedSortedColumns extends AbstractThreadUnsafeSortedColumn
         this.map = new TreeMap<>(metadata.comparator);
     }
 
-    private TreeMapBackedSortedColumns(CFMetaData metadata, SortedMap<CellName, Column> columns)
+    private TreeMapBackedSortedColumns(CFMetaData metadata, SortedMap<CellName, Cell> columns)
     {
         super(metadata);
         this.map = new TreeMap<>(columns);
@@ -77,58 +77,58 @@ public class TreeMapBackedSortedColumns extends AbstractThreadUnsafeSortedColumn
     }
 
     /*
-     * If we find an old column that has the same name
-     * the ask it to resolve itself else add the new column
+     * If we find an old cell that has the same name
+     * the ask it to resolve itself else add the new cell
     */
-    public void addColumn(Column column, Allocator allocator)
+    public void addColumn(Cell cell, Allocator allocator)
     {
-        CellName name = column.name();
+        CellName name = cell.name();
         // this is a slightly unusual way to structure this; a more natural way is shown in ThreadSafeSortedColumns,
         // but TreeMap lacks putAbsent.  Rather than split it into a "get, then put" check, we do it as follows,
         // which saves the extra "get" in the no-conflict case [for both normal and super columns],
         // in exchange for a re-put in the SuperColumn case.
-        Column oldColumn = map.put(name, column);
-        if (oldColumn == null)
+        Cell oldCell = map.put(name, cell);
+        if (oldCell == null)
             return;
 
         // calculate reconciled col from old (existing) col and new col
-        map.put(name, column.reconcile(oldColumn, allocator));
+        map.put(name, cell.reconcile(oldCell, allocator));
     }
 
     /**
      * We need to go through each column in the column container and resolve it before adding
      */
-    public void addAll(ColumnFamily cm, Allocator allocator, Function<Column, Column> transformation)
+    public void addAll(ColumnFamily cm, Allocator allocator, Function<Cell, Cell> transformation)
     {
         delete(cm.deletionInfo());
-        for (Column column : cm)
-            addColumn(transformation.apply(column), allocator);
+        for (Cell cell : cm)
+            addColumn(transformation.apply(cell), allocator);
     }
 
-    public boolean replace(Column oldColumn, Column newColumn)
+    public boolean replace(Cell oldCell, Cell newCell)
     {
-        if (!oldColumn.name().equals(newColumn.name()))
+        if (!oldCell.name().equals(newCell.name()))
             throw new IllegalArgumentException();
 
-        // We are not supposed to put the newColumn is either there was not
-        // column or the column was not equal to oldColumn (to be coherent
+        // We are not supposed to put the newCell is either there was not
+        // column or the column was not equal to oldCell (to be coherent
         // with other implementation). We optimize for the common case where
-        // oldColumn do is present though.
-        Column previous = map.put(oldColumn.name(), newColumn);
+        // oldCell do is present though.
+        Cell previous = map.put(oldCell.name(), newCell);
         if (previous == null)
         {
-            map.remove(oldColumn.name());
+            map.remove(oldCell.name());
             return false;
         }
-        if (!previous.equals(oldColumn))
+        if (!previous.equals(oldCell))
         {
-            map.put(oldColumn.name(), previous);
+            map.put(oldCell.name(), previous);
             return false;
         }
         return true;
     }
 
-    public Column getColumn(CellName name)
+    public Cell getColumn(CellName name)
     {
         return map.get(name);
     }
@@ -144,12 +144,12 @@ public class TreeMapBackedSortedColumns extends AbstractThreadUnsafeSortedColumn
         return map.size();
     }
 
-    public Collection<Column> getSortedColumns()
+    public Collection<Cell> getSortedColumns()
     {
         return map.values();
     }
 
-    public Collection<Column> getReverseSortedColumns()
+    public Collection<Cell> getReverseSortedColumns()
     {
         return map.descendingMap().values();
     }
@@ -159,17 +159,17 @@ public class TreeMapBackedSortedColumns extends AbstractThreadUnsafeSortedColumn
         return map.navigableKeySet();
     }
 
-    public Iterator<Column> iterator()
+    public Iterator<Cell> iterator()
     {
         return map.values().iterator();
     }
 
-    public Iterator<Column> iterator(ColumnSlice[] slices)
+    public Iterator<Cell> iterator(ColumnSlice[] slices)
     {
         return new ColumnSlice.NavigableMapIterator(map, slices);
     }
 
-    public Iterator<Column> reverseIterator(ColumnSlice[] slices)
+    public Iterator<Cell> reverseIterator(ColumnSlice[] slices)
     {
         return new ColumnSlice.NavigableMapIterator(map.descendingMap(), slices);
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/db/UnsortedColumns.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/UnsortedColumns.java b/src/java/org/apache/cassandra/db/UnsortedColumns.java
index d6520b0..a4957ae 100644
--- a/src/java/org/apache/cassandra/db/UnsortedColumns.java
+++ b/src/java/org/apache/cassandra/db/UnsortedColumns.java
@@ -37,7 +37,7 @@ import org.apache.cassandra.utils.Allocator;
  */
 public class UnsortedColumns extends AbstractThreadUnsafeSortedColumns
 {
-    private final ArrayList<Column> columns;
+    private final ArrayList<Cell> cells;
 
     public static final Factory<UnsortedColumns> factory = new Factory<UnsortedColumns>()
     {
@@ -50,13 +50,13 @@ public class UnsortedColumns extends AbstractThreadUnsafeSortedColumns
 
     private UnsortedColumns(CFMetaData metadata)
     {
-        this(metadata, new ArrayList<Column>());
+        this(metadata, new ArrayList<Cell>());
     }
 
-    private UnsortedColumns(CFMetaData metadata, ArrayList<Column> columns)
+    private UnsortedColumns(CFMetaData metadata, ArrayList<Cell> cells)
     {
         super(metadata);
-        this.columns = columns;
+        this.cells = cells;
     }
 
     public Factory getFactory()
@@ -66,7 +66,7 @@ public class UnsortedColumns extends AbstractThreadUnsafeSortedColumns
 
     public ColumnFamily cloneMe()
     {
-        return new UnsortedColumns(metadata, new ArrayList<Column>(columns));
+        return new UnsortedColumns(metadata, new ArrayList<Cell>(cells));
     }
 
     public boolean isInsertReversed()
@@ -76,68 +76,68 @@ public class UnsortedColumns extends AbstractThreadUnsafeSortedColumns
 
     public void clear()
     {
-        columns.clear();
+        cells.clear();
     }
 
-    public void addColumn(Column column, Allocator allocator)
+    public void addColumn(Cell cell, Allocator allocator)
     {
-        columns.add(column);
+        cells.add(cell);
     }
 
-    public void addAll(ColumnFamily cm, Allocator allocator, Function<Column, Column> transformation)
+    public void addAll(ColumnFamily cm, Allocator allocator, Function<Cell, Cell> transformation)
     {
         delete(cm.deletionInfo());
-        for (Column column : cm)
-            addColumn(column);
+        for (Cell cell : cm)
+            addColumn(cell);
     }
 
-    public Iterator<Column> iterator()
+    public Iterator<Cell> iterator()
     {
-        return columns.iterator();
+        return cells.iterator();
     }
 
-    public boolean replace(Column oldColumn, Column newColumn)
+    public boolean replace(Cell oldCell, Cell newCell)
     {
         throw new UnsupportedOperationException();
     }
 
-    public Column getColumn(CellName name)
+    public Cell getColumn(CellName name)
     {
         throw new UnsupportedOperationException();
     }
 
     public Iterable<CellName> getColumnNames()
     {
-        return Iterables.transform(columns, new Function<Column, CellName>()
+        return Iterables.transform(cells, new Function<Cell, CellName>()
         {
-            public CellName apply(Column column)
+            public CellName apply(Cell cell)
             {
-                return column.name;
+                return cell.name;
             }
         });
     }
 
-    public Collection<Column> getSortedColumns()
+    public Collection<Cell> getSortedColumns()
     {
         throw new UnsupportedOperationException();
     }
 
-    public Collection<Column> getReverseSortedColumns()
+    public Collection<Cell> getReverseSortedColumns()
     {
         throw new UnsupportedOperationException();
     }
 
     public int getColumnCount()
     {
-        return columns.size();
+        return cells.size();
     }
 
-    public Iterator<Column> iterator(ColumnSlice[] slices)
+    public Iterator<Cell> iterator(ColumnSlice[] slices)
     {
         throw new UnsupportedOperationException();
     }
 
-    public Iterator<Column> reverseIterator(ColumnSlice[] slices)
+    public Iterator<Cell> reverseIterator(ColumnSlice[] slices)
     {
         throw new UnsupportedOperationException();
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/db/columniterator/SSTableNamesIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/columniterator/SSTableNamesIterator.java b/src/java/org/apache/cassandra/db/columniterator/SSTableNamesIterator.java
index 41b95f9..2eb6745 100644
--- a/src/java/org/apache/cassandra/db/columniterator/SSTableNamesIterator.java
+++ b/src/java/org/apache/cassandra/db/columniterator/SSTableNamesIterator.java
@@ -156,7 +156,7 @@ public class SSTableNamesIterator extends AbstractIterator<OnDiskAtom> implement
         while (atomIterator.hasNext())
         {
             OnDiskAtom column = atomIterator.next();
-            if (column instanceof Column)
+            if (column instanceof Cell)
             {
                 if (columnNames.contains(column.name()))
                 {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/db/columniterator/SSTableSliceIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/columniterator/SSTableSliceIterator.java b/src/java/org/apache/cassandra/db/columniterator/SSTableSliceIterator.java
index d338580..0057d52 100644
--- a/src/java/org/apache/cassandra/db/columniterator/SSTableSliceIterator.java
+++ b/src/java/org/apache/cassandra/db/columniterator/SSTableSliceIterator.java
@@ -28,7 +28,7 @@ import org.apache.cassandra.io.sstable.SSTableReader;
 import org.apache.cassandra.io.util.FileDataInput;
 
 /**
- *  A Column Iterator over SSTable
+ *  A Cell Iterator over SSTable
  */
 public class SSTableSliceIterator implements OnDiskAtomIterator
 {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java
index dff423a..5e8dc9b 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogReplayer.java
@@ -328,7 +328,7 @@ public class CommitLogReplayer
                         rm = RowMutation.serializer.deserialize(new DataInputStream(bufIn), version, ColumnSerializer.Flag.LOCAL);
                         // doublecheck that what we read is [still] valid for the current schema
                         for (ColumnFamily cf : rm.getColumnFamilies())
-                            for (Column cell : cf)
+                            for (Cell cell : cf)
                                 cf.getComparator().validate(cell.name());
                     }
                     catch (UnknownColumnFamilyException ex)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
index 2090b6f..b72f91c 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
@@ -668,7 +668,7 @@ public class CompactionManager implements CompactionManagerMBean
         {
             private final Collection<Range<Token>> ranges;
             private final ColumnFamilyStore cfs;
-            private List<Column> indexedColumnsInRow;
+            private List<Cell> indexedColumnsInRow;
             private final CounterId.OneShotRenewer renewer;
 
             public Full(ColumnFamilyStore cfs, Collection<Range<Token>> ranges, CounterId.OneShotRenewer renewer)
@@ -699,15 +699,15 @@ public class CompactionManager implements CompactionManagerMBean
                 while (row.hasNext())
                 {
                     OnDiskAtom column = row.next();
-                    if (column instanceof CounterColumn)
-                        renewer.maybeRenew((CounterColumn) column);
+                    if (column instanceof CounterCell)
+                        renewer.maybeRenew((CounterCell) column);
 
-                    if (column instanceof Column && cfs.indexManager.indexes((Column) column))
+                    if (column instanceof Cell && cfs.indexManager.indexes((Cell) column))
                     {
                         if (indexedColumnsInRow == null)
                             indexedColumnsInRow = new ArrayList<>();
 
-                        indexedColumnsInRow.add((Column) column);
+                        indexedColumnsInRow.add((Cell) column);
                     }
                 }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/db/compaction/LazilyCompactedRow.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/LazilyCompactedRow.java b/src/java/org/apache/cassandra/db/compaction/LazilyCompactedRow.java
index 16e74e3..5d747a4 100644
--- a/src/java/org/apache/cassandra/db/compaction/LazilyCompactedRow.java
+++ b/src/java/org/apache/cassandra/db/compaction/LazilyCompactedRow.java
@@ -100,7 +100,7 @@ public class LazilyCompactedRow extends AbstractCompactedRow
 
         // if we have counters, remove old shards
         if (shouldPurge && cf.metadata().getDefaultValidator().isCommutative())
-            CounterColumn.mergeAndRemoveOldShards(key, cf, controller.gcBefore, controller.mergeShardBefore);
+            CounterCell.mergeAndRemoveOldShards(key, cf, controller.gcBefore, controller.mergeShardBefore);
     }
 
     public RowIndexEntry write(long currentPosition, DataOutput out) throws IOException
@@ -219,17 +219,17 @@ public class LazilyCompactedRow extends AbstractCompactedRow
             }
             else
             {
-                Column column = (Column) current;
-                container.addColumn(column);
+                Cell cell = (Cell) current;
+                container.addColumn(cell);
 
                 // skip the index-update checks if there is no indexing needed since they are a bit expensive
                 if (indexer == SecondaryIndexManager.nullUpdater)
                     return;
 
-                if (!column.isMarkedForDelete(System.currentTimeMillis())
-                    && !container.getColumn(column.name()).equals(column))
+                if (!cell.isMarkedForDelete(System.currentTimeMillis())
+                    && !container.getColumn(cell.name()).equals(cell))
                 {
-                    indexer.remove(column);
+                    indexer.remove(cell);
                 }
             }
         }
@@ -259,13 +259,13 @@ public class LazilyCompactedRow extends AbstractCompactedRow
                 // when we clear() the container, it removes the deletion info, so this needs to be reset each time
                 container.delete(maxRowTombstone);
                 removeDeletedAndOldShards(container, shouldPurge, key, controller);
-                Iterator<Column> iter = container.iterator();
+                Iterator<Cell> iter = container.iterator();
                 if (!iter.hasNext())
                 {
                     container.clear();
                     return null;
                 }
-                Column reduced = iter.next();
+                Cell reduced = iter.next();
                 container.clear();
 
                 // removeDeletedAndOldShards have only checked the top-level CF deletion times,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/db/composites/AbstractCellNameType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/composites/AbstractCellNameType.java b/src/java/org/apache/cassandra/db/composites/AbstractCellNameType.java
index 420ceab..2ca3b33 100644
--- a/src/java/org/apache/cassandra/db/composites/AbstractCellNameType.java
+++ b/src/java/org/apache/cassandra/db/composites/AbstractCellNameType.java
@@ -38,8 +38,8 @@ import org.apache.cassandra.utils.ByteBufferUtil;
 
 public abstract class AbstractCellNameType extends AbstractCType implements CellNameType
 {
-    private final Comparator<Column> columnComparator;
-    private final Comparator<Column> columnReverseComparator;
+    private final Comparator<Cell> columnComparator;
+    private final Comparator<Cell> columnReverseComparator;
     private final Comparator<OnDiskAtom> onDiskAtomComparator;
 
     private final ISerializer<CellName> cellSerializer;
@@ -50,16 +50,16 @@ public abstract class AbstractCellNameType extends AbstractCType implements Cell
 
     protected AbstractCellNameType()
     {
-        columnComparator = new Comparator<Column>()
+        columnComparator = new Comparator<Cell>()
         {
-            public int compare(Column c1, Column c2)
+            public int compare(Cell c1, Cell c2)
             {
                 return AbstractCellNameType.this.compare(c1.name(), c2.name());
             }
         };
-        columnReverseComparator = new Comparator<Column>()
+        columnReverseComparator = new Comparator<Cell>()
         {
-            public int compare(Column c1, Column c2)
+            public int compare(Cell c1, Cell c2)
             {
                 return AbstractCellNameType.this.compare(c2.name(), c1.name());
             }
@@ -122,12 +122,12 @@ public abstract class AbstractCellNameType extends AbstractCType implements Cell
         diskAtomFilterSerializer = new IDiskAtomFilter.Serializer(this);
     }
 
-    public Comparator<Column> columnComparator()
+    public Comparator<Cell> columnComparator()
     {
         return columnComparator;
     }
 
-    public Comparator<Column> columnReverseComparator()
+    public Comparator<Cell> columnReverseComparator()
     {
         return columnReverseComparator;
     }
@@ -220,7 +220,7 @@ public abstract class AbstractCellNameType extends AbstractCType implements Cell
     {
         return new CQL3Row.Builder()
         {
-            public Iterator<CQL3Row> group(final Iterator<Column> cells)
+            public Iterator<CQL3Row> group(final Iterator<Cell> cells)
             {
                 return new AbstractIterator<CQL3Row>()
                 {
@@ -228,7 +228,7 @@ public abstract class AbstractCellNameType extends AbstractCType implements Cell
                     {
                         while (cells.hasNext())
                         {
-                            final Column cell = cells.next();
+                            final Cell cell = cells.next();
                             if (cell.isMarkedForDelete(now))
                                 continue;
 
@@ -239,12 +239,12 @@ public abstract class AbstractCellNameType extends AbstractCType implements Cell
                                     return cell.name().get(i);
                                 }
 
-                                public Column getColumn(ColumnIdentifier name)
+                                public Cell getColumn(ColumnIdentifier name)
                                 {
                                     return cell;
                                 }
 
-                                public List<Column> getCollection(ColumnIdentifier name)
+                                public List<Cell> getCollection(ColumnIdentifier name)
                                 {
                                     return null;
                                 }
@@ -261,7 +261,7 @@ public abstract class AbstractCellNameType extends AbstractCType implements Cell
     {
         return new CQL3Row.Builder()
         {
-            public Iterator<CQL3Row> group(final Iterator<Column> cells)
+            public Iterator<CQL3Row> group(final Iterator<Cell> cells)
             {
                 return new AbstractIterator<CQL3Row>()
                 {
@@ -272,7 +272,7 @@ public abstract class AbstractCellNameType extends AbstractCType implements Cell
                     {
                         while (cells.hasNext())
                         {
-                            final Column cell = cells.next();
+                            final Cell cell = cells.next();
                             if (cell.isMarkedForDelete(now))
                                 continue;
 
@@ -305,8 +305,8 @@ public abstract class AbstractCellNameType extends AbstractCType implements Cell
     private static class CQL3RowOfSparse implements CQL3Row
     {
         private final CellName cell;
-        private Map<ColumnIdentifier, Column> columns;
-        private Map<ColumnIdentifier, List<Column>> collections;
+        private Map<ColumnIdentifier, Cell> columns;
+        private Map<ColumnIdentifier, List<Cell>> collections;
 
         CQL3RowOfSparse(CellName cell)
         {
@@ -318,7 +318,7 @@ public abstract class AbstractCellNameType extends AbstractCType implements Cell
             return cell.get(i);
         }
 
-        void add(Column cell)
+        void add(Cell cell)
         {
             CellName cellName = cell.name();
             ColumnIdentifier columnName =  cellName.cql3ColumnName();
@@ -327,10 +327,10 @@ public abstract class AbstractCellNameType extends AbstractCType implements Cell
                 if (collections == null)
                     collections = new HashMap<>();
 
-                List<Column> values = collections.get(columnName);
+                List<Cell> values = collections.get(columnName);
                 if (values == null)
                 {
-                    values = new ArrayList<Column>();
+                    values = new ArrayList<Cell>();
                     collections.put(columnName, values);
                 }
                 values.add(cell);
@@ -343,12 +343,12 @@ public abstract class AbstractCellNameType extends AbstractCType implements Cell
             }
         }
 
-        public Column getColumn(ColumnIdentifier name)
+        public Cell getColumn(ColumnIdentifier name)
         {
             return columns == null ? null : columns.get(name);
         }
 
-        public List<Column> getCollection(ColumnIdentifier name)
+        public List<Cell> getCollection(ColumnIdentifier name)
         {
             return collections == null ? null : collections.get(name);
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/db/composites/CellNameType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/composites/CellNameType.java b/src/java/org/apache/cassandra/db/composites/CellNameType.java
index 53d89f2..c4ddb60 100644
--- a/src/java/org/apache/cassandra/db/composites/CellNameType.java
+++ b/src/java/org/apache/cassandra/db/composites/CellNameType.java
@@ -24,7 +24,7 @@ import java.util.Comparator;
 
 import org.apache.cassandra.cql3.CQL3Row;
 import org.apache.cassandra.cql3.ColumnIdentifier;
-import org.apache.cassandra.db.Column;
+import org.apache.cassandra.db.Cell;
 import org.apache.cassandra.db.ColumnSerializer;
 import org.apache.cassandra.db.OnDiskAtom;
 import org.apache.cassandra.db.filter.IDiskAtomFilter;
@@ -162,8 +162,8 @@ public interface CellNameType extends CType
     // Ultimately, those might be split into an IVersionedSerializer and an ISSTableSerializer
     public ISerializer<CellName> cellSerializer();
 
-    public Comparator<Column> columnComparator();
-    public Comparator<Column> columnReverseComparator();
+    public Comparator<Cell> columnComparator();
+    public Comparator<Cell> columnReverseComparator();
     public Comparator<OnDiskAtom> onDiskAtomComparator();
 
     public ColumnSerializer columnSerializer();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/db/composites/CellNames.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/composites/CellNames.java b/src/java/org/apache/cassandra/db/composites/CellNames.java
index 15f4e16..7c8cbbd 100644
--- a/src/java/org/apache/cassandra/db/composites/CellNames.java
+++ b/src/java/org/apache/cassandra/db/composites/CellNames.java
@@ -20,7 +20,7 @@ package org.apache.cassandra.db.composites;
 import java.nio.ByteBuffer;
 import java.util.List;
 
-import org.apache.cassandra.db.Column;
+import org.apache.cassandra.db.Cell;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.marshal.ColumnToCollectionType;
 import org.apache.cassandra.db.marshal.CompositeType;
@@ -81,11 +81,11 @@ public abstract class CellNames
         return new CompoundDenseCellName(bbs);
     }
 
-    public static String getColumnsString(CellNameType type, Iterable<Column> columns)
+    public static String getColumnsString(CellNameType type, Iterable<Cell> columns)
     {
         StringBuilder builder = new StringBuilder();
-        for (Column column : columns)
-            builder.append(column.getString(type)).append(",");
+        for (Cell cell : columns)
+            builder.append(cell.getString(type)).append(",");
         return builder.toString();
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/db/filter/ColumnCounter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/filter/ColumnCounter.java b/src/java/org/apache/cassandra/db/filter/ColumnCounter.java
index 61f919e..982cd62 100644
--- a/src/java/org/apache/cassandra/db/filter/ColumnCounter.java
+++ b/src/java/org/apache/cassandra/db/filter/ColumnCounter.java
@@ -20,9 +20,9 @@
  */
 package org.apache.cassandra.db.filter;
 
+import org.apache.cassandra.db.Cell;
 import org.apache.cassandra.db.composites.CellName;
 import org.apache.cassandra.db.composites.CellNameType;
-import org.apache.cassandra.db.Column;
 import org.apache.cassandra.db.ColumnFamily;
 import org.apache.cassandra.db.DeletionInfo;
 
@@ -37,17 +37,17 @@ public class ColumnCounter
         this.timestamp = timestamp;
     }
 
-    public void count(Column column, DeletionInfo.InOrderTester tester)
+    public void count(Cell cell, DeletionInfo.InOrderTester tester)
     {
-        if (!isLive(column, tester, timestamp))
+        if (!isLive(cell, tester, timestamp))
             ignored++;
         else
             live++;
     }
 
-    protected static boolean isLive(Column column, DeletionInfo.InOrderTester tester, long timestamp)
+    protected static boolean isLive(Cell cell, DeletionInfo.InOrderTester tester, long timestamp)
     {
-        return column.isLive(timestamp) && (!tester.isDeleted(column));
+        return cell.isLive(timestamp) && (!tester.isDeleted(cell));
     }
 
     public int live()
@@ -66,7 +66,7 @@ public class ColumnCounter
             return this;
 
         DeletionInfo.InOrderTester tester = container.inOrderDeletionTester();
-        for (Column c : container)
+        for (Cell c : container)
             count(c, tester);
         return this;
     }
@@ -96,9 +96,9 @@ public class ColumnCounter
             assert toGroup == 0 || type != null;
         }
 
-        public void count(Column column, DeletionInfo.InOrderTester tester)
+        public void count(Cell cell, DeletionInfo.InOrderTester tester)
         {
-            if (!isLive(column, tester, timestamp))
+            if (!isLive(cell, tester, timestamp))
             {
                 ignored++;
                 return;
@@ -110,7 +110,7 @@ public class ColumnCounter
                 return;
             }
 
-            CellName current = column.name();
+            CellName current = cell.name();
             assert current.size() >= toGroup;
 
             if (last != null)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/db/filter/ColumnSlice.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/filter/ColumnSlice.java b/src/java/org/apache/cassandra/db/filter/ColumnSlice.java
index 5b504a5..4fd53bd 100644
--- a/src/java/org/apache/cassandra/db/filter/ColumnSlice.java
+++ b/src/java/org/apache/cassandra/db/filter/ColumnSlice.java
@@ -136,21 +136,21 @@ public class ColumnSlice
         }
     }
 
-    public static class NavigableMapIterator extends AbstractIterator<Column>
+    public static class NavigableMapIterator extends AbstractIterator<Cell>
     {
-        private final NavigableMap<CellName, Column> map;
+        private final NavigableMap<CellName, Cell> map;
         private final ColumnSlice[] slices;
 
         private int idx = 0;
-        private Iterator<Column> currentSlice;
+        private Iterator<Cell> currentSlice;
 
-        public NavigableMapIterator(NavigableMap<CellName, Column> map, ColumnSlice[] slices)
+        public NavigableMapIterator(NavigableMap<CellName, Cell> map, ColumnSlice[] slices)
         {
             this.map = map;
             this.slices = slices;
         }
 
-        protected Column computeNext()
+        protected Cell computeNext()
         {
             if (currentSlice == null)
             {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/db/filter/ExtendedFilter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/filter/ExtendedFilter.java b/src/java/org/apache/cassandra/db/filter/ExtendedFilter.java
index f5a6bc3..29976f6 100644
--- a/src/java/org/apache/cassandra/db/filter/ExtendedFilter.java
+++ b/src/java/org/apache/cassandra/db/filter/ExtendedFilter.java
@@ -291,11 +291,11 @@ public abstract class ExtendedFilter
                 if (def == null)
                 {
                     // This can't happen with CQL3 as this should be rejected upfront. For thrift however,
-                    // column name are not predefined. But that means the column name correspond to an internal one.
-                    Column column = data.getColumn(data.getComparator().cellFromByteBuffer(expression.column));
-                    if (column != null)
+                    // cell name are not predefined. But that means the cell name correspond to an internal one.
+                    Cell cell = data.getColumn(data.getComparator().cellFromByteBuffer(expression.column));
+                    if (cell != null)
                     {
-                        dataValue = column.value();
+                        dataValue = cell.value();
                         validator = data.metadata().getDefaultValidator();
                     }
                 }
@@ -363,8 +363,8 @@ public abstract class ExtendedFilter
                                    ? data.getComparator().cellFromByteBuffer(def.name.bytes)
                                    : data.getComparator().create(prefix, def.name);
 
-                    Column column = data.getColumn(cname);
-                    return column == null ? null : column.value();
+                    Cell cell = data.getColumn(cname);
+                    return cell == null ? null : cell.value();
                 case COMPACT_VALUE:
                     assert data.getColumnCount() == 1;
                     return data.getSortedColumns().iterator().next().value();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/db/filter/IDiskAtomFilter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/filter/IDiskAtomFilter.java b/src/java/org/apache/cassandra/db/filter/IDiskAtomFilter.java
index b55cfd7..2755afd 100644
--- a/src/java/org/apache/cassandra/db/filter/IDiskAtomFilter.java
+++ b/src/java/org/apache/cassandra/db/filter/IDiskAtomFilter.java
@@ -66,9 +66,9 @@ public interface IDiskAtomFilter
      * by the filter code, which should have some limit on the number of columns
      * to avoid running out of memory on large rows.
      */
-    public void collectReducedColumns(ColumnFamily container, Iterator<Column> reducedColumns, int gcBefore, long now);
+    public void collectReducedColumns(ColumnFamily container, Iterator<Cell> reducedColumns, int gcBefore, long now);
 
-    public Comparator<Column> getColumnComparator(CellNameType comparator);
+    public Comparator<Cell> getColumnComparator(CellNameType comparator);
 
     public boolean isReversed();
     public void updateColumnsLimit(int newLimit);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/db/filter/NamesQueryFilter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/filter/NamesQueryFilter.java b/src/java/org/apache/cassandra/db/filter/NamesQueryFilter.java
index 9058101..6f577b9 100644
--- a/src/java/org/apache/cassandra/db/filter/NamesQueryFilter.java
+++ b/src/java/org/apache/cassandra/db/filter/NamesQueryFilter.java
@@ -85,14 +85,14 @@ public class NamesQueryFilter implements IDiskAtomFilter
         return new SSTableNamesIterator(sstable, file, key, columns, indexEntry);
     }
 
-    public void collectReducedColumns(ColumnFamily container, Iterator<Column> reducedColumns, int gcBefore, long now)
+    public void collectReducedColumns(ColumnFamily container, Iterator<Cell> reducedColumns, int gcBefore, long now)
     {
         DeletionInfo.InOrderTester tester = container.inOrderDeletionTester();
         while (reducedColumns.hasNext())
             container.addIfRelevant(reducedColumns.next(), tester, gcBefore);
     }
 
-    public Comparator<Column> getColumnComparator(CellNameType comparator)
+    public Comparator<Cell> getColumnComparator(CellNameType comparator)
     {
         return comparator.columnComparator();
     }
@@ -122,9 +122,9 @@ public class NamesQueryFilter implements IDiskAtomFilter
             return cf.hasOnlyTombstones(now) ? 0 : 1;
 
         int count = 0;
-        for (Column column : cf)
+        for (Cell cell : cf)
         {
-            if (column.isLive(now))
+            if (cell.isLive(now))
                 count++;
         }
         return count;
@@ -185,9 +185,9 @@ public class NamesQueryFilter implements IDiskAtomFilter
             while (iter.hasNext())
             {
                 CellName current = iter.next();
-                Column column = cf.getColumn(current);
-                if (column != null)
-                    return column;
+                Cell cell = cf.getColumn(current);
+                if (cell != null)
+                    return cell;
             }
             return endOfData();
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/db/filter/QueryFilter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/filter/QueryFilter.java b/src/java/org/apache/cassandra/db/filter/QueryFilter.java
index 33e8904..8beb7ae 100644
--- a/src/java/org/apache/cassandra/db/filter/QueryFilter.java
+++ b/src/java/org/apache/cassandra/db/filter/QueryFilter.java
@@ -75,7 +75,7 @@ public class QueryFilter
 
     public static void collateOnDiskAtom(final ColumnFamily returnCF, List<? extends Iterator<? extends OnDiskAtom>> toCollate, IDiskAtomFilter filter, int gcBefore, long timestamp)
     {
-        List<Iterator<Column>> filteredIterators = new ArrayList<Iterator<Column>>(toCollate.size());
+        List<Iterator<Cell>> filteredIterators = new ArrayList<Iterator<Cell>>(toCollate.size());
         for (Iterator<? extends OnDiskAtom> iter : toCollate)
             filteredIterators.add(gatherTombstones(returnCF, iter));
         collateColumns(returnCF, filteredIterators, filter, gcBefore, timestamp);
@@ -86,39 +86,39 @@ public class QueryFilter
      */
     public void collateOnDiskAtom(ColumnFamily returnCF, Iterator<? extends OnDiskAtom> toCollate, int gcBefore)
     {
-        Iterator<Column> columns = gatherTombstones(returnCF, toCollate);
+        Iterator<Cell> columns = gatherTombstones(returnCF, toCollate);
         filter.collectReducedColumns(returnCF, columns, gcBefore, timestamp);
     }
 
-    public void collateColumns(final ColumnFamily returnCF, List<? extends Iterator<Column>> toCollate, int gcBefore)
+    public void collateColumns(final ColumnFamily returnCF, List<? extends Iterator<Cell>> toCollate, int gcBefore)
     {
         collateColumns(returnCF, toCollate, filter, gcBefore, timestamp);
     }
 
-    public static void collateColumns(final ColumnFamily returnCF, List<? extends Iterator<Column>> toCollate, IDiskAtomFilter filter, int gcBefore, long timestamp)
+    public static void collateColumns(final ColumnFamily returnCF, List<? extends Iterator<Cell>> toCollate, IDiskAtomFilter filter, int gcBefore, long timestamp)
     {
-        final Comparator<Column> fcomp = filter.getColumnComparator(returnCF.getComparator());
+        final Comparator<Cell> fcomp = filter.getColumnComparator(returnCF.getComparator());
         // define a 'reduced' iterator that merges columns w/ the same name, which
         // greatly simplifies computing liveColumns in the presence of tombstones.
-        MergeIterator.Reducer<Column, Column> reducer = new MergeIterator.Reducer<Column, Column>()
+        MergeIterator.Reducer<Cell, Cell> reducer = new MergeIterator.Reducer<Cell, Cell>()
         {
-            Column current;
+            Cell current;
 
-            public void reduce(Column next)
+            public void reduce(Cell next)
             {
                 assert current == null || fcomp.compare(current, next) == 0;
                 current = current == null ? next : current.reconcile(next, HeapAllocator.instance);
             }
 
-            protected Column getReduced()
+            protected Cell getReduced()
             {
                 assert current != null;
-                Column toReturn = current;
+                Cell toReturn = current;
                 current = null;
                 return toReturn;
             }
         };
-        Iterator<Column> reduced = MergeIterator.get(toCollate, fcomp, reducer);
+        Iterator<Cell> reduced = MergeIterator.get(toCollate, fcomp, reducer);
 
         filter.collectReducedColumns(returnCF, reduced, gcBefore, timestamp);
     }
@@ -127,11 +127,11 @@ public class QueryFilter
      * Given an iterator of on disk atom, returns an iterator that filters the tombstone range
      * markers adding them to {@code returnCF} and returns the normal column.
      */
-    public static Iterator<Column> gatherTombstones(final ColumnFamily returnCF, final Iterator<? extends OnDiskAtom> iter)
+    public static Iterator<Cell> gatherTombstones(final ColumnFamily returnCF, final Iterator<? extends OnDiskAtom> iter)
     {
-        return new Iterator<Column>()
+        return new Iterator<Cell>()
         {
-            private Column next;
+            private Cell next;
 
             public boolean hasNext()
             {
@@ -142,13 +142,13 @@ public class QueryFilter
                 return next != null;
             }
 
-            public Column next()
+            public Cell next()
             {
                 if (next == null)
                     getNext();
 
                 assert next != null;
-                Column toReturn = next;
+                Cell toReturn = next;
                 next = null;
                 return toReturn;
             }
@@ -159,9 +159,9 @@ public class QueryFilter
                 {
                     OnDiskAtom atom = iter.next();
 
-                    if (atom instanceof Column)
+                    if (atom instanceof Cell)
                     {
-                        next = (Column)atom;
+                        next = (Cell)atom;
                         break;
                     }
                     else

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/db/filter/SliceQueryFilter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/filter/SliceQueryFilter.java b/src/java/org/apache/cassandra/db/filter/SliceQueryFilter.java
index c3b019f..52fbb04 100644
--- a/src/java/org/apache/cassandra/db/filter/SliceQueryFilter.java
+++ b/src/java/org/apache/cassandra/db/filter/SliceQueryFilter.java
@@ -139,7 +139,7 @@ public class SliceQueryFilter implements IDiskAtomFilter
     public OnDiskAtomIterator getColumnFamilyIterator(final DecoratedKey key, final ColumnFamily cf)
     {
         assert cf != null;
-        final Iterator<Column> filteredIter = reversed ? cf.reverseIterator(slices) : cf.iterator(slices);
+        final Iterator<Cell> filteredIter = reversed ? cf.reverseIterator(slices) : cf.iterator(slices);
 
         return new OnDiskAtomIterator()
         {
@@ -182,24 +182,24 @@ public class SliceQueryFilter implements IDiskAtomFilter
         return new SSTableSliceIterator(sstable, file, key, slices, reversed, indexEntry);
     }
 
-    public Comparator<Column> getColumnComparator(CellNameType comparator)
+    public Comparator<Cell> getColumnComparator(CellNameType comparator)
     {
         return reversed ? comparator.columnReverseComparator() : comparator.columnComparator();
     }
 
-    public void collectReducedColumns(ColumnFamily container, Iterator<Column> reducedColumns, int gcBefore, long now)
+    public void collectReducedColumns(ColumnFamily container, Iterator<Cell> reducedColumns, int gcBefore, long now)
     {
         columnCounter = columnCounter(container.getComparator(), now);
         DeletionInfo.InOrderTester tester = container.deletionInfo().inOrderTester(reversed);
 
         while (reducedColumns.hasNext())
         {
-            Column column = reducedColumns.next();
+            Cell cell = reducedColumns.next();
             if (logger.isTraceEnabled())
                 logger.trace(String.format("collecting %s of %s: %s",
-                                           columnCounter.live(), count, column.getString(container.getComparator())));
+                                           columnCounter.live(), count, cell.getString(container.getComparator())));
 
-            columnCounter.count(column, tester);
+            columnCounter.count(cell, tester);
 
             if (columnCounter.live() > count)
                 break;
@@ -211,7 +211,7 @@ public class SliceQueryFilter implements IDiskAtomFilter
                 throw new TombstoneOverwhelmingException();
             }
 
-            container.addIfRelevant(column, tester, gcBefore);
+            container.addIfRelevant(cell, tester, gcBefore);
         }
 
         Tracing.trace("Read {} live and {} tombstoned cells", columnCounter.live(), columnCounter.ignored());
@@ -243,16 +243,16 @@ public class SliceQueryFilter implements IDiskAtomFilter
     {
         ColumnCounter counter = columnCounter(cf.getComparator(), now);
 
-        Collection<Column> columns = reversed
+        Collection<Cell> cells = reversed
                                    ? cf.getReverseSortedColumns()
                                    : cf.getSortedColumns();
 
         DeletionInfo.InOrderTester tester = cf.deletionInfo().inOrderTester(reversed);
 
-        for (Iterator<Column> iter = columns.iterator(); iter.hasNext(); )
+        for (Iterator<Cell> iter = cells.iterator(); iter.hasNext(); )
         {
-            Column column = iter.next();
-            counter.count(column, tester);
+            Cell cell = iter.next();
+            counter.count(cell, tester);
 
             if (counter.live() > trimTo)
             {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/db/index/AbstractSimplePerColumnSecondaryIndex.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/index/AbstractSimplePerColumnSecondaryIndex.java b/src/java/org/apache/cassandra/db/index/AbstractSimplePerColumnSecondaryIndex.java
index ce7a021..b062941 100644
--- a/src/java/org/apache/cassandra/db/index/AbstractSimplePerColumnSecondaryIndex.java
+++ b/src/java/org/apache/cassandra/db/index/AbstractSimplePerColumnSecondaryIndex.java
@@ -67,9 +67,9 @@ public abstract class AbstractSimplePerColumnSecondaryIndex extends PerColumnSec
         return new DecoratedKey(new LocalToken(getIndexKeyComparator(), value), value);
     }
 
-    protected abstract CellName makeIndexColumnName(ByteBuffer rowKey, Column column);
+    protected abstract CellName makeIndexColumnName(ByteBuffer rowKey, Cell cell);
 
-    protected abstract ByteBuffer getIndexedValue(ByteBuffer rowKey, Column column);
+    protected abstract ByteBuffer getIndexedValue(ByteBuffer rowKey, Cell cell);
 
     protected abstract AbstractType getExpressionComparator();
 
@@ -82,33 +82,33 @@ public abstract class AbstractSimplePerColumnSecondaryIndex extends PerColumnSec
                              baseCfs.metadata.getColumnDefinition(expr.column).type.getString(expr.value));
     }
 
-    public void delete(ByteBuffer rowKey, Column column)
+    public void delete(ByteBuffer rowKey, Cell cell)
     {
-        if (column.isMarkedForDelete(System.currentTimeMillis()))
+        if (cell.isMarkedForDelete(System.currentTimeMillis()))
             return;
 
-        DecoratedKey valueKey = getIndexKeyFor(getIndexedValue(rowKey, column));
+        DecoratedKey valueKey = getIndexKeyFor(getIndexedValue(rowKey, cell));
         int localDeletionTime = (int) (System.currentTimeMillis() / 1000);
         ColumnFamily cfi = ArrayBackedSortedColumns.factory.create(indexCfs.metadata);
-        cfi.addTombstone(makeIndexColumnName(rowKey, column), localDeletionTime, column.timestamp());
+        cfi.addTombstone(makeIndexColumnName(rowKey, cell), localDeletionTime, cell.timestamp());
         indexCfs.apply(valueKey, cfi, SecondaryIndexManager.nullUpdater);
         if (logger.isDebugEnabled())
             logger.debug("removed index entry for cleaned-up value {}:{}", valueKey, cfi);
     }
 
-    public void insert(ByteBuffer rowKey, Column column)
+    public void insert(ByteBuffer rowKey, Cell cell)
     {
-        DecoratedKey valueKey = getIndexKeyFor(getIndexedValue(rowKey, column));
+        DecoratedKey valueKey = getIndexKeyFor(getIndexedValue(rowKey, cell));
         ColumnFamily cfi = ArrayBackedSortedColumns.factory.create(indexCfs.metadata);
-        CellName name = makeIndexColumnName(rowKey, column);
-        if (column instanceof ExpiringColumn)
+        CellName name = makeIndexColumnName(rowKey, cell);
+        if (cell instanceof ExpiringCell)
         {
-            ExpiringColumn ec = (ExpiringColumn)column;
-            cfi.addColumn(new ExpiringColumn(name, ByteBufferUtil.EMPTY_BYTE_BUFFER, ec.timestamp(), ec.getTimeToLive(), ec.getLocalDeletionTime()));
+            ExpiringCell ec = (ExpiringCell) cell;
+            cfi.addColumn(new ExpiringCell(name, ByteBufferUtil.EMPTY_BYTE_BUFFER, ec.timestamp(), ec.getTimeToLive(), ec.getLocalDeletionTime()));
         }
         else
         {
-            cfi.addColumn(new Column(name, ByteBufferUtil.EMPTY_BYTE_BUFFER, column.timestamp()));
+            cfi.addColumn(new Cell(name, ByteBufferUtil.EMPTY_BYTE_BUFFER, cell.timestamp()));
         }
         if (logger.isDebugEnabled())
             logger.debug("applying index row {} in {}", indexCfs.metadata.getKeyValidator().getString(valueKey.key), cfi);
@@ -116,7 +116,7 @@ public abstract class AbstractSimplePerColumnSecondaryIndex extends PerColumnSec
         indexCfs.apply(valueKey, cfi, SecondaryIndexManager.nullUpdater);
     }
 
-    public void update(ByteBuffer rowKey, Column col)
+    public void update(ByteBuffer rowKey, Cell col)
     {
         insert(rowKey, col);
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/db/index/PerColumnSecondaryIndex.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/index/PerColumnSecondaryIndex.java b/src/java/org/apache/cassandra/db/index/PerColumnSecondaryIndex.java
index e77bd0f..15138ea 100644
--- a/src/java/org/apache/cassandra/db/index/PerColumnSecondaryIndex.java
+++ b/src/java/org/apache/cassandra/db/index/PerColumnSecondaryIndex.java
@@ -19,7 +19,7 @@ package org.apache.cassandra.db.index;
 
 import java.nio.ByteBuffer;
 
-import org.apache.cassandra.db.Column;
+import org.apache.cassandra.db.Cell;
 import org.apache.cassandra.utils.FBUtilities;
 
 /**
@@ -34,7 +34,7 @@ public abstract class PerColumnSecondaryIndex extends SecondaryIndex
      * @param rowKey the underlying row key which is indexed
      * @param col all the column info
      */
-    public abstract void delete(ByteBuffer rowKey, Column col);
+    public abstract void delete(ByteBuffer rowKey, Cell col);
 
     /**
      * insert a column to the index
@@ -42,7 +42,7 @@ public abstract class PerColumnSecondaryIndex extends SecondaryIndex
      * @param rowKey the underlying row key which is indexed
      * @param col all the column info
      */
-    public abstract void insert(ByteBuffer rowKey, Column col);
+    public abstract void insert(ByteBuffer rowKey, Cell col);
 
     /**
      * update a column from the index
@@ -50,7 +50,7 @@ public abstract class PerColumnSecondaryIndex extends SecondaryIndex
      * @param rowKey the underlying row key which is indexed
      * @param col all the column info
      */
-    public abstract void update(ByteBuffer rowKey, Column col);
+    public abstract void update(ByteBuffer rowKey, Cell col);
 
     public String getNameForSystemKeyspace(ByteBuffer column)
     {
@@ -58,8 +58,8 @@ public abstract class PerColumnSecondaryIndex extends SecondaryIndex
     }
 
     @Override
-    public boolean validate(Column column)
+    public boolean validate(Cell cell)
     {
-        return column.value().remaining() < FBUtilities.MAX_UNSIGNED_SHORT;
+        return cell.value().remaining() < FBUtilities.MAX_UNSIGNED_SHORT;
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/db/index/PerRowSecondaryIndex.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/index/PerRowSecondaryIndex.java b/src/java/org/apache/cassandra/db/index/PerRowSecondaryIndex.java
index 0419d83..7565c29 100644
--- a/src/java/org/apache/cassandra/db/index/PerRowSecondaryIndex.java
+++ b/src/java/org/apache/cassandra/db/index/PerRowSecondaryIndex.java
@@ -20,7 +20,7 @@ package org.apache.cassandra.db.index;
 import java.nio.ByteBuffer;
 import java.nio.charset.CharacterCodingException;
 
-import org.apache.cassandra.db.Column;
+import org.apache.cassandra.db.Cell;
 import org.apache.cassandra.db.ColumnFamily;
 import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.utils.ByteBufferUtil;
@@ -59,7 +59,7 @@ public abstract class PerRowSecondaryIndex extends SecondaryIndex
     }
 
     @Override
-    public boolean validate(Column column)
+    public boolean validate(Cell cell)
     {
         return true;
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/db/index/SecondaryIndex.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/index/SecondaryIndex.java b/src/java/org/apache/cassandra/db/index/SecondaryIndex.java
index 38d09be..84b3bfc 100644
--- a/src/java/org/apache/cassandra/db/index/SecondaryIndex.java
+++ b/src/java/org/apache/cassandra/db/index/SecondaryIndex.java
@@ -29,7 +29,7 @@ import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.db.composites.*;
-import org.apache.cassandra.db.Column;
+import org.apache.cassandra.db.Cell;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.SystemKeyspace;
@@ -324,7 +324,7 @@ public abstract class SecondaryIndex
         return index;
     }
 
-    public abstract boolean validate(Column column);
+    public abstract boolean validate(Cell cell);
 
     /**
      * Returns the index comparator for index backed by CFS, or null.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/db/index/SecondaryIndexManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/index/SecondaryIndexManager.java b/src/java/org/apache/cassandra/db/index/SecondaryIndexManager.java
index 27f0dd3..bbdcf9f 100644
--- a/src/java/org/apache/cassandra/db/index/SecondaryIndexManager.java
+++ b/src/java/org/apache/cassandra/db/index/SecondaryIndexManager.java
@@ -46,11 +46,11 @@ public class SecondaryIndexManager
 
     public static final Updater nullUpdater = new Updater()
     {
-        public void insert(Column column) { }
+        public void insert(Cell cell) { }
 
-        public void update(Column oldColumn, Column column) { }
+        public void update(Cell oldCell, Cell cell) { }
 
-        public void remove(Column current) { }
+        public void remove(Cell current) { }
 
         public void updateRowLevelIndexes() {}
     };
@@ -164,9 +164,9 @@ public class SecondaryIndexManager
         return matching == null ? Collections.<SecondaryIndex>emptyList() : matching;
     }
 
-    public boolean indexes(Column column)
+    public boolean indexes(Cell cell)
     {
-        return indexes(column.name());
+        return indexes(cell.name());
     }
 
     public boolean indexes(CellName name)
@@ -417,9 +417,9 @@ public class SecondaryIndexManager
             }
             else
             {
-                for (Column column : cf)
-                    if (index.indexes(column.name()))
-                        ((PerColumnSecondaryIndex) index).insert(key, column);
+                for (Cell cell : cf)
+                    if (index.indexes(cell.name()))
+                        ((PerColumnSecondaryIndex) index).insert(key, cell);
             }
         }
     }
@@ -430,15 +430,15 @@ public class SecondaryIndexManager
      * @param key the row key
      * @param indexedColumnsInRow all column names in row
      */
-    public void deleteFromIndexes(DecoratedKey key, List<Column> indexedColumnsInRow)
+    public void deleteFromIndexes(DecoratedKey key, List<Cell> indexedColumnsInRow)
     {
         // Update entire row only once per row level index
         Set<Class<? extends SecondaryIndex>> cleanedRowLevelIndexes = null;
 
-        for (Column column : indexedColumnsInRow)
+        for (Cell cell : indexedColumnsInRow)
         {
             // TODO: this is probably incorrect, we should pull all indexes
-            SecondaryIndex index = indexesByColumn.get(column.name().toByteBuffer());
+            SecondaryIndex index = indexesByColumn.get(cell.name().toByteBuffer());
             if (index == null)
                 continue;
 
@@ -452,7 +452,7 @@ public class SecondaryIndexManager
             }
             else
             {
-                ((PerColumnSecondaryIndex) index).delete(key.key, column);
+                ((PerColumnSecondaryIndex) index).delete(key.key, cell);
             }
         }
     }
@@ -558,11 +558,11 @@ public class SecondaryIndexManager
             index.setIndexRemoved();
     }
 
-    public boolean validate(Column column)
+    public boolean validate(Cell cell)
     {
-        for (SecondaryIndex index : indexFor(column.name()))
+        for (SecondaryIndex index : indexFor(cell.name()))
         {
-            if (!index.validate(column))
+            if (!index.validate(cell))
                 return false;
         }
         return true;
@@ -571,13 +571,13 @@ public class SecondaryIndexManager
     public static interface Updater
     {
         /** called when constructing the index against pre-existing data */
-        public void insert(Column column);
+        public void insert(Cell cell);
 
         /** called when updating the index from a memtable */
-        public void update(Column oldColumn, Column column);
+        public void update(Cell oldCell, Cell cell);
 
         /** called when lazy-updating the index during compaction (CASSANDRA-2897) */
-        public void remove(Column current);
+        public void remove(Cell current);
 
         /** called after memtable updates are complete (CASSANDRA-5397) */
         public void updateRowLevelIndexes();
@@ -594,42 +594,42 @@ public class SecondaryIndexManager
             this.cf = cf;
         }
 
-        public void insert(Column column)
+        public void insert(Cell cell)
         {
-            if (column.isMarkedForDelete(System.currentTimeMillis()))
+            if (cell.isMarkedForDelete(System.currentTimeMillis()))
                 return;
 
-            for (SecondaryIndex index : indexFor(column.name()))
+            for (SecondaryIndex index : indexFor(cell.name()))
                 if (index instanceof PerColumnSecondaryIndex)
-                    ((PerColumnSecondaryIndex) index).insert(key.key, column);
+                    ((PerColumnSecondaryIndex) index).insert(key.key, cell);
         }
 
-        public void update(Column oldColumn, Column column)
+        public void update(Cell oldCell, Cell cell)
         {
-            if (oldColumn.equals(column))
+            if (oldCell.equals(cell))
                 return;
             
-            for (SecondaryIndex index : indexFor(column.name()))
+            for (SecondaryIndex index : indexFor(cell.name()))
             {
                 if (index instanceof PerColumnSecondaryIndex)
                 {
                     // insert the new value before removing the old one, so we never have a period
                     // where the row is invisible to both queries (the opposite seems preferable); see CASSANDRA-5540
-                    if (!column.isMarkedForDelete(System.currentTimeMillis()))
-                        ((PerColumnSecondaryIndex) index).insert(key.key, column);
-                    ((PerColumnSecondaryIndex) index).delete(key.key, oldColumn);
+                    if (!cell.isMarkedForDelete(System.currentTimeMillis()))
+                        ((PerColumnSecondaryIndex) index).insert(key.key, cell);
+                    ((PerColumnSecondaryIndex) index).delete(key.key, oldCell);
                 }
             }
         }
 
-        public void remove(Column column)
+        public void remove(Cell cell)
         {
-            if (column.isMarkedForDelete(System.currentTimeMillis()))
+            if (cell.isMarkedForDelete(System.currentTimeMillis()))
                 return;
 
-            for (SecondaryIndex index : indexFor(column.name()))
+            for (SecondaryIndex index : indexFor(cell.name()))
                 if (index instanceof PerColumnSecondaryIndex)
-                   ((PerColumnSecondaryIndex) index).delete(key.key, column);
+                   ((PerColumnSecondaryIndex) index).delete(key.key, cell);
         }
 
         public void updateRowLevelIndexes()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/db/index/composites/CompositesIndex.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/index/composites/CompositesIndex.java b/src/java/org/apache/cassandra/db/index/composites/CompositesIndex.java
index 95314cf..839df26 100644
--- a/src/java/org/apache/cassandra/db/index/composites/CompositesIndex.java
+++ b/src/java/org/apache/cassandra/db/index/composites/CompositesIndex.java
@@ -113,14 +113,14 @@ public abstract class CompositesIndex extends AbstractSimplePerColumnSecondaryIn
         throw new AssertionError();
     }
 
-    protected CellName makeIndexColumnName(ByteBuffer rowKey, Column column)
+    protected CellName makeIndexColumnName(ByteBuffer rowKey, Cell cell)
     {
-        return getIndexComparator().create(makeIndexColumnPrefix(rowKey, column.name()), null);
+        return getIndexComparator().create(makeIndexColumnPrefix(rowKey, cell.name()), null);
     }
 
     protected abstract Composite makeIndexColumnPrefix(ByteBuffer rowKey, Composite columnName);
 
-    public abstract IndexedEntry decodeEntry(DecoratedKey indexedValue, Column indexEntry);
+    public abstract IndexedEntry decodeEntry(DecoratedKey indexedValue, Cell indexEntry);
 
     public abstract boolean isStale(IndexedEntry entry, ColumnFamily data, long now);
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnClusteringKey.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnClusteringKey.java b/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnClusteringKey.java
index 38c55fd..a795d7f 100644
--- a/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnClusteringKey.java
+++ b/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnClusteringKey.java
@@ -61,9 +61,9 @@ public class CompositesIndexOnClusteringKey extends CompositesIndex
         return new CompoundDenseCellNameType(types);
     }
 
-    protected ByteBuffer getIndexedValue(ByteBuffer rowKey, Column column)
+    protected ByteBuffer getIndexedValue(ByteBuffer rowKey, Cell cell)
     {
-        return column.name().get(columnDef.position());
+        return cell.name().get(columnDef.position());
     }
 
     protected Composite makeIndexColumnPrefix(ByteBuffer rowKey, Composite columnName)
@@ -78,7 +78,7 @@ public class CompositesIndexOnClusteringKey extends CompositesIndex
         return builder.build();
     }
 
-    public IndexedEntry decodeEntry(DecoratedKey indexedValue, Column indexEntry)
+    public IndexedEntry decodeEntry(DecoratedKey indexedValue, Cell indexEntry)
     {
         int ckCount = baseCfs.metadata.clusteringColumns().size();
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e50d6af1/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnCollectionKey.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnCollectionKey.java b/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnCollectionKey.java
index f3daaf2..36504ca 100644
--- a/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnCollectionKey.java
+++ b/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnCollectionKey.java
@@ -64,9 +64,9 @@ public class CompositesIndexOnCollectionKey extends CompositesIndex
         return ((CollectionType)columnDef.type).nameComparator();
     }
 
-    protected ByteBuffer getIndexedValue(ByteBuffer rowKey, Column column)
+    protected ByteBuffer getIndexedValue(ByteBuffer rowKey, Cell cell)
     {
-        return column.name().get(columnDef.position() + 1);
+        return cell.name().get(columnDef.position() + 1);
     }
 
     protected Composite makeIndexColumnPrefix(ByteBuffer rowKey, Composite cellName)
@@ -79,7 +79,7 @@ public class CompositesIndexOnCollectionKey extends CompositesIndex
         return builder.build();
     }
 
-    public IndexedEntry decodeEntry(DecoratedKey indexedValue, Column indexEntry)
+    public IndexedEntry decodeEntry(DecoratedKey indexedValue, Cell indexEntry)
     {
         int count = 1 + baseCfs.metadata.clusteringColumns().size();
         CBuilder builder = baseCfs.getComparator().builder();
@@ -100,7 +100,7 @@ public class CompositesIndexOnCollectionKey extends CompositesIndex
     public boolean isStale(IndexedEntry entry, ColumnFamily data, long now)
     {
         CellName name = data.getComparator().create(entry.indexedEntryPrefix, columnDef.name, entry.indexValue.key);
-        Column liveColumn = data.getColumn(name);
-        return (liveColumn == null || liveColumn.isMarkedForDelete(now));
+        Cell liveCell = data.getColumn(name);
+        return (liveCell == null || liveCell.isMarkedForDelete(now));
     }
 }