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

[4/6] Rename Column to Cell

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));
     }
 }