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 2014/04/29 14:05:03 UTC

[1/7] Push more of memtable data off-heap

Repository: cassandra
Updated Branches:
  refs/heads/cassandra-2.1 d402cf687 -> 8541cca71


http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/test/unit/org/apache/cassandra/repair/ValidatorTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/repair/ValidatorTest.java b/test/unit/org/apache/cassandra/repair/ValidatorTest.java
index 8da900a..48b8ac9 100644
--- a/test/unit/org/apache/cassandra/repair/ValidatorTest.java
+++ b/test/unit/org/apache/cassandra/repair/ValidatorTest.java
@@ -26,6 +26,7 @@ import org.junit.After;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.db.BufferDecoratedKey;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.Keyspace;
@@ -109,7 +110,7 @@ public class ValidatorTest extends SchemaLoader
 
         // add a row
         Token mid = partitioner.midpoint(range.left, range.right);
-        validator.add(new CompactedRowStub(new DecoratedKey(mid, ByteBufferUtil.bytes("inconceivable!"))));
+        validator.add(new CompactedRowStub(new BufferDecoratedKey(mid, ByteBufferUtil.bytes("inconceivable!"))));
         validator.complete();
 
         // confirm that the tree was validated

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/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 26295af..ad70eff 100644
--- a/test/unit/org/apache/cassandra/service/QueryPagerTest.java
+++ b/test/unit/org/apache/cassandra/service/QueryPagerTest.java
@@ -100,7 +100,7 @@ public class QueryPagerTest extends SchemaLoader
     {
         StringBuilder sb = new StringBuilder();
         for (Row row : rows)
-            sb.append(string(row.key.key)).append(":").append(toString(row.cf)).append("\n");
+            sb.append(string(row.key.getKey())).append(":").append(toString(row.cf)).append("\n");
         return sb.toString();
     }
 
@@ -159,7 +159,7 @@ public class QueryPagerTest extends SchemaLoader
 
     private static void assertRow(Row r, String key, ByteBuffer... names)
     {
-        assertEquals(key, string(r.key.key));
+        assertEquals(key, string(r.key.getKey()));
         assertNotNull(r.cf);
         int i = 0;
         for (Cell c : r.cf)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/test/unit/org/apache/cassandra/service/pager/AbstractQueryPagerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/pager/AbstractQueryPagerTest.java b/test/unit/org/apache/cassandra/service/pager/AbstractQueryPagerTest.java
index 0bb1660..cc03e5d 100644
--- a/test/unit/org/apache/cassandra/service/pager/AbstractQueryPagerTest.java
+++ b/test/unit/org/apache/cassandra/service/pager/AbstractQueryPagerTest.java
@@ -105,7 +105,7 @@ public class AbstractQueryPagerTest
 
     private void assertRow(Row row, String name, int... values)
     {
-        assertEquals(row.key.key, ByteBufferUtil.bytes(name));
+        assertEquals(row.key.getKey(), ByteBufferUtil.bytes(name));
         assertEquals(values.length, row.cf.getColumnCount());
 
         int i = 0;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/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 55b6f77..7d3b274 100644
--- a/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
+++ b/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
@@ -167,7 +167,7 @@ public class StreamingTransferTest extends SchemaLoader
             String key = "key" + offs[i];
             String col = "col" + offs[i];
             assert cfs.getColumnFamily(QueryFilter.getIdentityFilter(Util.dk(key), cfs.name, System.currentTimeMillis())) != null;
-            assert rows.get(i).key.key.equals(ByteBufferUtil.bytes(key));
+            assert rows.get(i).key.getKey().equals(ByteBufferUtil.bytes(key));
             assert rows.get(i).cf.getColumn(cellname(col)) != null;
         }
 
@@ -229,7 +229,7 @@ public class StreamingTransferTest extends SchemaLoader
                 long val = key.hashCode();
                 ColumnFamily cf = ArrayBackedSortedColumns.factory.create(keyspace.getName(), cfs.name);
                 cf.addColumn(column(col, "v", timestamp));
-                cf.addColumn(new Cell(cellname("birthdate"), ByteBufferUtil.bytes(val), timestamp));
+                cf.addColumn(new BufferCell(cellname("birthdate"), ByteBufferUtil.bytes(val), timestamp));
                 Mutation rm = new Mutation("Keyspace1", ByteBufferUtil.bytes(key), cf);
                 logger.debug("Applying row to transfer " + rm);
                 rm.apply();
@@ -248,7 +248,7 @@ public class StreamingTransferTest extends SchemaLoader
             Range<RowPosition> range = Util.range("", "");
             List<Row> rows = cfs.search(range, clause, filter, 100);
             assertEquals(1, rows.size());
-            assert rows.get(0).key.key.equals(ByteBufferUtil.bytes(key));
+            assert rows.get(0).key.getKey().equals(ByteBufferUtil.bytes(key));
         }
     }
 
@@ -320,8 +320,8 @@ public class StreamingTransferTest extends SchemaLoader
                 state.writeRemote(CounterId.fromInt(4), 4L, 2L);
                 state.writeRemote(CounterId.fromInt(6), 3L, 3L);
                 state.writeRemote(CounterId.fromInt(8), 2L, 4L);
-                cf.addColumn(new CounterCell(cellname(col), state.context, timestamp));
-                cfCleaned.addColumn(new CounterCell(cellname(col), cc.clearAllLocal(state.context), timestamp));
+                cf.addColumn(new BufferCounterCell(cellname(col), state.context, timestamp));
+                cfCleaned.addColumn(new BufferCounterCell(cellname(col), cc.clearAllLocal(state.context), timestamp));
 
                 entries.put(key, cf);
                 cleanedEntries.put(key, cfCleaned);
@@ -382,8 +382,8 @@ public class StreamingTransferTest extends SchemaLoader
         ColumnFamilyStore cfstore = Keyspace.open(keyspaceName).getColumnFamilyStore(cfname);
         List<Row> rows = Util.getRangeSlice(cfstore);
         assertEquals(2, rows.size());
-        assert rows.get(0).key.key.equals(ByteBufferUtil.bytes("test"));
-        assert rows.get(1).key.key.equals(ByteBufferUtil.bytes("transfer3"));
+        assert rows.get(0).key.getKey().equals(ByteBufferUtil.bytes("test"));
+        assert rows.get(1).key.getKey().equals(ByteBufferUtil.bytes("transfer3"));
         assert rows.get(0).cf.getColumnCount() == 1;
         assert rows.get(1).cf.getColumnCount() == 1;
 
@@ -422,9 +422,9 @@ public class StreamingTransferTest extends SchemaLoader
         Map.Entry<DecoratedKey,String> last = keys.lastEntry();
         Map.Entry<DecoratedKey,String> secondtolast = keys.lowerEntry(last.getKey());
         List<Range<Token>> ranges = new ArrayList<>();
-        ranges.add(new Range<>(p.getMinimumToken(), first.getKey().token));
+        ranges.add(new Range<>(p.getMinimumToken(), first.getKey().getToken()));
         // the left hand side of the range is exclusive, so we transfer from the second-to-last token
-        ranges.add(new Range<>(secondtolast.getKey().token, p.getMinimumToken()));
+        ranges.add(new Range<>(secondtolast.getKey().getToken(), p.getMinimumToken()));
 
         // Acquiring references, transferSSTables needs it
         if (!SSTableReader.acquireReferences(ssTableReaders))
@@ -453,7 +453,7 @@ public class StreamingTransferTest extends SchemaLoader
             {
                 ColumnFamily cf = ArrayBackedSortedColumns.factory.create(keyspace.getName(), cfs.name);
                 cf.addColumn(column(colName, "value", timestamp));
-                cf.addColumn(new Cell(cellname("birthdate"), ByteBufferUtil.bytes(new Date(timestamp).toString()), timestamp));
+                cf.addColumn(new BufferCell(cellname("birthdate"), ByteBufferUtil.bytes(new Date(timestamp).toString()), timestamp));
                 Mutation rm = new Mutation("Keyspace1", ByteBufferUtil.bytes(key), cf);
                 logger.debug("Applying row to transfer " + rm);
                 rm.apply();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/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 9e2014d..d9133f6 100644
--- a/test/unit/org/apache/cassandra/tools/SSTableExportTest.java
+++ b/test/unit/org/apache/cassandra/tools/SSTableExportTest.java
@@ -101,7 +101,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 ExpiringCell(Util.cellname("colExp"), ByteBufferUtil.bytes("valExp"), System.currentTimeMillis(), 42, nowInSec));
+        cfamily.addColumn(new BufferExpiringCell(Util.cellname("colExp"), ByteBufferUtil.bytes("valExp"), System.currentTimeMillis(), 42, nowInSec));
         writer.append(Util.dk("rowA"), cfamily);
         cfamily.clear();
 
@@ -194,7 +194,7 @@ public class SSTableExportTest extends SchemaLoader
         SSTableWriter writer = new SSTableWriter(tempSS.getPath(), 2, ActiveRepairService.UNREPAIRED_SSTABLE);
 
         // Add rowA
-        cfamily.addColumn(CounterCell.createLocal(Util.cellname("colA"), 42, System.currentTimeMillis(), Long.MIN_VALUE));
+        cfamily.addColumn(BufferCounterCell.createLocal(Util.cellname("colA"), 42, System.currentTimeMillis(), Long.MIN_VALUE));
         writer.append(Util.dk("rowA"), cfamily);
         cfamily.clear();
 
@@ -225,7 +225,7 @@ public class SSTableExportTest extends SchemaLoader
         SSTableWriter writer = new SSTableWriter(tempSS.getPath(), 2, ActiveRepairService.UNREPAIRED_SSTABLE);
 
         // Add rowA
-        cfamily.addColumn(new Cell(Util.cellname("data"), UTF8Type.instance.fromString("{\"foo\":\"bar\"}")));
+        cfamily.addColumn(new BufferCell(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/8541cca7/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 3befc44..2fdeaf4 100644
--- a/test/unit/org/apache/cassandra/tools/SSTableImportTest.java
+++ b/test/unit/org/apache/cassandra/tools/SSTableImportTest.java
@@ -55,7 +55,7 @@ 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 DeletedCell);
+        assert !(cf.getColumn(Util.cellname("colAA")) instanceof BufferDeletedCell);
         Cell expCol = cf.getColumn(Util.cellname("colAC"));
         assert expCol.value().equals(hexToBytes("76616c4143"));
         assert expCol instanceof ExpiringCell;
@@ -89,7 +89,7 @@ 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 DeletedCell);
+        assert !(cf.getColumn(Util.cellname("colAA")) instanceof BufferDeletedCell);
         Cell expCol = cf.getColumn(Util.cellname("colAC"));
         assert expCol.value().equals(hexToBytes("76616c4143"));
         assert expCol instanceof ExpiringCell;
@@ -113,7 +113,7 @@ 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 DeletedCell);
+        assert !(cf.getColumn(Util.cellname("colAA")) instanceof BufferDeletedCell);
         Cell expCol = cf.getColumn(Util.cellname("colAC"));
         assert expCol.value().equals(hexToBytes("76616c4143"));
         assert expCol instanceof ExpiringCell;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/test/unit/org/apache/cassandra/triggers/TriggerExecutorTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/triggers/TriggerExecutorTest.java b/test/unit/org/apache/cassandra/triggers/TriggerExecutorTest.java
index b0c45b9..7d009c1 100644
--- a/test/unit/org/apache/cassandra/triggers/TriggerExecutorTest.java
+++ b/test/unit/org/apache/cassandra/triggers/TriggerExecutorTest.java
@@ -243,10 +243,10 @@ public class TriggerExecutorTest
         ColumnFamily cf = ArrayBackedSortedColumns.factory.create(metadata);
 
         if (columnValue1 != null)
-            cf.addColumn(new Cell(getColumnName(metadata, "c1"), bytes(columnValue1)));
+            cf.addColumn(new BufferCell(getColumnName(metadata, "c1"), bytes(columnValue1)));
 
         if (columnValue2 != null)
-            cf.addColumn(new Cell(getColumnName(metadata, "c2"), bytes(columnValue2)));
+            cf.addColumn(new BufferCell(getColumnName(metadata, "c2"), bytes(columnValue2)));
 
         return cf;
     }
@@ -269,7 +269,7 @@ public class TriggerExecutorTest
         public Collection<Mutation> augment(ByteBuffer key, ColumnFamily update)
         {
             ColumnFamily cf = ArrayBackedSortedColumns.factory.create(update.metadata());
-            cf.addColumn(new Cell(getColumnName(update.metadata(), "c2"), bytes("trigger")));
+            cf.addColumn(new BufferCell(getColumnName(update.metadata(), "c2"), bytes("trigger")));
             return Collections.singletonList(new Mutation(update.metadata().ksName, key, cf));
         }
     }
@@ -282,7 +282,7 @@ public class TriggerExecutorTest
                 return null;
 
             ColumnFamily cf = ArrayBackedSortedColumns.factory.create(update.metadata());
-            cf.addColumn(new Cell(getColumnName(update.metadata(), "c2"), bytes("trigger")));
+            cf.addColumn(new BufferCell(getColumnName(update.metadata(), "c2"), bytes("trigger")));
             return Collections.singletonList(new Mutation(update.metadata().ksName, key, cf));
         }
     }
@@ -292,7 +292,7 @@ public class TriggerExecutorTest
         public Collection<Mutation> augment(ByteBuffer key, ColumnFamily update)
         {
             ColumnFamily cf = ArrayBackedSortedColumns.factory.create(makeCfMetaData(update.metadata().ksName, "otherCf", null));
-            cf.addColumn(new Cell(getColumnName(update.metadata(), "c2"), bytes("trigger")));
+            cf.addColumn(new BufferCell(getColumnName(update.metadata(), "c2"), bytes("trigger")));
             return Collections.singletonList(new Mutation(cf.metadata().ksName, key, cf));
         }
     }
@@ -302,7 +302,7 @@ public class TriggerExecutorTest
         public Collection<Mutation> augment(ByteBuffer key, ColumnFamily update)
         {
             ColumnFamily cf = ArrayBackedSortedColumns.factory.create(makeCfMetaData("otherKs", "otherCf", null));
-            cf.addColumn(new Cell(getColumnName(update.metadata(), "c2"), bytes("trigger")));
+            cf.addColumn(new BufferCell(getColumnName(update.metadata(), "c2"), bytes("trigger")));
             return Collections.singletonList(new Mutation(cf.metadata().ksName, key, cf));
         }
     }
@@ -312,7 +312,7 @@ public class TriggerExecutorTest
         public Collection<Mutation> augment(ByteBuffer key, ColumnFamily update)
         {
             ColumnFamily cf = ArrayBackedSortedColumns.factory.create(update.metadata());
-            cf.addColumn(new Cell(getColumnName(update.metadata(), "c2"), bytes("trigger")));
+            cf.addColumn(new BufferCell(getColumnName(update.metadata(), "c2"), bytes("trigger")));
             return Collections.singletonList(new Mutation(cf.metadata().ksName, bytes("otherKey"), cf));
         }
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/test/unit/org/apache/cassandra/triggers/TriggersTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/triggers/TriggersTest.java b/test/unit/org/apache/cassandra/triggers/TriggersTest.java
index 28464e5..ee97d8a 100644
--- a/test/unit/org/apache/cassandra/triggers/TriggersTest.java
+++ b/test/unit/org/apache/cassandra/triggers/TriggersTest.java
@@ -31,7 +31,7 @@ import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.cql3.QueryProcessor;
 import org.apache.cassandra.cql3.UntypedResultSet;
 import org.apache.cassandra.db.ArrayBackedSortedColumns;
-import org.apache.cassandra.db.Cell;
+import org.apache.cassandra.db.BufferCell;
 import org.apache.cassandra.db.ColumnFamily;
 import org.apache.cassandra.db.ConsistencyLevel;
 import org.apache.cassandra.db.Mutation;
@@ -316,7 +316,7 @@ public class TriggersTest extends SchemaLoader
         public Collection<Mutation> augment(ByteBuffer key, ColumnFamily update)
         {
             ColumnFamily extraUpdate = update.cloneMeShallow(ArrayBackedSortedColumns.factory, false);
-            extraUpdate.addColumn(new Cell(update.metadata().comparator.makeCellName(bytes("v2")), bytes(999)));
+            extraUpdate.addColumn(new BufferCell(update.metadata().comparator.makeCellName(bytes("v2")), bytes(999)));
             return Collections.singletonList(new Mutation(ksName, key, extraUpdate));
         }
     }
@@ -326,7 +326,7 @@ public class TriggersTest extends SchemaLoader
         public Collection<Mutation> augment(ByteBuffer key, ColumnFamily update)
         {
             ColumnFamily extraUpdate = update.cloneMeShallow(ArrayBackedSortedColumns.factory, false);
-            extraUpdate.addColumn(new Cell(update.metadata().comparator.makeCellName(bytes("v2")), bytes(999)));
+            extraUpdate.addColumn(new BufferCell(update.metadata().comparator.makeCellName(bytes("v2")), bytes(999)));
 
             int newKey = toInt(key) + 1000;
             return Collections.singletonList(new Mutation(ksName, bytes(newKey), extraUpdate));
@@ -338,7 +338,7 @@ public class TriggersTest extends SchemaLoader
         public Collection<Mutation> augment(ByteBuffer key, ColumnFamily update)
         {
             ColumnFamily extraUpdate = ArrayBackedSortedColumns.factory.create(ksName, otherCf);
-            extraUpdate.addColumn(new Cell(extraUpdate.metadata().comparator.makeCellName(bytes("v2")), bytes(999)));
+            extraUpdate.addColumn(new BufferCell(extraUpdate.metadata().comparator.makeCellName(bytes("v2")), bytes(999)));
             return Collections.singletonList(new Mutation(ksName, key, extraUpdate));
         }
     }


[7/7] git commit: Push more of memtable data off-heap

Posted by sl...@apache.org.
Push more of memtable data off-heap

patch by benedict & xedin; reviewed by benedict, iamaleksey & xedin for CASSANDRA-6694


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

Branch: refs/heads/cassandra-2.1
Commit: 8541cca718fc324c2545831fc945247a4aeb3437
Parents: d402cf6
Author: Pavel Yaskevich <xe...@apache.org>
Authored: Mon Apr 14 14:16:08 2014 -0700
Committer: Sylvain Lebresne <sy...@datastax.com>
Committed: Tue Apr 29 13:58:20 2014 +0200

----------------------------------------------------------------------
 conf/cassandra.yaml                             |   1 +
 .../org/apache/cassandra/cache/RowCacheKey.java |   2 +-
 .../org/apache/cassandra/config/CFMetaData.java |   5 +-
 .../org/apache/cassandra/config/Config.java     |   3 +-
 .../cassandra/config/DatabaseDescriptor.java    |   7 +-
 .../org/apache/cassandra/config/Schema.java     |   2 +-
 .../apache/cassandra/cql/QueryProcessor.java    |  12 +-
 .../apache/cassandra/cql3/ColumnIdentifier.java |   2 +-
 .../apache/cassandra/cql3/UpdateParameters.java |   4 +-
 .../cql3/statements/ModificationStatement.java  |   7 +-
 .../cql3/statements/SelectStatement.java        |   8 +-
 .../org/apache/cassandra/db/AbstractCell.java   | 265 ++++++++
 .../apache/cassandra/db/AbstractNativeCell.java | 647 +++++++++++++++++++
 .../cassandra/db/ArrayBackedSortedColumns.java  |   6 +-
 .../apache/cassandra/db/AtomicBTreeColumns.java | 204 +++---
 .../org/apache/cassandra/db/BufferCell.java     | 103 +++
 .../apache/cassandra/db/BufferCounterCell.java  | 181 ++++++
 .../cassandra/db/BufferCounterUpdateCell.java   |  93 +++
 .../apache/cassandra/db/BufferDecoratedKey.java |  39 ++
 .../apache/cassandra/db/BufferDeletedCell.java  | 123 ++++
 .../apache/cassandra/db/BufferExpiringCell.java | 170 +++++
 .../org/apache/cassandra/db/CFRowAdder.java     |   6 +-
 src/java/org/apache/cassandra/db/Cell.java      | 245 +------
 .../cassandra/db/CollationController.java       |   6 +-
 .../org/apache/cassandra/db/ColumnFamily.java   |  16 +-
 .../apache/cassandra/db/ColumnFamilyStore.java  |  29 +-
 .../apache/cassandra/db/ColumnSerializer.java   |  10 +-
 .../org/apache/cassandra/db/CounterCell.java    | 217 +------
 .../apache/cassandra/db/CounterMutation.java    |   8 +-
 .../apache/cassandra/db/CounterUpdateCell.java  |  65 +-
 src/java/org/apache/cassandra/db/DataRange.java |   2 +-
 .../org/apache/cassandra/db/DecoratedKey.java   |  35 +-
 .../org/apache/cassandra/db/DefsTables.java     |   4 +-
 .../org/apache/cassandra/db/DeletedCell.java    | 100 +--
 .../org/apache/cassandra/db/ExpiringCell.java   | 159 +----
 .../cassandra/db/HintedHandOffManager.java      |   4 +-
 src/java/org/apache/cassandra/db/Keyspace.java  |   6 +-
 src/java/org/apache/cassandra/db/Memtable.java  |  47 +-
 src/java/org/apache/cassandra/db/Mutation.java  |   2 +-
 .../org/apache/cassandra/db/NativeCell.java     |  88 +++
 .../apache/cassandra/db/NativeCounterCell.java  | 190 ++++++
 .../apache/cassandra/db/NativeDecoratedKey.java |  45 ++
 .../apache/cassandra/db/NativeDeletedCell.java  | 125 ++++
 .../apache/cassandra/db/NativeExpiringCell.java | 173 +++++
 src/java/org/apache/cassandra/db/Row.java       |   4 +-
 .../apache/cassandra/db/RowIteratorFactory.java |   4 +-
 .../org/apache/cassandra/db/RowPosition.java    |  24 +-
 .../org/apache/cassandra/db/SystemKeyspace.java |   6 +-
 .../compaction/AbstractCompactionStrategy.java  |   2 +-
 .../db/compaction/CompactionController.java     |   2 +-
 .../db/compaction/CompactionManager.java        |  12 +-
 .../db/compaction/LazilyCompactedRow.java       |   4 +-
 .../db/compaction/LeveledManifest.java          |  12 +-
 .../cassandra/db/compaction/Scrubber.java       |   6 +-
 .../db/composites/AbstractCellNameType.java     |  17 +-
 .../db/composites/AbstractComposite.java        |   1 -
 .../db/composites/BoundedComposite.java         |  12 +-
 .../cassandra/db/composites/CellName.java       |   6 +-
 .../cassandra/db/composites/CellNameType.java   |   3 +-
 .../cassandra/db/composites/CellNames.java      |  16 +
 .../cassandra/db/composites/Composite.java      |   5 +-
 .../cassandra/db/composites/Composites.java     |  10 +-
 .../db/composites/CompoundComposite.java        |  12 +-
 .../db/composites/CompoundDenseCellName.java    |   5 +-
 .../composites/CompoundDenseCellNameType.java   |   3 +-
 .../db/composites/CompoundSparseCellName.java   |  15 +-
 .../composites/CompoundSparseCellNameType.java  |  15 +-
 .../db/composites/SimpleComposite.java          |  10 +-
 .../db/composites/SimpleDenseCellName.java      |   5 +-
 .../db/composites/SimpleDenseCellNameType.java  |   3 +-
 .../db/composites/SimpleSparseCellName.java     |  11 +-
 .../db/composites/SimpleSparseCellNameType.java |   5 +-
 .../SimpleSparseInternedCellName.java           |  11 +-
 .../apache/cassandra/db/filter/ColumnSlice.java |  14 +-
 .../cassandra/db/filter/ExtendedFilter.java     |   6 +-
 .../AbstractSimplePerColumnSecondaryIndex.java  |  23 +-
 .../cassandra/db/index/SecondaryIndex.java      |  10 +-
 .../db/index/SecondaryIndexManager.java         |  16 +-
 .../CompositesIndexOnClusteringKey.java         |   2 +-
 .../CompositesIndexOnCollectionKey.java         |   2 +-
 .../CompositesIndexOnCollectionValue.java       |   2 +-
 .../composites/CompositesIndexOnRegular.java    |   2 +-
 .../db/index/composites/CompositesSearcher.java |   8 +-
 .../cassandra/db/index/keys/KeysSearcher.java   |  12 +-
 .../db/marshal/LocalByPartionerType.java        |   2 +-
 .../apache/cassandra/dht/AbstractBounds.java    |   3 +-
 .../dht/AbstractByteOrderedPartitioner.java     |   3 +-
 .../apache/cassandra/dht/LocalPartitioner.java  |   3 +-
 .../cassandra/dht/Murmur3Partitioner.java       |   3 +-
 .../dht/OrderPreservingPartitioner.java         |   3 +-
 .../apache/cassandra/dht/RandomPartitioner.java |   3 +-
 src/java/org/apache/cassandra/dht/Token.java    |   7 +-
 .../hadoop/ColumnFamilyRecordReader.java        |   5 +-
 .../apache/cassandra/hadoop/pig/CqlStorage.java |   3 +-
 .../io/sstable/AbstractSSTableSimpleWriter.java |  10 +-
 .../cassandra/io/sstable/CQLSSTableWriter.java  |   2 +-
 .../io/sstable/IndexSummaryBuilder.java         |   9 +-
 .../apache/cassandra/io/sstable/SSTable.java    |   5 +-
 .../cassandra/io/sstable/SSTableReader.java     |  18 +-
 .../cassandra/io/sstable/SSTableScanner.java    |   4 +-
 .../io/sstable/SSTableSimpleUnsortedWriter.java |   4 +-
 .../cassandra/io/sstable/SSTableWriter.java     |  12 +-
 .../repair/RepairMessageVerbHandler.java        |   2 +-
 .../org/apache/cassandra/repair/Validator.java  |  10 +-
 .../cassandra/service/ActiveRepairService.java  |   2 +-
 .../cassandra/service/RowDataResolver.java      |   2 +-
 .../cassandra/service/StorageService.java       |   2 +-
 .../service/pager/RangeNamesQueryPager.java     |   2 +-
 .../service/pager/RangeSliceQueryPager.java     |   2 +-
 .../cassandra/thrift/CassandraServer.java       |  12 +-
 .../cassandra/thrift/ThriftValidation.java      |   6 +-
 .../apache/cassandra/tools/SSTableExport.java   |   6 +-
 .../apache/cassandra/tools/SSTableImport.java   |   4 +-
 .../org/apache/cassandra/tracing/Tracing.java   |   6 +-
 .../org/apache/cassandra/utils/FBUtilities.java |  35 +-
 .../apache/cassandra/utils/PureJavaCrc32.java   |  33 +-
 .../utils/memory/ContextAllocator.java          |  25 +-
 .../apache/cassandra/utils/memory/HeapPool.java |  58 +-
 .../cassandra/utils/memory/MemoryUtil.java      | 305 +++++++++
 .../utils/memory/MemtableAllocator.java         | 245 +++++++
 .../utils/memory/MemtableBufferAllocator.java   |  70 ++
 .../utils/memory/MemtableCleanerThread.java     |  73 +++
 .../cassandra/utils/memory/MemtablePool.java    | 199 ++++++
 .../cassandra/utils/memory/NativeAllocator.java | 258 ++++++++
 .../cassandra/utils/memory/NativePool.java      |  39 ++
 .../org/apache/cassandra/utils/memory/Pool.java | 199 ------
 .../cassandra/utils/memory/PoolAllocator.java   | 225 -------
 .../utils/memory/PoolCleanerThread.java         |  73 ---
 .../cassandra/utils/memory/SlabAllocator.java   |  13 +-
 .../apache/cassandra/utils/memory/SlabPool.java |   7 +-
 test/conf/cassandra.yaml                        |   2 +-
 .../cassandra/db/LongFlushMemtableTest.java     |   2 +-
 .../apache/cassandra/db/LongKeyspaceTest.java   |   2 +-
 .../db/compaction/LongCompactionsTest.java      |   2 +-
 .../LongLeveledCompactionStrategyTest.java      |   2 +-
 test/unit/org/apache/cassandra/Util.java        |   8 +-
 .../org/apache/cassandra/config/DefsTest.java   |  17 +-
 .../db/ArrayBackedSortedColumnsTest.java        |  20 +-
 .../cassandra/db/CollationControllerTest.java   |  12 +-
 .../cassandra/db/ColumnFamilyStoreTest.java     |  86 +--
 .../apache/cassandra/db/CounterCacheTest.java   |   4 +-
 .../apache/cassandra/db/CounterCellTest.java    |  80 +--
 .../org/apache/cassandra/db/KeyCacheTest.java   |   4 +-
 .../apache/cassandra/db/KeyCollisionTest.java   |  10 +-
 .../org/apache/cassandra/db/KeyspaceTest.java   |  38 +-
 .../org/apache/cassandra/db/MultitableTest.java |   4 +-
 .../org/apache/cassandra/db/NativeCellTest.java | 171 +++++
 .../apache/cassandra/db/RangeTombstoneTest.java |   9 +-
 .../apache/cassandra/db/ReadMessageTest.java    |  10 +-
 .../cassandra/db/RecoveryManager3Test.java      |   4 +-
 .../cassandra/db/RecoveryManagerTest.java       |  10 +-
 .../org/apache/cassandra/db/RemoveCellTest.java |   8 +-
 .../cassandra/db/RemoveColumnFamilyTest.java    |   4 +-
 .../db/RemoveColumnFamilyWithFlush1Test.java    |   4 +-
 .../db/RemoveColumnFamilyWithFlush2Test.java    |   4 +-
 .../apache/cassandra/db/RemoveSubCellTest.java  |  10 +-
 .../org/apache/cassandra/db/RowCacheTest.java   |   2 +-
 .../apache/cassandra/db/RowIterationTest.java   |   8 +-
 test/unit/org/apache/cassandra/db/RowTest.java  |   2 +-
 .../unit/org/apache/cassandra/db/ScrubTest.java |   4 +-
 .../db/SecondaryIndexCellSizeTest.java          |  15 +-
 .../apache/cassandra/db/SerializationsTest.java |  31 +-
 .../org/apache/cassandra/db/TimeSortTest.java   |   8 +-
 .../db/compaction/AntiCompactionTest.java       |   6 +-
 .../compaction/BlacklistingCompactionsTest.java |   2 +-
 .../db/compaction/CompactionsPurgeTest.java     |  34 +-
 .../db/compaction/CompactionsTest.java          |  18 +-
 .../LeveledCompactionStrategyTest.java          |   8 +-
 .../db/compaction/OneCompactionTest.java        |   2 +-
 .../SizeTieredCompactionStrategyTest.java       |   4 +-
 .../cassandra/db/compaction/TTLExpiryTest.java  |  16 +-
 .../db/context/CounterContextTest.java          |  13 -
 .../db/index/PerRowSecondaryIndexTest.java      |   8 +-
 .../io/sstable/IndexSummaryManagerTest.java     |   6 +-
 .../cassandra/io/sstable/IndexSummaryTest.java  |  10 +-
 .../cassandra/io/sstable/SSTableLoaderTest.java |   2 +-
 .../io/sstable/SSTableMetadataTest.java         |  30 +-
 .../cassandra/io/sstable/SSTableReaderTest.java |  13 +-
 .../io/sstable/SSTableScannerTest.java          |   8 +-
 .../cassandra/io/sstable/SSTableUtils.java      |   2 +-
 .../apache/cassandra/repair/ValidatorTest.java  |   3 +-
 .../cassandra/service/QueryPagerTest.java       |   4 +-
 .../service/pager/AbstractQueryPagerTest.java   |   2 +-
 .../streaming/StreamingTransferTest.java        |  20 +-
 .../cassandra/tools/SSTableExportTest.java      |   6 +-
 .../cassandra/tools/SSTableImportTest.java      |   6 +-
 .../cassandra/triggers/TriggerExecutorTest.java |  14 +-
 .../apache/cassandra/triggers/TriggersTest.java |   8 +-
 188 files changed, 4529 insertions(+), 2129 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/conf/cassandra.yaml
----------------------------------------------------------------------
diff --git a/conf/cassandra.yaml b/conf/cassandra.yaml
index 2176bf9..6e47918 100644
--- a/conf/cassandra.yaml
+++ b/conf/cassandra.yaml
@@ -283,6 +283,7 @@ memtable_cleanup_threshold: 0.4
 # Options are:
 #   heap_buffers:    on heap nio buffers
 #   offheap_buffers: off heap (direct) nio buffers
+#   offheap_objects: native memory, eliminating nio buffer heap overhead
 memtable_allocation_type: heap_buffers
 
 # Total space to use for commitlogs.  Since commitlog segments are

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/cache/RowCacheKey.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cache/RowCacheKey.java b/src/java/org/apache/cassandra/cache/RowCacheKey.java
index bbd8591..aebb129 100644
--- a/src/java/org/apache/cassandra/cache/RowCacheKey.java
+++ b/src/java/org/apache/cassandra/cache/RowCacheKey.java
@@ -37,7 +37,7 @@ public class RowCacheKey implements CacheKey, Comparable<RowCacheKey>
 
     public RowCacheKey(UUID cfId, DecoratedKey key)
     {
-        this(cfId, key.key);
+        this(cfId, key.getKey());
     }
 
     public RowCacheKey(UUID cfId, ByteBuffer key)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/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 b8c1fae..b4b3fbe 100644
--- a/src/java/org/apache/cassandra/config/CFMetaData.java
+++ b/src/java/org/apache/cassandra/config/CFMetaData.java
@@ -55,6 +55,7 @@ import org.apache.cassandra.cql3.QueryProcessor;
 import org.apache.cassandra.cql3.UntypedResultSet;
 import org.apache.cassandra.cql3.statements.CFStatement;
 import org.apache.cassandra.cql3.statements.CreateTableStatement;
+import org.apache.cassandra.db.AbstractCell;
 import org.apache.cassandra.db.AtomDeserializer;
 import org.apache.cassandra.db.CFRowAdder;
 import org.apache.cassandra.db.Cell;
@@ -1328,7 +1329,7 @@ public final class CFMetaData
      */
     public ColumnDefinition getColumnDefinition(CellName cellName)
     {
-        ColumnIdentifier id = cellName.cql3ColumnName();
+        ColumnIdentifier id = cellName.cql3ColumnName(this);
         ColumnDefinition def = id == null
                              ? getColumnDefinition(cellName.toByteBuffer())  // Means a dense layout, try the full column name
                              : getColumnDefinition(id);
@@ -1409,7 +1410,7 @@ public final class CFMetaData
 
     public Iterator<OnDiskAtom> getOnDiskIterator(DataInput in, ColumnSerializer.Flag flag, int expireBefore, Descriptor.Version version)
     {
-        return Cell.onDiskIterator(in, flag, expireBefore, version, comparator);
+        return AbstractCell.onDiskIterator(in, flag, expireBefore, version, comparator);
     }
 
     public AtomDeserializer getOnDiskDeserializer(DataInput in, Descriptor.Version version)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/config/Config.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/Config.java b/src/java/org/apache/cassandra/config/Config.java
index 7ab7a8c..97ceb7f 100644
--- a/src/java/org/apache/cassandra/config/Config.java
+++ b/src/java/org/apache/cassandra/config/Config.java
@@ -290,7 +290,8 @@ public class Config
     {
         unslabbed_heap_buffers,
         heap_buffers,
-        offheap_buffers
+        offheap_buffers,
+        offheap_objects
     }
 
     public static enum DiskFailurePolicy

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
index 4b0043c..cdcceb1 100644
--- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
+++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
@@ -66,7 +66,8 @@ import org.apache.cassandra.service.CacheService;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.memory.HeapPool;
-import org.apache.cassandra.utils.memory.Pool;
+import org.apache.cassandra.utils.memory.NativePool;
+import org.apache.cassandra.utils.memory.MemtablePool;
 import org.apache.cassandra.utils.memory.SlabPool;
 
 public class DatabaseDescriptor
@@ -1476,7 +1477,7 @@ public class DatabaseDescriptor
         return conf.inter_dc_tcp_nodelay;
     }
 
-    public static Pool getMemtableAllocatorPool()
+    public static MemtablePool getMemtableAllocatorPool()
     {
         long heapLimit = ((long) conf.memtable_heap_space_in_mb) << 20;
         long offHeapLimit = ((long) conf.memtable_offheap_space_in_mb) << 20;
@@ -1493,6 +1494,8 @@ public class DatabaseDescriptor
                     System.exit(-1);
                 }
                 return new SlabPool(heapLimit, offHeapLimit, conf.memtable_cleanup_threshold, new ColumnFamilyStore.FlushLargestColumnFamily());
+            case offheap_objects:
+                return new NativePool(heapLimit, offHeapLimit, conf.memtable_cleanup_threshold, new ColumnFamilyStore.FlushLargestColumnFamily());
             default:
                 throw new AssertionError();
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/config/Schema.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/Schema.java b/src/java/org/apache/cassandra/config/Schema.java
index b1e0f2f..106ad9b 100644
--- a/src/java/org/apache/cassandra/config/Schema.java
+++ b/src/java/org/apache/cassandra/config/Schema.java
@@ -416,7 +416,7 @@ public class Schema
     {
         try
         {
-            return systemKeyspaceNames.contains(ByteBufferUtil.string(row.key.key));
+            return systemKeyspaceNames.contains(ByteBufferUtil.string(row.key.getKey()));
         }
         catch (CharacterCodingException e)
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/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 baf89b2..3b35555 100644
--- a/src/java/org/apache/cassandra/cql/QueryProcessor.java
+++ b/src/java/org/apache/cassandra/cql/QueryProcessor.java
@@ -171,7 +171,7 @@ public class QueryProcessor
                                     ? select.getKeyFinish().getByteBuffer(keyType,variables)
                                     : null;
 
-        RowPosition startKey = RowPosition.forKey(startKeyBytes, p), finishKey = RowPosition.forKey(finishKeyBytes, p);
+        RowPosition startKey = RowPosition.ForKey.get(startKeyBytes, p), finishKey = RowPosition.ForKey.get(finishKeyBytes, p);
         if (startKey.compareTo(finishKey) > 0 && !finishKey.isMinimum(p))
         {
             if (p instanceof RandomPartitioner)
@@ -213,7 +213,7 @@ public class QueryProcessor
         // if start key was set and relation was "greater than"
         if (select.getKeyStart() != null && !select.includeStartKey() && !rows.isEmpty())
         {
-            if (rows.get(0).key.key.equals(startKeyBytes))
+            if (rows.get(0).key.getKey().equals(startKeyBytes))
                 rows.remove(0);
         }
 
@@ -221,7 +221,7 @@ public class QueryProcessor
         if (select.getKeyFinish() != null && !select.includeFinishKey() && !rows.isEmpty())
         {
             int lastIndex = rows.size() - 1;
-            if (rows.get(lastIndex).key.key.equals(finishKeyBytes))
+            if (rows.get(lastIndex).key.getKey().equals(finishKeyBytes))
                 rows.remove(lastIndex);
         }
 
@@ -459,7 +459,7 @@ public class QueryProcessor
                         {
                             // prepend key
                             ByteBuffer keyName = ByteBufferUtil.bytes(metadata.getCQL2KeyName());
-                            thriftColumns.add(new Column(keyName).setValue(row.key.key).setTimestamp(-1));
+                            thriftColumns.add(new Column(keyName).setValue(row.key.getKey()).setTimestamp(-1));
                             result.schema.name_types.put(keyName, TypeParser.getShortName(AsciiType.instance));
                             result.schema.value_types.put(keyName, TypeParser.getShortName(metadata.getKeyValidator()));
                         }
@@ -491,7 +491,7 @@ public class QueryProcessor
                             {
                                 // preserve case of key as it was requested
                                 ByteBuffer requestedKey = ByteBufferUtil.bytes(term.getText());
-                                thriftColumns.add(new Column(requestedKey).setValue(row.key.key).setTimestamp(-1));
+                                thriftColumns.add(new Column(requestedKey).setValue(row.key.getKey()).setTimestamp(-1));
                                 result.schema.name_types.put(requestedKey, TypeParser.getShortName(AsciiType.instance));
                                 result.schema.value_types.put(requestedKey, TypeParser.getShortName(metadata.getKeyValidator()));
                                 continue;
@@ -524,7 +524,7 @@ public class QueryProcessor
 
                     // Create a new row, add the columns to it, and then add it to the list of rows
                     CqlRow cqlRow = new CqlRow();
-                    cqlRow.key = row.key.key;
+                    cqlRow.key = row.key.getKey();
                     cqlRow.columns = thriftColumns;
                     if (select.isColumnsReversed())
                         Collections.reverse(cqlRow.columns);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/cql3/ColumnIdentifier.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/ColumnIdentifier.java b/src/java/org/apache/cassandra/cql3/ColumnIdentifier.java
index 3bf3acc..1b232cf 100644
--- a/src/java/org/apache/cassandra/cql3/ColumnIdentifier.java
+++ b/src/java/org/apache/cassandra/cql3/ColumnIdentifier.java
@@ -50,7 +50,7 @@ public class ColumnIdentifier implements Selectable, IMeasurableMemory
         this.text = type.getString(bytes);
     }
 
-    private ColumnIdentifier(ByteBuffer bytes, String text)
+    public ColumnIdentifier(ByteBuffer bytes, String text)
     {
         this.bytes = bytes;
         this.text = text;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/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 87ce22e..fad8fae 100644
--- a/src/java/org/apache/cassandra/cql3/UpdateParameters.java
+++ b/src/java/org/apache/cassandra/cql3/UpdateParameters.java
@@ -55,13 +55,13 @@ public class UpdateParameters
     public Cell makeColumn(CellName name, ByteBuffer value) throws InvalidRequestException
     {
         QueryProcessor.validateCellName(name, metadata.comparator);
-        return Cell.create(name, value, timestamp, ttl, metadata);
+        return AbstractCell.create(name, value, timestamp, ttl, metadata);
     }
 
     public Cell makeTombstone(CellName name) throws InvalidRequestException
     {
         QueryProcessor.validateCellName(name, metadata.comparator);
-        return new DeletedCell(name, localDeletionTime, timestamp);
+        return new BufferDeletedCell(name, localDeletionTime, timestamp);
     }
 
     public RangeTombstone makeRangeTombstone(ColumnSlice slice) throws InvalidRequestException

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
index acc4802..4741b9a 100644
--- a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
@@ -32,12 +32,9 @@ import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.composites.CBuilder;
 import org.apache.cassandra.db.composites.Composite;
 import org.apache.cassandra.db.filter.ColumnSlice;
-import org.apache.cassandra.db.filter.IDiskAtomFilter;
 import org.apache.cassandra.db.filter.SliceQueryFilter;
-import org.apache.cassandra.db.marshal.ListType;
 import org.apache.cassandra.db.marshal.BooleanType;
 import org.apache.cassandra.exceptions.*;
-import org.apache.cassandra.service.CASConditions;
 import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.service.QueryState;
 import org.apache.cassandra.service.StorageProxy;
@@ -449,10 +446,10 @@ public abstract class ModificationStatement implements CQLStatement, MeasurableF
             if (row.cf == null || row.cf.isEmpty())
                 continue;
 
-            Iterator<CQL3Row> iter = cfm.comparator.CQL3RowBuilder(now).group(row.cf.getSortedColumns().iterator());
+            Iterator<CQL3Row> iter = cfm.comparator.CQL3RowBuilder(cfm, now).group(row.cf.getSortedColumns().iterator());
             if (iter.hasNext())
             {
-                map.put(row.key.key, iter.next());
+                map.put(row.key.getKey(), iter.next());
                 // We can only update one CQ3Row per partition key at a time (we don't allow IN for clustering key)
                 assert !iter.hasNext();
             }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/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 f28d570..d79bd5b 100644
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@ -400,8 +400,8 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
             ByteBuffer startKeyBytes = getKeyBound(Bound.START, variables);
             ByteBuffer finishKeyBytes = getKeyBound(Bound.END, variables);
 
-            RowPosition startKey = RowPosition.forKey(startKeyBytes, p);
-            RowPosition finishKey = RowPosition.forKey(finishKeyBytes, p);
+            RowPosition startKey = RowPosition.ForKey.get(startKeyBytes, p);
+            RowPosition finishKey = RowPosition.ForKey.get(finishKeyBytes, p);
 
             if (startKey.compareTo(finishKey) > 0 && !finishKey.isMinimum(p))
                 return null;
@@ -1007,7 +1007,7 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
             if (row.cf == null)
                 continue;
 
-            processColumnFamily(row.key.key, row.cf, variables, now, result);
+            processColumnFamily(row.key.getKey(), row.cf, variables, now, result);
         }
 
         ResultSet cqlRows = result.build();
@@ -1042,7 +1042,7 @@ public class SelectStatement implements CQLStatement, MeasurableForPreparedCache
         if (sliceRestriction != null)
             cells = applySliceRestriction(cells, variables);
 
-        CQL3Row.RowIterator iter = cfm.comparator.CQL3RowBuilder(now).group(cells);
+        CQL3Row.RowIterator iter = cfm.comparator.CQL3RowBuilder(cfm, now).group(cells);
 
         // If there is static columns but there is no non-static row, then provided the select was a full
         // partition selection (i.e. not a 2ndary index search and there was no condition on clustering columns)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/db/AbstractCell.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/AbstractCell.java b/src/java/org/apache/cassandra/db/AbstractCell.java
new file mode 100644
index 0000000..1075278
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/AbstractCell.java
@@ -0,0 +1,265 @@
+/**
+ * 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.context.CounterContext;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.io.sstable.Descriptor;
+import org.apache.cassandra.serializers.MarshalException;
+import org.apache.cassandra.utils.FBUtilities;
+
+public abstract class AbstractCell implements Cell
+{
+    public static Iterator<OnDiskAtom> onDiskIterator(final DataInput in,
+                                                      final ColumnSerializer.Flag flag,
+                                                      final int expireBefore,
+                                                      final Descriptor.Version version,
+                                                      final CellNameType type)
+    {
+        return new AbstractIterator<OnDiskAtom>()
+        {
+            protected OnDiskAtom computeNext()
+            {
+                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;
+            }
+        };
+    }
+
+    @Override
+    public boolean isMarkedForDelete(long now)
+    {
+        return false;
+    }
+
+    @Override
+    public boolean isLive(long now)
+    {
+        return !isMarkedForDelete(now);
+    }
+
+    // Don't call unless the column is actually marked for delete.
+    @Override
+    public long getMarkedForDeleteAt()
+    {
+        return Long.MAX_VALUE;
+    }
+
+    @Override
+    public int cellDataSize()
+    {
+        return name().dataSize() + value().remaining() + TypeSizes.NATIVE.sizeof(timestamp());
+    }
+
+    @Override
+    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;
+    }
+
+    @Override
+    public int serializationFlags()
+    {
+        return 0;
+    }
+
+    @Override
+    public Cell diff(Cell cell)
+    {
+        if (timestamp() < cell.timestamp())
+            return cell;
+        return null;
+    }
+
+    @Override
+    public void updateDigest(MessageDigest digest)
+    {
+        digest.update(name().toByteBuffer().duplicate());
+        digest.update(value().duplicate());
+
+        FBUtilities.updateWithLong(digest, timestamp());
+        FBUtilities.updateWithByte(digest, serializationFlags());
+    }
+
+    @Override
+    public int getLocalDeletionTime()
+    {
+        return Integer.MAX_VALUE;
+    }
+
+    @Override
+    public Cell reconcile(Cell cell)
+    {
+        // 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)
+    {
+        return this == o || (o instanceof Cell && equals((Cell) o));
+    }
+
+    public boolean equals(Cell cell)
+    {
+        return timestamp() == cell.timestamp() && name().equals(cell.name()) && value().equals(cell.value());
+    }
+
+    public int hashCode()
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public String getString(CellNameType comparator)
+    {
+        return String.format("%s:%b:%d@%d",
+                comparator.getString(name()),
+                isMarkedForDelete(System.currentTimeMillis()),
+                value().remaining(),
+                timestamp());
+    }
+
+    @Override
+    public void validateName(CFMetaData metadata) throws MarshalException
+    {
+        metadata.comparator.validate(name());
+    }
+
+    @Override
+    public void validateFields(CFMetaData metadata) throws MarshalException
+    {
+        validateName(metadata);
+
+        AbstractType<?> valueValidator = metadata.getValueValidator(name());
+        if (valueValidator != null)
+            valueValidator.validate(value());
+    }
+
+    public static Cell create(CellName name, ByteBuffer value, long timestamp, int ttl, CFMetaData metadata)
+    {
+        if (ttl <= 0)
+            ttl = metadata.getDefaultTimeToLive();
+
+        return ttl > 0
+                ? new BufferExpiringCell(name, value, timestamp, ttl)
+                : new BufferCell(name, value, timestamp);
+    }
+
+    public static Cell diff(CounterCell a, Cell b)
+    {
+        if (a.timestamp() < b.timestamp())
+            return b;
+
+        // 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 b instanceof CounterCell : "Wrong class type: " + b.getClass();
+
+        if (a.timestampOfLastDelete() < ((CounterCell) b).timestampOfLastDelete())
+            return b;
+
+        CounterContext.Relationship rel = CounterCell.contextManager.diff(b.value(), a.value());
+        return (rel == CounterContext.Relationship.GREATER_THAN || rel == CounterContext.Relationship.DISJOINT) ? b : null;
+    }
+
+    /** This is temporary until we start creating Cells of the different type (buffer vs. native) */
+    public static Cell reconcile(CounterCell a, Cell b)
+    {
+        assert (b instanceof CounterCell) || (b instanceof DeletedCell) : "Wrong class type: " + b.getClass();
+
+        // live + tombstone: track last tombstone
+        if (b.isMarkedForDelete(Long.MIN_VALUE)) // cannot be an expired cell, so the current time is irrelevant
+        {
+            // live < tombstone
+            if (a.timestamp() < b.timestamp())
+                return b;
+
+            // live last delete >= tombstone
+            if (a.timestampOfLastDelete() >= b.timestamp())
+                return a;
+
+            // live last delete < tombstone
+            return new BufferCounterCell(a.name(), a.value(), a.timestamp(), b.timestamp());
+        }
+
+        assert b instanceof CounterCell : "Wrong class type: " + b.getClass();
+
+        // live < live last delete
+        if (a.timestamp() < ((CounterCell) b).timestampOfLastDelete())
+            return b;
+
+        // live last delete > live
+        if (a.timestampOfLastDelete() > b.timestamp())
+            return a;
+
+        // live + live. return one of the cells if its context is a superset of the other's, or merge them otherwise
+        ByteBuffer context = CounterCell.contextManager.merge(a.value(), b.value());
+        if (context == a.value() && a.timestamp() >= b.timestamp() && a.timestampOfLastDelete() >= ((CounterCell) b).timestampOfLastDelete())
+            return a;
+        else if (context == b.value() && b.timestamp() >= a.timestamp() && ((CounterCell) b).timestampOfLastDelete() >= a.timestampOfLastDelete())
+            return b;
+        else // merge clocks and timsestamps.
+            return new BufferCounterCell(a.name(),
+                                         context,
+                                         Math.max(a.timestamp(), b.timestamp()),
+                                         Math.max(a.timestampOfLastDelete(), ((CounterCell) b).timestampOfLastDelete()));
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/db/AbstractNativeCell.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/AbstractNativeCell.java b/src/java/org/apache/cassandra/db/AbstractNativeCell.java
new file mode 100644
index 0000000..d21171f
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/AbstractNativeCell.java
@@ -0,0 +1,647 @@
+/*
+ * 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.security.MessageDigest;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.db.composites.*;
+import org.apache.cassandra.db.filter.ColumnSlice;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.CompositeType;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.concurrent.OpOrder;
+import org.apache.cassandra.utils.memory.*;
+
+
+/**
+ * Packs a CellName AND a Cell into one off-heap representation.
+ * Layout is:
+ *
+ * Note we store the ColumnIdentifier in full as bytes. This seems an okay tradeoff for now, as we just
+ * look it back up again when we need to, and in the near future we hope to switch to ints, longs or
+ * UUIDs representing column identifiers on disk, at which point we can switch that here as well.
+ *
+ * [timestamp][value offset][name size]][name extra][name offset deltas][cell names][value][Descendants]
+ * [   8b    ][     4b     ][    2b   ][     1b    ][     each 2b      ][ arb < 64k][ arb ][ arbitrary ]
+ *
+ * descendants: any overriding classes will put their state here
+ * name offsets are deltas from their base offset, and don't include the first offset, or the end position of the final entry,
+ * i.e. there will be size - 1 entries, and each is a delta that is added to the offset of the position of the first name
+ * (which is always CELL_NAME_OFFSETS_OFFSET + (2 * (size - 1))). The length of the final name fills up any remaining
+ * space upto the value offset
+ * name extra:  lowest 2 bits indicate the clustering size delta (i.e. how many name items are NOT part of the clustering key)
+ *              the next 2 bits indicate the CellNameType
+ *              the next bit indicates if the column is a static or clustered/dynamic column
+ */
+public abstract class AbstractNativeCell extends AbstractCell implements CellName
+{
+    static final int TIMESTAMP_OFFSET = 4;
+    private static final int VALUE_OFFSET_OFFSET = 12;
+    private static final int CELL_NAME_SIZE_OFFSET = 16;
+    private static final int CELL_NAME_EXTRA_OFFSET = 18;
+    private static final int CELL_NAME_OFFSETS_OFFSET = 19;
+    private static final int CELL_NAME_SIZE_DELTA_MASK = 3;
+    private static final int CELL_NAME_TYPE_SHIFT = 2;
+    private static final int CELL_NAME_TYPE_MASK = 7;
+
+    private static enum NameType
+    {
+        COMPOUND_DENSE(0 << 2), COMPOUND_SPARSE(1 << 2), COMPOUND_SPARSE_STATIC(2 << 2), SIMPLE_DENSE(3 << 2), SIMPLE_SPARSE(4 << 2);
+        static final NameType[] TYPES = NameType.values();
+        final int bits;
+
+        NameType(int bits)
+        {
+            this.bits = bits;
+        }
+
+        static NameType typeOf(CellName name)
+        {
+            if (name instanceof CompoundDenseCellName)
+            {
+                assert !name.isStatic();
+                return COMPOUND_DENSE;
+            }
+
+            if (name instanceof CompoundSparseCellName)
+                return name.isStatic() ? COMPOUND_SPARSE_STATIC : COMPOUND_SPARSE;
+
+            if (name instanceof SimpleDenseCellName)
+            {
+                assert !name.isStatic();
+                return SIMPLE_DENSE;
+            }
+
+            if (name instanceof SimpleSparseCellName)
+            {
+                assert !name.isStatic();
+                return SIMPLE_SPARSE;
+            }
+
+            if (name instanceof NativeCell)
+                return ((NativeCell) name).nametype();
+
+            throw new AssertionError();
+        }
+    }
+
+    private final long peer; // peer is assigned by peer updater in setPeer method
+
+    AbstractNativeCell()
+    {
+        peer = -1;
+    }
+
+    public AbstractNativeCell(NativeAllocator allocator, OpOrder.Group writeOp, Cell copyOf)
+    {
+        int size = sizeOf(copyOf);
+        peer = allocator.allocate(size, writeOp);
+
+        MemoryUtil.setInt(peer, size);
+        construct(copyOf);
+    }
+
+    protected int sizeOf(Cell cell)
+    {
+        int size = CELL_NAME_OFFSETS_OFFSET + Math.max(0, cell.name().size() - 1) * 2 + cell.value().remaining();
+        CellName name = cell.name();
+        for (int i = 0; i < name.size(); i++)
+            size += name.get(i).remaining();
+        return size;
+    }
+
+    protected void construct(Cell from)
+    {
+        setLong(TIMESTAMP_OFFSET, from.timestamp());
+        CellName name = from.name();
+        int nameSize = name.size();
+        int offset = CELL_NAME_SIZE_OFFSET;
+        setShort(offset, (short) nameSize);
+        assert nameSize - name.clusteringSize() <= 2;
+        byte cellNameExtraBits = (byte) ((nameSize - name.clusteringSize()) | NameType.typeOf(name).bits);
+        setByte(offset += 2, cellNameExtraBits);
+        offset += 1;
+        short cellNameDelta = 0;
+        for (int i = 1; i < nameSize; i++)
+        {
+            cellNameDelta += name.get(i - 1).remaining();
+            setShort(offset, cellNameDelta);
+            offset += 2;
+        }
+        for (int i = 0; i < nameSize; i++)
+        {
+            ByteBuffer bb = name.get(i);
+            setBytes(offset, bb);
+            offset += bb.remaining();
+        }
+        setInt(VALUE_OFFSET_OFFSET, offset);
+        setBytes(offset, from.value());
+    }
+
+    // the offset at which to read the short that gives the names
+    private int nameDeltaOffset(int i)
+    {
+        return CELL_NAME_OFFSETS_OFFSET + ((i - 1) * 2);
+    }
+
+    int valueStartOffset()
+    {
+        return getInt(VALUE_OFFSET_OFFSET);
+    }
+
+    private int valueEndOffset()
+    {
+        return (int) (internalSize() - postfixSize());
+    }
+
+    protected int postfixSize()
+    {
+        return 0;
+    }
+
+    @Override
+    public ByteBuffer value()
+    {
+        long offset = valueStartOffset();
+        return getByteBuffer(offset, (int) (internalSize() - (postfixSize() + offset)));
+    }
+
+    private int clusteringSizeDelta()
+    {
+        return getByte(CELL_NAME_EXTRA_OFFSET) & CELL_NAME_SIZE_DELTA_MASK;
+    }
+
+    public boolean isStatic()
+    {
+        return nametype() == NameType.COMPOUND_SPARSE_STATIC;
+    }
+
+    NameType nametype()
+    {
+        return NameType.TYPES[(((int) this.getByte(CELL_NAME_EXTRA_OFFSET)) >> CELL_NAME_TYPE_SHIFT) & CELL_NAME_TYPE_MASK];
+    }
+
+    public long minTimestamp()
+    {
+        return timestamp();
+    }
+
+    public long maxTimestamp()
+    {
+        return timestamp();
+    }
+
+    public int clusteringSize()
+    {
+        return size() - clusteringSizeDelta();
+    }
+
+    @Override
+    public ColumnIdentifier cql3ColumnName(CFMetaData metadata)
+    {
+        switch (nametype())
+        {
+            case SIMPLE_SPARSE:
+                return getIdentifier(metadata, get(clusteringSize()));
+            case COMPOUND_SPARSE_STATIC:
+            case COMPOUND_SPARSE:
+                ByteBuffer buffer = get(clusteringSize());
+                if (buffer.remaining() == 0)
+                    return CompoundSparseCellNameType.rowMarkerId;
+
+                return getIdentifier(metadata, buffer);
+            case SIMPLE_DENSE:
+            case COMPOUND_DENSE:
+                return null;
+            default:
+                throw new AssertionError();
+        }
+    }
+
+    public ByteBuffer collectionElement()
+    {
+        return isCollectionCell() ? get(size() - 1) : null;
+    }
+
+    // we always have a collection element if our clustering size is 2 less than our total size,
+    // and we never have one otherwiss
+    public boolean isCollectionCell()
+    {
+        return clusteringSizeDelta() == 2;
+    }
+
+    public boolean isSameCQL3RowAs(CellNameType type, CellName other)
+    {
+        switch (nametype())
+        {
+            case SIMPLE_DENSE:
+            case COMPOUND_DENSE:
+                return type.compare(this, other) == 0;
+            case COMPOUND_SPARSE_STATIC:
+            case COMPOUND_SPARSE:
+                int clusteringSize = clusteringSize();
+                if (clusteringSize != other.clusteringSize() || other.isStatic() != isStatic())
+                    return false;
+                for (int i = 0; i < clusteringSize; i++)
+                    if (type.subtype(i).compare(get(i), other.get(i)) != 0)
+                        return false;
+                return true;
+            case SIMPLE_SPARSE:
+                return true;
+            default:
+                throw new AssertionError();
+        }
+    }
+
+    public int size()
+    {
+        return getShort(CELL_NAME_SIZE_OFFSET);
+    }
+
+    public boolean isEmpty()
+    {
+        return size() == 0;
+    }
+
+    public ByteBuffer get(int i)
+    {
+        // remember to take dense/sparse into account, and only return EOC when not dense
+        int size = size();
+        assert i >= 0 && i < size();
+        int cellNamesOffset = nameDeltaOffset(size);
+        int startDelta = i == 0 ? 0 : getShort(nameDeltaOffset(i));
+        int endDelta = i < size - 1 ? getShort(nameDeltaOffset(i + 1)) : valueStartOffset() - cellNamesOffset;
+        return getByteBuffer(cellNamesOffset + startDelta, endDelta - startDelta);
+    }
+
+    private static final ThreadLocal<byte[]> BUFFER = new ThreadLocal<byte[]>()
+    {
+        protected byte[] initialValue()
+        {
+            return new byte[256];
+        }
+    };
+
+    protected void writeComponentTo(MessageDigest digest, int i, boolean includeSize)
+    {
+        // remember to take dense/sparse into account, and only return EOC when not dense
+        int size = size();
+        assert i >= 0 && i < size();
+        int cellNamesOffset = nameDeltaOffset(size);
+        int startDelta = i == 0 ? 0 : getShort(nameDeltaOffset(i));
+        int endDelta = i < size - 1 ? getShort(nameDeltaOffset(i + 1)) : valueStartOffset() - cellNamesOffset;
+
+        int componentStart = cellNamesOffset + startDelta;
+        int count = endDelta - startDelta;
+
+        if (includeSize)
+            FBUtilities.updateWithShort(digest, count);
+
+        writeMemoryTo(digest, componentStart, count);
+    }
+
+    protected void writeMemoryTo(MessageDigest digest, int from, int count)
+    {
+        // only batch if we have more than 16 bytes remaining to transfer, otherwise fall-back to single-byte updates
+        int i = 0, batchEnd = count - 16;
+        if (i < batchEnd)
+        {
+            byte[] buffer = BUFFER.get();
+            while (i < batchEnd)
+            {
+                int transfer = Math.min(count - i, 256);
+                getBytes(from + i, buffer, 0, transfer);
+                digest.update(buffer, 0, transfer);
+                i += transfer;
+            }
+        }
+        while (i < count)
+            digest.update(getByte(from + i++));
+    }
+
+    public EOC eoc()
+    {
+        return EOC.NONE;
+    }
+
+    public Composite withEOC(EOC eoc)
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    public Composite start()
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    public Composite end()
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    public ColumnSlice slice()
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    public boolean isPrefixOf(CType type, Composite c)
+    {
+        if (size() > c.size() || isStatic() != c.isStatic())
+            return false;
+
+        for (int i = 0; i < size(); i++)
+        {
+            if (type.subtype(i).compare(get(i), c.get(i)) != 0)
+                return false;
+        }
+        return true;
+    }
+
+    public ByteBuffer toByteBuffer()
+    {
+        // for simple sparse we just return our one name buffer
+        switch (nametype())
+        {
+            case SIMPLE_DENSE:
+            case SIMPLE_SPARSE:
+                return get(0);
+            case COMPOUND_DENSE:
+            case COMPOUND_SPARSE_STATIC:
+            case COMPOUND_SPARSE:
+                // This is the legacy format of composites.
+                // See org.apache.cassandra.db.marshal.CompositeType for details.
+                ByteBuffer result = ByteBuffer.allocate(cellDataSize());
+                if (isStatic())
+                    ByteBufferUtil.writeShortLength(result, CompositeType.STATIC_MARKER);
+
+                for (int i = 0; i < size(); i++)
+                {
+                    ByteBuffer bb = get(i);
+                    ByteBufferUtil.writeShortLength(result, bb.remaining());
+                    result.put(bb);
+                    result.put((byte) 0);
+                }
+                result.flip();
+                return result;
+            default:
+                throw new AssertionError();
+        }
+    }
+
+    protected void updateWithName(MessageDigest digest)
+    {
+        // for simple sparse we just return our one name buffer
+        switch (nametype())
+        {
+            case SIMPLE_DENSE:
+            case SIMPLE_SPARSE:
+                writeComponentTo(digest, 0, false);
+                break;
+
+            case COMPOUND_DENSE:
+            case COMPOUND_SPARSE_STATIC:
+            case COMPOUND_SPARSE:
+                // This is the legacy format of composites.
+                // See org.apache.cassandra.db.marshal.CompositeType for details.
+                if (isStatic())
+                    FBUtilities.updateWithShort(digest, CompositeType.STATIC_MARKER);
+
+                for (int i = 0; i < size(); i++)
+                {
+                    writeComponentTo(digest, i, true);
+                    digest.update((byte) 0);
+                }
+                break;
+
+            default:
+                throw new AssertionError();
+        }
+    }
+
+    protected void updateWithValue(MessageDigest digest)
+    {
+        int offset = valueStartOffset();
+        int length = valueEndOffset() - offset;
+        writeMemoryTo(digest, offset, length);
+    }
+
+    @Override // this is the NAME dataSize, only!
+    public int dataSize()
+    {
+        switch (nametype())
+        {
+            case SIMPLE_DENSE:
+            case SIMPLE_SPARSE:
+                return valueStartOffset() - nameDeltaOffset(size());
+            case COMPOUND_DENSE:
+            case COMPOUND_SPARSE_STATIC:
+            case COMPOUND_SPARSE:
+                int size = size();
+                return valueStartOffset() - nameDeltaOffset(size) + 3 * size + (isStatic() ? 2 : 0);
+            default:
+                throw new AssertionError();
+        }
+    }
+
+    public boolean equals(Object obj)
+    {
+        if (obj == this)
+            return true;
+        if (obj instanceof CellName)
+            return equals((CellName) obj);
+        if (obj instanceof Cell)
+            return equals((Cell) obj);
+        return false;
+    }
+
+    public boolean equals(CellName that)
+    {
+        int size = this.size();
+        if (size != that.size())
+            return false;
+
+        for (int i = 0 ; i < size ; i++)
+            if (!get(i).equals(that.get(i)))
+                return false;
+        return true;
+    }
+
+    private static final ByteBuffer[] EMPTY = new ByteBuffer[0];
+
+    @Override
+    public CellName copy(CFMetaData cfm, AbstractAllocator allocator)
+    {
+        ByteBuffer[] r;
+        switch (nametype())
+        {
+            case SIMPLE_DENSE:
+                return CellNames.simpleDense(allocator.clone(get(0)));
+
+            case COMPOUND_DENSE:
+                r = new ByteBuffer[size()];
+                for (int i = 0; i < r.length; i++)
+                    r[i] = allocator.clone(get(i));
+                return CellNames.compositeDense(r);
+
+            case COMPOUND_SPARSE_STATIC:
+            case COMPOUND_SPARSE:
+                int clusteringSize = clusteringSize();
+                r = clusteringSize == 0 ? EMPTY : new ByteBuffer[clusteringSize()];
+                for (int i = 0; i < clusteringSize; i++)
+                    r[i] = allocator.clone(get(i));
+
+                ByteBuffer nameBuffer = get(r.length);
+                ColumnIdentifier name;
+
+                if (nameBuffer.remaining() == 0)
+                {
+                    name = CompoundSparseCellNameType.rowMarkerId;
+                }
+                else
+                {
+                    name = getIdentifier(cfm, nameBuffer);
+                }
+
+                if (clusteringSizeDelta() == 2)
+                {
+                    ByteBuffer element = allocator.clone(get(size() - 1));
+                    return CellNames.compositeSparseWithCollection(r, element, name, isStatic());
+                }
+                return CellNames.compositeSparse(r, name, isStatic());
+
+            case SIMPLE_SPARSE:
+                return CellNames.simpleSparse(getIdentifier(cfm, get(0)));
+        }
+        throw new IllegalStateException();
+    }
+
+    private static ColumnIdentifier getIdentifier(CFMetaData cfMetaData, ByteBuffer name)
+    {
+        ColumnDefinition def = cfMetaData.getColumnDefinition(name);
+        if (def != null)
+        {
+            return def.name;
+        }
+        else
+        {
+            // it's safe to simply grab based on clusteringPrefixSize() as we are only called if not a dense type
+            AbstractType<?> type = cfMetaData.comparator.subtype(cfMetaData.comparator.clusteringPrefixSize());
+            return new ColumnIdentifier(HeapAllocator.instance.clone(name), type);
+        }
+    }
+
+    @Override
+    public Cell withUpdatedName(CellName newName)
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public Cell withUpdatedTimestamp(long newTimestamp)
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    protected long internalSize()
+    {
+        return MemoryUtil.getInt(peer);
+    }
+
+    private void checkPosition(long offset, long size)
+    {
+        assert size >= 0;
+        assert peer > 0 : "Memory was freed";
+        assert offset >= 0 && offset + size <= internalSize() : String.format("Illegal range: [%d..%d), size: %s", offset, offset + size, internalSize());
+    }
+
+    protected final void setByte(long offset, byte b)
+    {
+        checkPosition(offset, 1);
+        MemoryUtil.setByte(peer + offset, b);
+    }
+
+    protected final void setShort(long offset, short s)
+    {
+        checkPosition(offset, 1);
+        MemoryUtil.setShort(peer + offset, s);
+    }
+
+    protected final void setInt(long offset, int l)
+    {
+        checkPosition(offset, 4);
+        MemoryUtil.setInt(peer + offset, l);
+    }
+
+    protected final void setLong(long offset, long l)
+    {
+        checkPosition(offset, 8);
+        MemoryUtil.setLong(peer + offset, l);
+    }
+
+    protected final void setBytes(long offset, ByteBuffer buffer)
+    {
+        int start = buffer.position();
+        int count = buffer.limit() - start;
+        if (count == 0)
+            return;
+
+        checkPosition(offset, count);
+        MemoryUtil.setBytes(peer + offset, buffer);
+    }
+
+    protected final byte getByte(long offset)
+    {
+        checkPosition(offset, 1);
+        return MemoryUtil.getByte(peer + offset);
+    }
+
+    protected final void getBytes(long offset, byte[] trg, int trgOffset, int count)
+    {
+        checkPosition(offset, count);
+        MemoryUtil.getBytes(peer + offset, trg, trgOffset, count);
+    }
+
+    protected final int getShort(long offset)
+    {
+        checkPosition(offset, 2);
+        return MemoryUtil.getShort(peer + offset);
+    }
+
+    protected final int getInt(long offset)
+    {
+        checkPosition(offset, 4);
+        return MemoryUtil.getInt(peer + offset);
+    }
+
+    protected final long getLong(long offset)
+    {
+        checkPosition(offset, 8);
+        return MemoryUtil.getLong(peer + offset);
+    }
+
+    protected final ByteBuffer getByteBuffer(long offset, int length)
+    {
+        checkPosition(offset, length);
+        return MemoryUtil.getByteBuffer(peer + offset, length);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/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 e04867a..c9cff77 100644
--- a/src/java/org/apache/cassandra/db/ArrayBackedSortedColumns.java
+++ b/src/java/org/apache/cassandra/db/ArrayBackedSortedColumns.java
@@ -90,7 +90,7 @@ public class ArrayBackedSortedColumns extends ColumnFamily
     {
         ArrayBackedSortedColumns copy = new ArrayBackedSortedColumns(original.metadata, false, new Cell[original.getColumnCount()], 0, 0);
         for (Cell cell : original)
-            copy.internalAdd(cell.localCopy(allocator));
+            copy.internalAdd(cell.localCopy(original.metadata, allocator));
         copy.sortedSize = copy.size; // internalAdd doesn't update sortedSize.
         copy.delete(original);
         return copy;
@@ -138,7 +138,7 @@ public class ArrayBackedSortedColumns extends ColumnFamily
         Arrays.sort(cells, sortedSize, size, comparator);
 
         // Determine the merge start position for that segment
-        int pos = binarySearch(0, sortedSize, cells[sortedSize].name, internalComparator());
+        int pos = binarySearch(0, sortedSize, cells[sortedSize].name(), internalComparator());
         if (pos < 0)
             pos = -pos - 1;
 
@@ -420,7 +420,7 @@ public class ArrayBackedSortedColumns extends ColumnFamily
         {
             public CellName apply(Cell cell)
             {
-                return cell.name;
+                return cell.name();
             }
         });
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/db/AtomicBTreeColumns.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/AtomicBTreeColumns.java b/src/java/org/apache/cassandra/db/AtomicBTreeColumns.java
index 72038b6..20fe64c 100644
--- a/src/java/org/apache/cassandra/db/AtomicBTreeColumns.java
+++ b/src/java/org/apache/cassandra/db/AtomicBTreeColumns.java
@@ -19,7 +19,6 @@ package org.apache.cassandra.db;
 
 import java.util.AbstractCollection;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Collection;
 import java.util.Comparator;
 import java.util.Iterator;
@@ -28,17 +27,19 @@ import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
 
 import com.google.common.base.Function;
 import com.google.common.base.Functions;
-import com.google.common.collect.*;
+import com.google.common.collect.Iterators;
 
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.composites.CellName;
 import org.apache.cassandra.db.composites.Composite;
-import org.apache.cassandra.db.index.SecondaryIndexManager;
 import org.apache.cassandra.db.filter.ColumnSlice;
+import org.apache.cassandra.db.index.SecondaryIndexManager;
 import org.apache.cassandra.utils.ObjectSizes;
 import org.apache.cassandra.utils.btree.BTree;
 import org.apache.cassandra.utils.btree.BTreeSet;
 import org.apache.cassandra.utils.btree.UpdateFunction;
+import org.apache.cassandra.utils.concurrent.OpOrder;
+import org.apache.cassandra.utils.memory.MemtableAllocator;
 
 import static org.apache.cassandra.db.index.SecondaryIndexManager.Updater;
 
@@ -53,14 +54,14 @@ import static org.apache.cassandra.db.index.SecondaryIndexManager.Updater;
  */
 public class AtomicBTreeColumns extends ColumnFamily
 {
-    static final long HEAP_SIZE = ObjectSizes.measure(new AtomicBTreeColumns(CFMetaData.IndexCf, null))
+    static final long EMPTY_SIZE = ObjectSizes.measure(new AtomicBTreeColumns(CFMetaData.IndexCf, null))
             + ObjectSizes.measure(new Holder(null, null));
 
     private static final Function<Cell, CellName> NAME = new Function<Cell, CellName>()
     {
         public CellName apply(Cell column)
         {
-            return column.name;
+            return column.name();
         }
     };
 
@@ -155,110 +156,40 @@ public class AtomicBTreeColumns extends ColumnFamily
         }
     }
 
-    // the function we provide to the btree utilities to perform any column replacements
-    private static final class ColumnUpdater implements UpdateFunction<Cell>
-    {
-        final AtomicBTreeColumns updating;
-        final Holder ref;
-        final Function<Cell, Cell> transform;
-        final Updater indexer;
-        final Delta delta;
-
-        private ColumnUpdater(AtomicBTreeColumns updating, Holder ref, Function<Cell, Cell> transform, Updater indexer, Delta delta)
-        {
-            this.updating = updating;
-            this.ref = ref;
-            this.transform = transform;
-            this.indexer = indexer;
-            this.delta = delta;
-        }
-
-        public Cell apply(Cell inserted)
-        {
-            indexer.insert(inserted);
-            delta.insert(inserted);
-            return transform.apply(inserted);
-        }
-
-        public Cell apply(Cell existing, Cell update)
-        {
-            Cell reconciled = update.reconcile(existing);
-            indexer.update(existing, reconciled);
-            if (existing != reconciled)
-                delta.swap(existing, reconciled);
-            else
-                delta.abort(update);
-            return transform.apply(reconciled);
-        }
-
-        public boolean abortEarly()
-        {
-            return updating.ref != ref;
-        }
-
-        public void allocated(long heapSize)
-        {
-            delta.addHeapSize(heapSize);
-        }
-    }
-
-    private static Collection<Cell> transform(Comparator<Cell> cmp, ColumnFamily cf, Function<Cell, Cell> transformation, boolean sort)
-    {
-        Cell[] tmp = new Cell[cf.getColumnCount()];
-
-        int i = 0;
-        for (Cell c : cf)
-            tmp[i++] = transformation.apply(c);
-
-        if (sort)
-            Arrays.sort(tmp, cmp);
-
-        return Arrays.asList(tmp);
-    }
-
     /**
      * This is only called by Memtable.resolve, so only AtomicBTreeColumns needs to implement it.
      *
      * @return the difference in size seen after merging the given columns
      */
-    public Delta addAllWithSizeDelta(final ColumnFamily cm, Function<Cell, Cell> transformation, Updater indexer, Delta delta)
+    public long addAllWithSizeDelta(final ColumnFamily cm, MemtableAllocator allocator, OpOrder.Group writeOp, Updater indexer)
     {
-        boolean transformed = false;
-        Collection<Cell> insert = cm.getSortedColumns();
-
+        ColumnUpdater updater = new ColumnUpdater(this, cm.metadata, allocator, writeOp, indexer);
         while (true)
         {
             Holder current = ref;
+            updater.ref = current;
+            updater.reset();
 
-            delta.reset();
             DeletionInfo deletionInfo;
             if (cm.deletionInfo().mayModify(current.deletionInfo))
             {
                 deletionInfo = current.deletionInfo.copy().add(cm.deletionInfo());
-                delta.addHeapSize(deletionInfo.unsharedHeapSize() - current.deletionInfo.unsharedHeapSize());
+                updater.allocated(deletionInfo.unsharedHeapSize() - current.deletionInfo.unsharedHeapSize());
             }
             else
             {
                 deletionInfo = current.deletionInfo;
             }
 
-            ColumnUpdater updater = new ColumnUpdater(this, current, transformation, indexer, delta);
-            Object[] tree = BTree.update(current.tree, metadata.comparator.columnComparator(), insert, true, updater);
+            Object[] tree = BTree.update(current.tree, metadata.comparator.columnComparator(), cm, cm.getColumnCount(), true, updater);
 
             if (tree != null && refUpdater.compareAndSet(this, current, new Holder(tree, deletionInfo)))
             {
                 indexer.updateRowLevelIndexes();
-                return updater.delta;
-            }
-
-            if (!transformed)
-            {
-                // After failing once, transform Columns into a new collection to avoid repeatedly allocating Slab space
-                insert = transform(metadata.comparator.columnComparator(), cm, transformation, false);
-                transformed = true;
+                updater.finish();
+                return updater.dataSize;
             }
         }
-
     }
 
     // no particular reason not to implement these next methods, we just haven't needed them yet
@@ -290,7 +221,7 @@ public class AtomicBTreeColumns extends ColumnFamily
         {
             public int compare(Object o1, Object o2)
             {
-                return cmp.compare((CellName) o1, ((Cell) o2).name);
+                return cmp.compare((CellName) o1, ((Cell) o2).name());
             }
         };
     }
@@ -352,7 +283,7 @@ public class AtomicBTreeColumns extends ColumnFamily
         return false;
     }
 
-    private static class Holder
+    private static final class Holder
     {
         final DeletionInfo deletionInfo;
         // the btree of columns
@@ -375,69 +306,96 @@ public class AtomicBTreeColumns extends ColumnFamily
         }
     }
 
-    // TODO: create a stack-allocation-friendly list to help optimise garbage for updates to rows with few columns
-
-    /**
-     * tracks the size changes made while merging a new group of cells in
-     */
-    public static final class Delta
+    // the function we provide to the btree utilities to perform any column replacements
+    private static final class ColumnUpdater implements UpdateFunction<Cell>
     {
-        private long dataSize;
-        private long heapSize;
+        final AtomicBTreeColumns updating;
+        final CFMetaData metadata;
+        final MemtableAllocator allocator;
+        final OpOrder.Group writeOp;
+        final Updater indexer;
+        Holder ref;
+        long dataSize;
+        long heapSize;
+        final MemtableAllocator.DataReclaimer reclaimer;
+        List<Cell> inserted; // TODO: replace with walk of aborted BTree
 
-        // we track the discarded cells (cells that were in the btree, but replaced by new ones)
-        // separately from aborted ones (were part of an update but older than existing cells)
-        // since we need to reset the former when we race on the btree update, but not the latter
-        private List<Cell> discarded = new ArrayList<>();
-        private List<Cell> aborted;
+        private ColumnUpdater(AtomicBTreeColumns updating, CFMetaData metadata, MemtableAllocator allocator, OpOrder.Group writeOp, Updater indexer)
+        {
+            this.updating = updating;
+            this.allocator = allocator;
+            this.writeOp = writeOp;
+            this.indexer = indexer;
+            this.metadata = metadata;
+            this.reclaimer = allocator.reclaimer();
+        }
 
-        protected void reset()
+        public Cell apply(Cell insert)
         {
-            this.dataSize = 0;
-            this.heapSize = 0;
-            discarded.clear();
+            indexer.insert(insert);
+            insert = insert.localCopy(metadata, allocator, writeOp);
+            this.dataSize += insert.cellDataSize();
+            this.heapSize += insert.excessHeapSizeExcludingData();
+            if (inserted == null)
+                inserted = new ArrayList<>();
+            inserted.add(insert);
+            return insert;
         }
 
-        protected void addHeapSize(long heapSize)
+        public Cell apply(Cell existing, Cell update)
         {
-            this.heapSize += heapSize;
+            Cell reconciled = existing.reconcile(update);
+            indexer.update(existing, reconciled);
+            if (existing != reconciled)
+            {
+                reconciled = reconciled.localCopy(metadata, allocator, writeOp);
+                dataSize += reconciled.cellDataSize() - existing.cellDataSize();
+                heapSize += reconciled.excessHeapSizeExcludingData() - existing.excessHeapSizeExcludingData();
+                if (inserted == null)
+                    inserted = new ArrayList<>();
+                inserted.add(reconciled);
+                discard(existing);
+            }
+            return reconciled;
         }
 
-        protected void swap(Cell old, Cell updated)
+        protected void reset()
         {
-            dataSize += updated.dataSize() - old.dataSize();
-            heapSize += updated.excessHeapSizeExcludingData() - old.excessHeapSizeExcludingData();
-            discarded.add(old);
+            this.dataSize = 0;
+            this.heapSize = 0;
+            if (inserted != null)
+            {
+                for (Cell cell : inserted)
+                    abort(cell);
+                inserted.clear();
+            }
+            reclaimer.cancel();
         }
 
-        protected void insert(Cell insert)
+        protected void abort(Cell abort)
         {
-            this.dataSize += insert.dataSize();
-            this.heapSize += insert.excessHeapSizeExcludingData();
+            reclaimer.reclaimImmediately(abort);
         }
 
-        private void abort(Cell neverUsed)
+        protected void discard(Cell discard)
         {
-            if (aborted == null)
-                aborted = new ArrayList<>();
-            aborted.add(neverUsed);
+            reclaimer.reclaim(discard);
         }
 
-        public long dataSize()
+        public boolean abortEarly()
         {
-            return dataSize;
+            return updating.ref != ref;
         }
 
-        public long excessHeapSize()
+        public void allocated(long heapSize)
         {
-            return heapSize;
+            this.heapSize += heapSize;
         }
 
-        public Iterable<Cell> reclaimed()
+        protected void finish()
         {
-            if (aborted == null)
-                return discarded;
-            return Iterables.concat(discarded, aborted);
+            allocator.onHeap().allocate(heapSize, writeOp);
+            reclaimer.commit();
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/db/BufferCell.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/BufferCell.java b/src/java/org/apache/cassandra/db/BufferCell.java
new file mode 100644
index 0000000..93251c8
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/BufferCell.java
@@ -0,0 +1,103 @@
+/*
+ * 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.config.CFMetaData;
+import org.apache.cassandra.db.composites.CellName;
+import org.apache.cassandra.db.composites.CellNames;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.ObjectSizes;
+import org.apache.cassandra.utils.concurrent.OpOrder;
+import org.apache.cassandra.utils.memory.AbstractAllocator;
+import org.apache.cassandra.utils.memory.MemtableAllocator;
+
+public class BufferCell extends AbstractCell
+{
+    private static final long EMPTY_SIZE = ObjectSizes.measure(new BufferCell(CellNames.simpleDense(ByteBuffer.allocate(1))));
+
+    protected final CellName name;
+    protected final ByteBuffer value;
+    protected final long timestamp;
+
+    BufferCell(CellName name)
+    {
+        this(name, ByteBufferUtil.EMPTY_BYTE_BUFFER);
+    }
+
+    public BufferCell(CellName name, ByteBuffer value)
+    {
+        this(name, value, 0);
+    }
+
+    public BufferCell(CellName name, ByteBuffer value, long timestamp)
+    {
+        assert name != null;
+        assert value != null;
+
+        this.name = name;
+        this.value = value;
+        this.timestamp = timestamp;
+    }
+
+    @Override
+    public Cell withUpdatedName(CellName newName)
+    {
+        return new BufferCell(newName, value, timestamp);
+    }
+
+    @Override
+    public Cell withUpdatedTimestamp(long newTimestamp)
+    {
+        return new BufferCell(name, value, newTimestamp);
+    }
+
+    @Override
+    public CellName name() {
+        return name;
+    }
+
+    @Override
+    public ByteBuffer value() {
+        return value;
+    }
+
+    @Override
+    public long timestamp() {
+        return timestamp;
+    }
+
+    @Override
+    public long excessHeapSizeExcludingData()
+    {
+        return EMPTY_SIZE + name.excessHeapSizeExcludingData() + ObjectSizes.sizeOnHeapExcludingData(value);
+    }
+
+    @Override
+    public Cell localCopy(CFMetaData metadata, AbstractAllocator allocator)
+    {
+        return new BufferCell(name.copy(metadata, allocator), allocator.clone(value), timestamp);
+    }
+
+    @Override
+    public Cell localCopy(CFMetaData metadata, MemtableAllocator allocator, OpOrder.Group opGroup)
+    {
+        return allocator.clone(this, metadata, opGroup);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/db/BufferCounterCell.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/BufferCounterCell.java b/src/java/org/apache/cassandra/db/BufferCounterCell.java
new file mode 100644
index 0000000..a70e274
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/BufferCounterCell.java
@@ -0,0 +1,181 @@
+/*
+ * 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.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.serializers.MarshalException;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.concurrent.OpOrder;
+import org.apache.cassandra.utils.memory.AbstractAllocator;
+import org.apache.cassandra.utils.memory.MemtableAllocator;
+
+public class BufferCounterCell extends BufferCell implements CounterCell
+{
+    private final long timestampOfLastDelete;
+
+    public BufferCounterCell(CellName name, ByteBuffer value, long timestamp)
+    {
+        this(name, value, timestamp, Long.MIN_VALUE);
+    }
+
+    public BufferCounterCell(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)
+    {
+        if (flag == ColumnSerializer.Flag.FROM_REMOTE || (flag == ColumnSerializer.Flag.LOCAL && contextManager.shouldClearLocal(value)))
+            value = contextManager.clearAllLocal(value);
+        return new BufferCounterCell(name, value, timestamp, timestampOfLastDelete);
+    }
+
+    // For use by tests of compatibility with pre-2.1 counter only.
+    public static CounterCell createLocal(CellName name, long value, long timestamp, long timestampOfLastDelete)
+    {
+        return new BufferCounterCell(name, contextManager.createLocal(value), timestamp, timestampOfLastDelete);
+    }
+
+    @Override
+    public Cell withUpdatedName(CellName newName)
+    {
+        return new BufferCounterCell(newName, value, timestamp, timestampOfLastDelete);
+    }
+
+    @Override
+    public long timestampOfLastDelete()
+    {
+        return timestampOfLastDelete;
+    }
+
+    @Override
+    public long total()
+    {
+        return contextManager.total(value);
+    }
+
+    @Override
+    public int cellDataSize()
+    {
+        // A counter column adds 8 bytes for timestampOfLastDelete to Cell.
+        return super.cellDataSize() + 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)
+    {
+        return diff(this, cell);
+    }
+
+    /*
+     * 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());
+
+        FBUtilities.updateWithLong(digest, timestamp);
+        FBUtilities.updateWithByte(digest, serializationFlags());
+        FBUtilities.updateWithLong(digest, timestampOfLastDelete);
+    }
+
+    @Override
+    public Cell reconcile(Cell cell)
+    {
+        return reconcile(this, cell);
+    }
+
+    @Override
+    public boolean hasLegacyShards()
+    {
+        return contextManager.hasLegacyShards(value);
+    }
+
+    @Override
+    public CounterCell localCopy(CFMetaData metadata, AbstractAllocator allocator)
+    {
+        return new BufferCounterCell(name.copy(metadata, allocator), allocator.clone(value), timestamp, timestampOfLastDelete);
+    }
+
+    @Override
+    public CounterCell localCopy(CFMetaData metadata, MemtableAllocator allocator, OpOrder.Group opGroup)
+    {
+        return allocator.clone(this, metadata, opGroup);
+    }
+
+    @Override
+    public String getString(CellNameType comparator)
+    {
+        return String.format("%s:false:%s@%d!%d",
+                             comparator.getString(name()),
+                             contextManager.toString(value()),
+                             timestamp(),
+                             timestampOfLastDelete);
+    }
+
+    @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());
+    }
+
+    @Override
+    public Cell markLocalToBeCleared()
+    {
+        ByteBuffer marked = contextManager.markLocalToBeCleared(value());
+        return marked == value() ? this : new BufferCounterCell(name(), marked, timestamp(), timestampOfLastDelete);
+    }
+
+    @Override
+    public boolean equals(Cell cell)
+    {
+        return cell instanceof CounterCell && equals((CounterCell) cell);
+    }
+
+    public boolean equals(CounterCell cell)
+    {
+        return super.equals(cell) && timestampOfLastDelete == cell.timestampOfLastDelete();
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/db/BufferCounterUpdateCell.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/BufferCounterUpdateCell.java b/src/java/org/apache/cassandra/db/BufferCounterUpdateCell.java
new file mode 100644
index 0000000..44ab83e
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/BufferCounterUpdateCell.java
@@ -0,0 +1,93 @@
+/*
+ * 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.config.CFMetaData;
+import org.apache.cassandra.db.composites.CellName;
+import org.apache.cassandra.db.composites.CellNameType;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.concurrent.OpOrder;
+import org.apache.cassandra.utils.memory.AbstractAllocator;
+import org.apache.cassandra.utils.memory.MemtableAllocator;
+
+public class BufferCounterUpdateCell extends BufferCell implements CounterUpdateCell
+{
+    public BufferCounterUpdateCell(CellName name, long value, long timestamp)
+    {
+        this(name, ByteBufferUtil.bytes(value), timestamp);
+    }
+
+    public BufferCounterUpdateCell(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)
+    {
+        // The only time this could happen is if a batchAdd ships two
+        // increment for the same cell. Hence we simply sums the delta.
+
+        // 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
+        assert cell instanceof CounterUpdateCell : "Wrong class type.";
+        CounterUpdateCell c = (CounterUpdateCell) cell;
+        return new BufferCounterUpdateCell(name, delta() + c.delta(), Math.max(timestamp, c.timestamp()));
+    }
+
+    @Override
+    public int serializationFlags()
+    {
+        return ColumnSerializer.COUNTER_UPDATE_MASK;
+    }
+
+    @Override
+    public Cell localCopy(CFMetaData metadata, AbstractAllocator allocator)
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public Cell localCopy(CFMetaData metadata, MemtableAllocator allocator, OpOrder.Group opGroup)
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public String getString(CellNameType comparator)
+    {
+        return String.format("%s:%s@%d", comparator.getString(name()), ByteBufferUtil.toLong(value), timestamp());
+    }
+}


[4/7] Push more of memtable data off-heap

Posted by sl...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/dht/OrderPreservingPartitioner.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/dht/OrderPreservingPartitioner.java b/src/java/org/apache/cassandra/dht/OrderPreservingPartitioner.java
index ab80ef7..ad1ffaa 100644
--- a/src/java/org/apache/cassandra/dht/OrderPreservingPartitioner.java
+++ b/src/java/org/apache/cassandra/dht/OrderPreservingPartitioner.java
@@ -23,6 +23,7 @@ import java.nio.charset.CharacterCodingException;
 import java.util.*;
 
 import org.apache.cassandra.config.*;
+import org.apache.cassandra.db.BufferDecoratedKey;
 import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.marshal.UTF8Type;
@@ -44,7 +45,7 @@ public class OrderPreservingPartitioner extends AbstractPartitioner<StringToken>
 
     public DecoratedKey decorateKey(ByteBuffer key)
     {
-        return new DecoratedKey(getToken(key), key);
+        return new BufferDecoratedKey(getToken(key), key);
     }
 
     public StringToken midpoint(Token ltoken, Token rtoken)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/dht/RandomPartitioner.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/dht/RandomPartitioner.java b/src/java/org/apache/cassandra/dht/RandomPartitioner.java
index a5b0ce9..3ccd086 100644
--- a/src/java/org/apache/cassandra/dht/RandomPartitioner.java
+++ b/src/java/org/apache/cassandra/dht/RandomPartitioner.java
@@ -22,6 +22,7 @@ import java.math.BigInteger;
 import java.nio.ByteBuffer;
 import java.util.*;
 
+import org.apache.cassandra.db.BufferDecoratedKey;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.marshal.AbstractType;
@@ -45,7 +46,7 @@ public class RandomPartitioner extends AbstractPartitioner<BigIntegerToken>
 
     public DecoratedKey decorateKey(ByteBuffer key)
     {
-        return new DecoratedKey(getToken(key), key);
+        return new BufferDecoratedKey(getToken(key), key);
     }
 
     public Token midpoint(Token ltoken, Token rtoken)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/dht/Token.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/dht/Token.java b/src/java/org/apache/cassandra/dht/Token.java
index 0f50d42..09b73b7 100644
--- a/src/java/org/apache/cassandra/dht/Token.java
+++ b/src/java/org/apache/cassandra/dht/Token.java
@@ -173,7 +173,7 @@ public abstract class Token<T> implements RingPosition<Token<T>>, Serializable
             return (R)maxKeyBound();
     }
 
-    public static class KeyBound extends RowPosition
+    public static class KeyBound implements RowPosition
     {
         private final Token token;
         public final boolean isMinimumBound;
@@ -209,6 +209,11 @@ public abstract class Token<T> implements RingPosition<Token<T>>, Serializable
             return getToken().isMinimum(partitioner);
         }
 
+        public boolean isMinimum()
+        {
+            return isMinimum(StorageService.getPartitioner());
+        }
+
         public RowPosition.Kind kind()
         {
             return isMinimumBound ? RowPosition.Kind.MIN_BOUND : RowPosition.Kind.MAX_BOUND;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/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 f6d2b7e..b6b12ec 100644
--- a/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordReader.java
+++ b/src/java/org/apache/cassandra/hadoop/ColumnFamilyRecordReader.java
@@ -24,6 +24,7 @@ import java.nio.ByteBuffer;
 import java.util.*;
 
 import com.google.common.collect.*;
+import org.apache.cassandra.db.BufferCell;
 import org.apache.cassandra.db.Cell;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -304,14 +305,14 @@ public class ColumnFamilyRecordReader extends RecordReader<ByteBuffer, SortedMap
 
         protected Cell unthriftifySimple(org.apache.cassandra.thrift.Column column)
         {
-            return new Cell(comparator.cellFromByteBuffer(column.name), column.value, column.timestamp);
+            return new BufferCell(comparator.cellFromByteBuffer(column.name), column.value, column.timestamp);
         }
 
         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 Cell.
-            return new Cell(comparator.cellFromByteBuffer(column.name), ByteBufferUtil.bytes(column.value), 0);
+            return new BufferCell(comparator.cellFromByteBuffer(column.name), ByteBufferUtil.bytes(column.value), 0);
         }
 
         private List<Cell> unthriftifySuperCounter(CounterSuperColumn super_column)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/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 efa6d34..6f17468 100644
--- a/src/java/org/apache/cassandra/hadoop/pig/CqlStorage.java
+++ b/src/java/org/apache/cassandra/hadoop/pig/CqlStorage.java
@@ -24,6 +24,7 @@ import java.util.*;
 
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.db.BufferCell;
 import org.apache.cassandra.db.composites.CellNames;
 import org.apache.cassandra.db.Cell;
 import org.apache.cassandra.db.marshal.*;
@@ -112,7 +113,7 @@ public class CqlStorage extends AbstractCassandraStorage
                 ByteBuffer columnValue = columns.get(ByteBufferUtil.string(cdef.name.duplicate()));
                 if (columnValue != null)
                 {
-                    Cell cell = new Cell(CellNames.simpleDense(cdef.name), columnValue);
+                    Cell cell = new BufferCell(CellNames.simpleDense(cdef.name), columnValue);
                     AbstractType<?> validator = getValidatorMap(cfDef).get(cdef.name);
                     setTupleValue(tuple, i, cqlColumnToObj(cell, cfDef), validator);
                 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/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 7e7b364..1b407c5 100644
--- a/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
@@ -131,7 +131,7 @@ public abstract class AbstractSSTableSimpleWriter
      */
     public void addColumn(ByteBuffer name, ByteBuffer value, long timestamp)
     {
-        addColumn(new Cell(metadata.comparator.cellFromByteBuffer(name), value, timestamp));
+        addColumn(new BufferCell(metadata.comparator.cellFromByteBuffer(name), value, timestamp));
     }
 
     /**
@@ -146,7 +146,7 @@ public abstract class AbstractSSTableSimpleWriter
      */
     public void addExpiringColumn(ByteBuffer name, ByteBuffer value, long timestamp, int ttl, long expirationTimestampMS)
     {
-        addColumn(new ExpiringCell(metadata.comparator.cellFromByteBuffer(name), value, timestamp, ttl, (int)(expirationTimestampMS / 1000)));
+        addColumn(new BufferExpiringCell(metadata.comparator.cellFromByteBuffer(name), value, timestamp, ttl, (int)(expirationTimestampMS / 1000)));
     }
 
     /**
@@ -156,9 +156,9 @@ public abstract class AbstractSSTableSimpleWriter
      */
     public void addCounterColumn(ByteBuffer name, long value)
     {
-        addColumn(new CounterCell(metadata.comparator.cellFromByteBuffer(name),
-                                  CounterContext.instance().createGlobal(counterid, 1L, value),
-                                  System.currentTimeMillis()));
+        addColumn(new BufferCounterCell(metadata.comparator.cellFromByteBuffer(name),
+                                        CounterContext.instance().createGlobal(counterid, 1L, value),
+                                        System.currentTimeMillis()));
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java b/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java
index 592ad6c..9b7a8e7 100644
--- a/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/CQLSSTableWriter.java
@@ -207,7 +207,7 @@ public class CQLSSTableWriter
 
         for (ByteBuffer key: keys)
         {
-            if (writer.currentKey() == null || !key.equals(writer.currentKey().key))
+            if (writer.currentKey() == null || !key.equals(writer.currentKey().getKey()))
                 writer.newRow(key);
             insert.addUpdateForKey(writer.currentColumnFamily(), key, clusteringPrefix, params);
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/io/sstable/IndexSummaryBuilder.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/IndexSummaryBuilder.java b/src/java/org/apache/cassandra/io/sstable/IndexSummaryBuilder.java
index 8580dce..8e9cc30 100644
--- a/src/java/org/apache/cassandra/io/sstable/IndexSummaryBuilder.java
+++ b/src/java/org/apache/cassandra/io/sstable/IndexSummaryBuilder.java
@@ -30,6 +30,7 @@ import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.dht.IPartitioner;
 
 import static org.apache.cassandra.io.sstable.Downsampling.BASE_SAMPLING_LEVEL;
+import static org.apache.cassandra.io.sstable.SSTable.getMinimalKey;
 
 public class IndexSummaryBuilder
 {
@@ -109,8 +110,8 @@ public class IndexSummaryBuilder
 
             if (!shouldSkip)
             {
-                keys.add(decoratedKey);
-                offheapSize += decoratedKey.key.remaining();
+                keys.add(getMinimalKey(decoratedKey));
+                offheapSize += decoratedKey.getKey().remaining();
                 positions.add(indexPosition);
                 offheapSize += TypeSizes.NATIVE.sizeof(indexPosition);
             }
@@ -143,7 +144,7 @@ public class IndexSummaryBuilder
         long offheapSize = this.offheapSize;
         if (length < keys.size())
             for (int i = length ; i < keys.size() ; i++)
-                offheapSize -= keys.get(i).key.remaining() + TypeSizes.NATIVE.sizeof(positions.get(i));
+                offheapSize -= keys.get(i).getKey().remaining() + TypeSizes.NATIVE.sizeof(positions.get(i));
 
         // first we write out the position in the *summary* for each key in the summary,
         // then we write out (key, actual index position) pairs
@@ -157,7 +158,7 @@ public class IndexSummaryBuilder
             idxPosition += TypeSizes.NATIVE.sizeof(keyPosition);
 
             // write the key
-            ByteBuffer keyBytes = keys.get(i).key;
+            ByteBuffer keyBytes = keys.get(i).getKey();
             memory.setBytes(keyPosition, keyBytes);
             keyPosition += keyBytes.remaining();
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/io/sstable/SSTable.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTable.java b/src/java/org/apache/cassandra/io/sstable/SSTable.java
index 247343e..6eff369 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTable.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTable.java
@@ -30,6 +30,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.db.BufferDecoratedKey;
 import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.RowIndexEntry;
 import org.apache.cassandra.dht.IPartitioner;
@@ -123,8 +124,8 @@ public abstract class SSTable
      */
     public static DecoratedKey getMinimalKey(DecoratedKey key)
     {
-        return key.key.position() > 0 || key.key.hasRemaining() || !key.key.hasArray()
-                                       ? new DecoratedKey(key.token, HeapAllocator.instance.clone(key.key))
+        return key.getKey().position() > 0 || key.getKey().hasRemaining() || !key.getKey().hasArray()
+                                       ? new BufferDecoratedKey(key.getToken(), HeapAllocator.instance.clone(key.getKey()))
                                        : key;
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/io/sstable/SSTableReader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableReader.java b/src/java/org/apache/cassandra/io/sstable/SSTableReader.java
index c84eec2..98fe5b6 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableReader.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableReader.java
@@ -791,7 +791,7 @@ public class SSTableReader extends SSTable
                 last = decoratedKey;
 
                 if (recreateBloomFilter)
-                    bf.add(decoratedKey.key);
+                    bf.add(decoratedKey.getKey());
 
                 // if summary was already read from disk we don't want to re-populate it using primary index
                 if (!summaryLoaded)
@@ -879,8 +879,8 @@ public class SSTableReader extends SSTable
         {
             oStream = new DataOutputStreamAndChannel(new FileOutputStream(summariesFile));
             IndexSummary.serializer.serialize(summary, oStream, descriptor.version.hasSamplingLevel);
-            ByteBufferUtil.writeWithLength(first.key, oStream);
-            ByteBufferUtil.writeWithLength(last.key, oStream);
+            ByteBufferUtil.writeWithLength(first.getKey(), oStream);
+            ByteBufferUtil.writeWithLength(last.getKey(), oStream);
             ibuilder.serializeBounds(oStream);
             dbuilder.serializeBounds(oStream);
         }
@@ -1309,7 +1309,7 @@ public class SSTableReader extends SSTable
 
     public void invalidateCacheKey(DecoratedKey key)
     {
-        KeyCacheKey cacheKey = new KeyCacheKey(metadata.cfId, descriptor, key.key);
+        KeyCacheKey cacheKey = new KeyCacheKey(metadata.cfId, descriptor, key.getKey());
         keyCache.remove(cacheKey);
     }
 
@@ -1324,14 +1324,14 @@ public class SSTableReader extends SSTable
             return;
         }
 
-        KeyCacheKey cacheKey = new KeyCacheKey(metadata.cfId, descriptor, key.key);
+        KeyCacheKey cacheKey = new KeyCacheKey(metadata.cfId, descriptor, key.getKey());
         logger.trace("Adding cache entry for {} -> {}", cacheKey, info);
         keyCache.put(cacheKey, info);
     }
 
     public RowIndexEntry getCachedPosition(DecoratedKey key, boolean updateStats)
     {
-        return getCachedPosition(new KeyCacheKey(metadata.cfId, descriptor, key.key), updateStats);
+        return getCachedPosition(new KeyCacheKey(metadata.cfId, descriptor, key.getKey()), updateStats);
     }
 
     private RowIndexEntry getCachedPosition(KeyCacheKey unifiedKey, boolean updateStats)
@@ -1375,7 +1375,7 @@ public class SSTableReader extends SSTable
         if (op == Operator.EQ)
         {
             assert key instanceof DecoratedKey; // EQ only make sense if the key is a valid row key
-            if (!bf.isPresent(((DecoratedKey)key).key))
+            if (!bf.isPresent(((DecoratedKey)key).getKey()))
             {
                 Tracing.trace("Bloom filter allows skipping sstable {}", descriptor.generation);
                 return null;
@@ -1386,7 +1386,7 @@ public class SSTableReader extends SSTable
         if ((op == Operator.EQ || op == Operator.GE) && (key instanceof DecoratedKey))
         {
             DecoratedKey decoratedKey = (DecoratedKey)key;
-            KeyCacheKey cacheKey = new KeyCacheKey(metadata.cfId, descriptor, decoratedKey.key);
+            KeyCacheKey cacheKey = new KeyCacheKey(metadata.cfId, descriptor, decoratedKey.getKey());
             RowIndexEntry cachedPosition = getCachedPosition(cacheKey, updateCacheAndStats);
             if (cachedPosition != null)
             {
@@ -1442,7 +1442,7 @@ public class SSTableReader extends SSTable
                     // Compare raw keys if possible for performance, otherwise compare decorated keys.
                     if (op == Operator.EQ)
                     {
-                        opSatisfied = exactMatch = indexKey.equals(((DecoratedKey) key).key);
+                        opSatisfied = exactMatch = indexKey.equals(((DecoratedKey) key).getKey());
                     }
                     else
                     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/io/sstable/SSTableScanner.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableScanner.java b/src/java/org/apache/cassandra/io/sstable/SSTableScanner.java
index 2af68ae..62ac175 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableScanner.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableScanner.java
@@ -257,7 +257,7 @@ public class SSTableScanner implements ICompactionScanner
                     }
                 }
 
-                if (dataRange == null || dataRange.selectsFullRowFor(currentKey.key))
+                if (dataRange == null || dataRange.selectsFullRowFor(currentKey.getKey()))
                 {
                     dfile.seek(currentEntry.position);
                     ByteBufferUtil.readWithShortLength(dfile); // key
@@ -269,7 +269,7 @@ public class SSTableScanner implements ICompactionScanner
                 {
                     public OnDiskAtomIterator create()
                     {
-                        return dataRange.columnFilter(currentKey.key).getSSTableColumnIterator(sstable, dfile, currentKey, currentEntry);
+                        return dataRange.columnFilter(currentKey.getKey()).getSSTableColumnIterator(sstable, dfile, currentKey, currentEntry);
                     }
                 });
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java b/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java
index 38fbece..c871a35 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableSimpleUnsortedWriter.java
@@ -99,7 +99,7 @@ public class SSTableSimpleUnsortedWriter extends AbstractSSTableSimpleWriter
 
     protected void writeRow(DecoratedKey key, ColumnFamily columnFamily) throws IOException
     {
-        currentSize += key.key.remaining() + ColumnFamily.serializer.serializedSize(columnFamily, MessagingService.current_version) * 1.2;
+        currentSize += key.getKey().remaining() + ColumnFamily.serializer.serializedSize(columnFamily, MessagingService.current_version) * 1.2;
 
         if (currentSize > bufferSize)
             sync();
@@ -118,7 +118,7 @@ public class SSTableSimpleUnsortedWriter extends AbstractSSTableSimpleWriter
         {
             // We will reuse a CF that we have counted already. But because it will be easier to add the full size
             // of the CF in the next writeRow call than to find out the delta, we just remove the size until that next call
-            currentSize -= currentKey.key.remaining() + ColumnFamily.serializer.serializedSize(previous, MessagingService.current_version) * 1.2;
+            currentSize -= currentKey.getKey().remaining() + ColumnFamily.serializer.serializedSize(previous, MessagingService.current_version) * 1.2;
         }
         return previous;
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/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 1c9c5fd..f29e4a3 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableWriter.java
@@ -175,7 +175,7 @@ public class SSTableWriter extends SSTable
 
     private void afterAppend(DecoratedKey decoratedKey, long dataPosition, RowIndexEntry index)
     {
-        sstableMetadataCollector.addKey(decoratedKey.key);
+        sstableMetadataCollector.addKey(decoratedKey.getKey());
         lastWrittenKey = decoratedKey;
         last = lastWrittenKey;
         if (first == null)
@@ -229,7 +229,7 @@ public class SSTableWriter extends SSTable
     {
         assert cf.hasColumns() || cf.isMarkedForDelete();
 
-        ColumnIndex.Builder builder = new ColumnIndex.Builder(cf, key.key, out);
+        ColumnIndex.Builder builder = new ColumnIndex.Builder(cf, key.getKey(), out);
         ColumnIndex index = builder.build(cf);
 
         out.writeShort(END_OF_ROW);
@@ -256,7 +256,7 @@ public class SSTableWriter extends SSTable
         ColumnFamily cf = ArrayBackedSortedColumns.factory.create(metadata);
         cf.delete(DeletionTime.serializer.deserialize(in));
 
-        ColumnIndex.Builder columnIndexer = new ColumnIndex.Builder(cf, key.key, dataFile.stream);
+        ColumnIndex.Builder columnIndexer = new ColumnIndex.Builder(cf, key.getKey(), dataFile.stream);
 
         if (cf.deletionInfo().getTopLevelDeletion().localDeletionTime < Integer.MAX_VALUE)
             tombstones.update(cf.deletionInfo().getTopLevelDeletion().localDeletionTime);
@@ -273,7 +273,7 @@ public class SSTableWriter extends SSTable
         {
             while (iter.hasNext())
             {
-                // deserialize column with PRESERVE_SIZE because we've written the dataSize based on the
+                // deserialize column with PRESERVE_SIZE because we've written the cellDataSize based on the
                 // data size received, so we must reserialize the exact same data
                 OnDiskAtom atom = iter.next();
                 if (atom == null)
@@ -548,11 +548,11 @@ public class SSTableWriter extends SSTable
 
         public void append(DecoratedKey key, RowIndexEntry indexEntry)
         {
-            bf.add(key.key);
+            bf.add(key.getKey());
             long indexPosition = indexFile.getFilePointer();
             try
             {
-                ByteBufferUtil.writeWithShortLength(key.key, indexFile.stream);
+                ByteBufferUtil.writeWithShortLength(key.getKey(), indexFile.stream);
                 metadata.comparator.rowIndexEntrySerializer().serialize(indexEntry, indexFile.stream);
             }
             catch (IOException e)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/repair/RepairMessageVerbHandler.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/RepairMessageVerbHandler.java b/src/java/org/apache/cassandra/repair/RepairMessageVerbHandler.java
index d710652..2dac5cd 100644
--- a/src/java/org/apache/cassandra/repair/RepairMessageVerbHandler.java
+++ b/src/java/org/apache/cassandra/repair/RepairMessageVerbHandler.java
@@ -80,7 +80,7 @@ public class RepairMessageVerbHandler implements IVerbHandler<RepairMessage>
                 {
                     public boolean apply(SSTableReader sstable)
                     {
-                        return sstable != null && new Bounds<>(sstable.first.token, sstable.last.token).intersects(Collections.singleton(repairingRange));
+                        return sstable != null && new Bounds<>(sstable.first.getToken(), sstable.last.getToken()).intersects(Collections.singleton(repairingRange));
                     }
                 });
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/repair/Validator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/repair/Validator.java b/src/java/org/apache/cassandra/repair/Validator.java
index 6263307..d93b4a5 100644
--- a/src/java/org/apache/cassandra/repair/Validator.java
+++ b/src/java/org/apache/cassandra/repair/Validator.java
@@ -97,7 +97,7 @@ public class Validator implements Runnable
             List<DecoratedKey> keys = new ArrayList<>();
             for (DecoratedKey sample : cfs.keySamples(desc.range))
             {
-                assert desc.range.contains(sample.token): "Token " + sample.token + " is not within range " + desc.range;
+                assert desc.range.contains(sample.getToken()): "Token " + sample.getToken() + " is not within range " + desc.range;
                 keys.add(sample);
             }
 
@@ -114,7 +114,7 @@ public class Validator implements Runnable
                 while (true)
                 {
                     DecoratedKey dk = keys.get(random.nextInt(numkeys));
-                    if (!tree.split(dk.token))
+                    if (!tree.split(dk.getToken()))
                         break;
                 }
             }
@@ -131,7 +131,7 @@ public class Validator implements Runnable
      */
     public void add(AbstractCompactedRow row)
     {
-        assert desc.range.contains(row.key.token) : row.key.token + " is not contained in " + desc.range;
+        assert desc.range.contains(row.key.getToken()) : row.key.getToken() + " is not contained in " + desc.range;
         assert lastKey == null || lastKey.compareTo(row.key) < 0
                : "row " + row.key + " received out of order wrt " + lastKey;
         lastKey = row.key;
@@ -140,7 +140,7 @@ public class Validator implements Runnable
             range = ranges.next();
 
         // generate new ranges as long as case 1 is true
-        while (!range.contains(row.key.token))
+        while (!range.contains(row.key.getToken()))
         {
             // add the empty hash, and move to the next range
             range.ensureHashInitialised();
@@ -196,7 +196,7 @@ public class Validator implements Runnable
         // MerkleTree uses XOR internally, so we want lots of output bits here
         CountingDigest digest = new CountingDigest(FBUtilities.newMessageDigest("SHA-256"));
         row.update(digest);
-        return new MerkleTree.RowHash(row.key.token, digest.digest(), digest.count);
+        return new MerkleTree.RowHash(row.key.getToken(), digest.digest(), digest.count);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/service/ActiveRepairService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/ActiveRepairService.java b/src/java/org/apache/cassandra/service/ActiveRepairService.java
index 7a7aac9..b300547 100644
--- a/src/java/org/apache/cassandra/service/ActiveRepairService.java
+++ b/src/java/org/apache/cassandra/service/ActiveRepairService.java
@@ -307,7 +307,7 @@ public class ActiveRepairService
             Set<SSTableReader> sstables = new HashSet<>();
             for (SSTableReader sstable : cfs.getSSTables())
             {
-                if (new Bounds<>(sstable.first.token, sstable.last.token).intersects(ranges))
+                if (new Bounds<>(sstable.first.getToken(), sstable.last.getToken()).intersects(ranges))
                 {
                     if (!sstable.isRepaired())
                     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/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 5422d82..b7d1682 100644
--- a/src/java/org/apache/cassandra/service/RowDataResolver.java
+++ b/src/java/org/apache/cassandra/service/RowDataResolver.java
@@ -116,7 +116,7 @@ public class RowDataResolver extends AbstractRowResolver
                 continue;
 
             // create and send the mutation message based on the diff
-            Mutation mutation = new Mutation(keyspaceName, key.key, diffCf);
+            Mutation mutation = new Mutation(keyspaceName, key.getKey(), diffCf);
             // use a separate verb here because we don't want these to be get the white glove hint-
             // on-timeout behavior that a "real" mutation gets
             results.add(MessagingService.instance().sendRR(mutation.createMessage(MessagingService.Verb.READ_REPAIR),

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java
index a284ab4..25a3670 100644
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@ -2967,7 +2967,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         List<Token> tokens = Lists.newArrayListWithExpectedSize(keys.size() + 2);
         tokens.add(range.left);
         for (DecoratedKey key : keys)
-            tokens.add(key.token);
+            tokens.add(key.getToken());
         tokens.add(range.right);
         return tokens;
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/service/pager/RangeNamesQueryPager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/pager/RangeNamesQueryPager.java b/src/java/org/apache/cassandra/service/pager/RangeNamesQueryPager.java
index e3b0cf8..50d1280 100644
--- a/src/java/org/apache/cassandra/service/pager/RangeNamesQueryPager.java
+++ b/src/java/org/apache/cassandra/service/pager/RangeNamesQueryPager.java
@@ -63,7 +63,7 @@ public class RangeNamesQueryPager extends AbstractQueryPager
     {
         return lastReturnedKey == null
              ? null
-             : new PagingState(lastReturnedKey.key, null, maxRemaining());
+             : new PagingState(lastReturnedKey.getKey(), null, maxRemaining());
     }
 
     protected List<Row> queryNextPage(int pageSize, ConsistencyLevel consistencyLevel, boolean localQuery)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/service/pager/RangeSliceQueryPager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/pager/RangeSliceQueryPager.java b/src/java/org/apache/cassandra/service/pager/RangeSliceQueryPager.java
index b03cc69..cfcd953 100644
--- a/src/java/org/apache/cassandra/service/pager/RangeSliceQueryPager.java
+++ b/src/java/org/apache/cassandra/service/pager/RangeSliceQueryPager.java
@@ -64,7 +64,7 @@ public class RangeSliceQueryPager extends AbstractQueryPager
     {
         return lastReturnedKey == null
              ? null
-             : new PagingState(lastReturnedKey.key, lastReturnedName.toByteBuffer(), maxRemaining());
+             : new PagingState(lastReturnedKey.getKey(), lastReturnedName.toByteBuffer(), maxRemaining());
     }
 
     protected List<Row> queryNextPage(int pageSize, ConsistencyLevel consistencyLevel, boolean localQuery)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/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 b1ad77e..4c5e576 100644
--- a/src/java/org/apache/cassandra/thrift/CassandraServer.java
+++ b/src/java/org/apache/cassandra/thrift/CassandraServer.java
@@ -1167,8 +1167,8 @@ public class CassandraServer implements Cassandra.Iface
             {
                 RowPosition end = range.end_key == null
                                 ? p.getTokenFactory().fromString(range.end_token).maxKeyBound(p)
-                                : RowPosition.forKey(range.end_key, p);
-                bounds = new Bounds<RowPosition>(RowPosition.forKey(range.start_key, p), end);
+                                : RowPosition.ForKey.get(range.end_key, p);
+                bounds = new Bounds<RowPosition>(RowPosition.ForKey.get(range.start_key, p), end);
             }
             long now = System.currentTimeMillis();
             schedule(DatabaseDescriptor.getRangeRpcTimeout());
@@ -1255,8 +1255,8 @@ public class CassandraServer implements Cassandra.Iface
             {
                 RowPosition end = range.end_key == null
                                 ? p.getTokenFactory().fromString(range.end_token).maxKeyBound(p)
-                                : RowPosition.forKey(range.end_key, p);
-                bounds = new Bounds<RowPosition>(RowPosition.forKey(range.start_key, p), end);
+                                : RowPosition.ForKey.get(range.end_key, p);
+                bounds = new Bounds<RowPosition>(RowPosition.ForKey.get(range.start_key, p), end);
             }
 
             if (range.row_filter != null && !range.row_filter.isEmpty())
@@ -1303,7 +1303,7 @@ public class CassandraServer implements Cassandra.Iface
         for (Row row : rows)
         {
             List<ColumnOrSuperColumn> thriftifiedColumns = thriftifyColumnFamily(row.cf, column_parent.super_column != null, reversed, now);
-            keySlices.add(new KeySlice(row.key.key, thriftifiedColumns));
+            keySlices.add(new KeySlice(row.key.getKey(), thriftifiedColumns));
         }
 
         return keySlices;
@@ -1338,7 +1338,7 @@ public class CassandraServer implements Cassandra.Iface
             consistencyLevel.validateForRead(keyspace);
 
             IPartitioner p = StorageService.getPartitioner();
-            AbstractBounds<RowPosition> bounds = new Bounds<RowPosition>(RowPosition.forKey(index_clause.start_key, p),
+            AbstractBounds<RowPosition> bounds = new Bounds<RowPosition>(RowPosition.ForKey.get(index_clause.start_key, p),
                                                                          p.getMinimumToken().minKeyBound());
 
             IDiskAtomFilter filter = ThriftValidation.asIFilter(column_predicate, metadata, column_parent.super_column);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/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 222a904..3b5663b 100644
--- a/src/java/org/apache/cassandra/thrift/ThriftValidation.java
+++ b/src/java/org/apache/cassandra/thrift/ThriftValidation.java
@@ -472,9 +472,9 @@ public class ThriftValidation
     private static Cell asDBColumn(CellName name, Column column)
     {
         if (column.ttl <= 0)
-            return new Cell(name, column.value, column.timestamp);
+            return new BufferCell(name, column.value, column.timestamp);
         else
-            return new ExpiringCell(name, column.value, column.timestamp, column.ttl);
+            return new BufferExpiringCell(name, column.value, column.timestamp, column.ttl);
     }
 
     /**
@@ -534,7 +534,7 @@ public class ThriftValidation
         {
             // start_token/end_token can wrap, but key/token should not
             RowPosition stop = p.getTokenFactory().fromString(range.end_token).maxKeyBound(p);
-            if (RowPosition.forKey(range.start_key, p).compareTo(stop) > 0 && !stop.isMinimum())
+            if (RowPosition.ForKey.get(range.start_key, p).compareTo(stop) > 0 && !stop.isMinimum())
                 throw new org.apache.cassandra.exceptions.InvalidRequestException("Start key's token sorts after end token");
         }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/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 02deb65..41e9fdc 100644
--- a/src/java/org/apache/cassandra/tools/SSTableExport.java
+++ b/src/java/org/apache/cassandra/tools/SSTableExport.java
@@ -188,7 +188,7 @@ public class SSTableExport
     {
         out.print("{");
         writeKey(out, "key");
-        writeJSON(out, bytesToHex(key.key));
+        writeJSON(out, bytesToHex(key.getKey()));
         out.print(",\n");
 
         if (!deletionInfo.isLive())
@@ -238,7 +238,7 @@ public class SSTableExport
                 throw new IOException("Key out of order! " + lastKey + " > " + key);
             lastKey = key;
 
-            outs.println(bytesToHex(key.key));
+            outs.println(bytesToHex(key.getKey()));
             checkStream(outs); // flushes
         }
         iter.close();
@@ -322,7 +322,7 @@ public class SSTableExport
         {
             row = (SSTableIdentityIterator) scanner.next();
 
-            String currentKey = bytesToHex(row.getKey().key);
+            String currentKey = bytesToHex(row.getKey().getKey());
 
             if (excludeSet.contains(currentKey))
                 continue;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/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 6b34f59..4e7bf06 100644
--- a/src/java/org/apache/cassandra/tools/SSTableImport.java
+++ b/src/java/org/apache/cassandra/tools/SSTableImport.java
@@ -220,11 +220,11 @@ public class SSTableImport
 
             if (col.isExpiring())
             {
-                cfamily.addColumn(new ExpiringCell(cname, col.getValue(), col.timestamp, col.ttl, col.localExpirationTime));
+                cfamily.addColumn(new BufferExpiringCell(cname, col.getValue(), col.timestamp, col.ttl, col.localExpirationTime));
             }
             else if (col.isCounter())
             {
-                cfamily.addColumn(new CounterCell(cname, col.getValue(), col.timestamp, col.timestampOfLastDelete));
+                cfamily.addColumn(new BufferCounterCell(cname, col.getValue(), col.timestamp, col.timestampOfLastDelete));
             }
             else if (col.isDeleted())
             {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/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 ad63bd8..6c68b29 100644
--- a/src/java/org/apache/cassandra/tracing/Tracing.java
+++ b/src/java/org/apache/cassandra/tracing/Tracing.java
@@ -93,15 +93,15 @@ public class Tracing
 
     private static void addColumn(ColumnFamily cf, CellName name, ByteBuffer value)
     {
-        cf.addColumn(new ExpiringCell(name, value, System.currentTimeMillis(), TTL));
+        cf.addColumn(new BufferExpiringCell(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 ExpiringCell(buildName(CFMetaData.TraceSessionsCf, "parameters", entry.getKey()),
-                                          bytes(entry.getValue()), System.currentTimeMillis(), TTL));
+            cf.addColumn(new BufferExpiringCell(buildName(CFMetaData.TraceSessionsCf, "parameters", entry.getKey()),
+                                                bytes(entry.getValue()), System.currentTimeMillis(), TTL));
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/utils/FBUtilities.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/FBUtilities.java b/src/java/org/apache/cassandra/utils/FBUtilities.java
index 12c393a..0b89aec 100644
--- a/src/java/org/apache/cassandra/utils/FBUtilities.java
+++ b/src/java/org/apache/cassandra/utils/FBUtilities.java
@@ -313,8 +313,8 @@ public class FBUtilities
             {
                 public int compare(DecoratedKey o1, DecoratedKey o2)
                 {
-                    if ((right.compareTo(o1.token) < 0 && right.compareTo(o2.token) < 0)
-                        || (right.compareTo(o1.token) > 0 && right.compareTo(o2.token) > 0))
+                    if ((right.compareTo(o1.getToken()) < 0 && right.compareTo(o2.getToken()) < 0)
+                        || (right.compareTo(o1.getToken()) > 0 && right.compareTo(o2.getToken()) > 0))
                     {
                         // both tokens are on the same side of the wrap point
                         return o1.compareTo(o2);
@@ -693,4 +693,35 @@ public class FBUtilities
     {
         return OPERATING_SYSTEM.contains("nix") || OPERATING_SYSTEM.contains("nux") || OPERATING_SYSTEM.contains("aix");
     }
+
+    public static void updateWithShort(MessageDigest digest, int val)
+    {
+        digest.update((byte) ((val >> 8) & 0xFF));
+        digest.update((byte) (val & 0xFF));
+    }
+
+    public static void updateWithByte(MessageDigest digest, int val)
+    {
+        digest.update((byte) (val & 0xFF));
+    }
+
+    public static void updateWithInt(MessageDigest digest, int val)
+    {
+        digest.update((byte) ((val >>> 24) & 0xFF));
+        digest.update((byte) ((val >>> 16) & 0xFF));
+        digest.update((byte) ((val >>>  8) & 0xFF));
+        digest.update((byte) ((val >>> 0) & 0xFF));
+    }
+
+    public static void updateWithLong(MessageDigest digest, long val)
+    {
+        digest.update((byte) ((val >>> 56) & 0xFF));
+        digest.update((byte) ((val >>> 48) & 0xFF));
+        digest.update((byte) ((val >>> 40) & 0xFF));
+        digest.update((byte) ((val >>> 32) & 0xFF));
+        digest.update((byte) ((val >>> 24) & 0xFF));
+        digest.update((byte) ((val >>> 16) & 0xFF));
+        digest.update((byte) ((val >>>  8) & 0xFF));
+        digest.update((byte)  ((val >>> 0) & 0xFF));
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/utils/PureJavaCrc32.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/PureJavaCrc32.java b/src/java/org/apache/cassandra/utils/PureJavaCrc32.java
index 1cdefd7..15fe0c6 100644
--- a/src/java/org/apache/cassandra/utils/PureJavaCrc32.java
+++ b/src/java/org/apache/cassandra/utils/PureJavaCrc32.java
@@ -95,7 +95,38 @@ public class PureJavaCrc32 implements Checksum {
     crc = localCrc;
   }
 
-    public void update(ByteBuffer b, int off, int len) {
+  private static final ThreadLocal<byte[]> BUFFER = new ThreadLocal<byte[]>()
+  {
+      protected byte[] initialValue()
+      {
+          return new byte[256];
+      }
+  };
+
+    public void update(ByteBuffer b, int off, int len)
+    {
+        if (b.hasArray())
+        {
+            update(b.array(), b.arrayOffset() + off, len);
+        }
+        else if (len < 16)
+        {
+            doUpdate(b, off, len);
+        }
+        else
+        {
+            byte[] buf = BUFFER.get();
+            while (len > 0)
+            {
+                int l = Math.min(len, buf.length);
+                ByteBufferUtil.arrayCopy(b, off, buf, 0, l);
+                update(buf, 0, l);
+                len -= l;
+            }
+        }
+    }
+
+    private void doUpdate(ByteBuffer b, int off, int len) {
         int localCrc = crc;
 
         while(len > 7) {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/utils/memory/ContextAllocator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/memory/ContextAllocator.java b/src/java/org/apache/cassandra/utils/memory/ContextAllocator.java
index a2b1446..62c8f9c 100644
--- a/src/java/org/apache/cassandra/utils/memory/ContextAllocator.java
+++ b/src/java/org/apache/cassandra/utils/memory/ContextAllocator.java
@@ -20,21 +20,19 @@ package org.apache.cassandra.utils.memory;
 
 import java.nio.ByteBuffer;
 
-import org.apache.cassandra.db.Cell;
-import com.google.common.base.Function;
-
+import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.concurrent.OpOrder;
 
 /**
  * Wraps calls to a PoolAllocator with the provided writeOp. Also doubles as a Function that clones Cells
  * using itself
  */
-public final class ContextAllocator extends AbstractAllocator implements Function<Cell, Cell>
+public final class ContextAllocator extends AbstractAllocator
 {
     private final OpOrder.Group opGroup;
-    private final PoolAllocator allocator;
+    private final MemtableBufferAllocator allocator;
 
-    public ContextAllocator(OpOrder.Group opGroup, PoolAllocator allocator)
+    public ContextAllocator(OpOrder.Group opGroup, MemtableBufferAllocator allocator)
     {
         this.opGroup = opGroup;
         this.allocator = allocator;
@@ -43,16 +41,19 @@ public final class ContextAllocator extends AbstractAllocator implements Functio
     @Override
     public ByteBuffer clone(ByteBuffer buffer)
     {
-        return allocator.clone(buffer, opGroup);
+        assert buffer != null;
+        if (buffer.remaining() == 0)
+            return ByteBufferUtil.EMPTY_BYTE_BUFFER;
+        ByteBuffer cloned = allocate(buffer.remaining());
+
+        cloned.mark();
+        cloned.put(buffer.duplicate());
+        cloned.reset();
+        return cloned;
     }
 
     public ByteBuffer allocate(int size)
     {
         return allocator.allocate(size, opGroup);
     }
-
-    public Cell apply(Cell column)
-    {
-        return column.localCopy(this);
-    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/utils/memory/HeapPool.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/memory/HeapPool.java b/src/java/org/apache/cassandra/utils/memory/HeapPool.java
index 9d08698..a04947c 100644
--- a/src/java/org/apache/cassandra/utils/memory/HeapPool.java
+++ b/src/java/org/apache/cassandra/utils/memory/HeapPool.java
@@ -19,10 +19,14 @@
 package org.apache.cassandra.utils.memory;
 
 import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
 
+import org.apache.cassandra.db.Cell;
+import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.utils.concurrent.OpOrder;
 
-public class HeapPool extends Pool
+public class HeapPool extends MemtablePool
 {
     public HeapPool(long maxOnHeapMemory, float cleanupThreshold, Runnable cleaner)
     {
@@ -34,33 +38,65 @@ public class HeapPool extends Pool
         return false;
     }
 
-    public Allocator newAllocator()
+    public MemtableAllocator newAllocator()
     {
         return new Allocator(this);
     }
 
-    public final class Allocator extends PoolAllocator
+    public class Allocator extends MemtableBufferAllocator
     {
         Allocator(HeapPool pool)
         {
             super(pool.onHeap.newAllocator(), pool.offHeap.newAllocator());
         }
 
-        public ByteBuffer allocate(int size)
+        public ByteBuffer allocate(int size, OpOrder.Group opGroup)
         {
-            return allocate(size, null);
+            super.onHeap().allocate(size, opGroup);
+            return ByteBuffer.allocate(size);
         }
 
-        public ByteBuffer allocate(int size, OpOrder.Group opGroup)
+        public DataReclaimer reclaimer()
         {
-            onHeap().allocate(size, opGroup);
-            // must loop trying to acquire
-            return ByteBuffer.allocate(size);
+            return new Reclaimer();
         }
 
-        public void free(ByteBuffer buffer)
+        private class Reclaimer implements DataReclaimer
         {
-            onHeap().release(buffer.remaining());
+            List<Cell> delayed;
+
+            public Reclaimer reclaim(Cell cell)
+            {
+                if (delayed == null)
+                    delayed = new ArrayList<>();
+                delayed.add(cell);
+                return this;
+            }
+
+            public Reclaimer reclaimImmediately(Cell cell)
+            {
+                onHeap().release(cell.name().dataSize() + cell.value().remaining());
+                return this;
+            }
+
+            public Reclaimer reclaimImmediately(DecoratedKey key)
+            {
+                onHeap().release(key.getKey().remaining());
+                return this;
+            }
+
+            public void cancel()
+            {
+                if (delayed != null)
+                    delayed.clear();
+            }
+
+            public void commit()
+            {
+                if (delayed != null)
+                    for (Cell cell : delayed)
+                        reclaimImmediately(cell);
+            }
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/utils/memory/MemoryUtil.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/memory/MemoryUtil.java b/src/java/org/apache/cassandra/utils/memory/MemoryUtil.java
new file mode 100644
index 0000000..5f7d410
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/memory/MemoryUtil.java
@@ -0,0 +1,305 @@
+/*
+ * 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.utils.memory;
+
+import java.lang.reflect.Field;
+import java.nio.Buffer;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+
+import sun.misc.Unsafe;
+
+public abstract class MemoryUtil
+{
+    private static final long UNSAFE_COPY_THRESHOLD = 1024 * 1024L; // copied from java.nio.Bits
+
+    private static final Unsafe unsafe;
+    private static final Class<?> DIRECT_BYTE_BUFFER_CLASS;
+    private static final long DIRECT_BYTE_BUFFER_ADDRESS_OFFSET;
+    private static final long DIRECT_BYTE_BUFFER_CAPACITY_OFFSET;
+    private static final long DIRECT_BYTE_BUFFER_LIMIT_OFFSET;
+    private static final long BYTE_ARRAY_BASE_OFFSET;
+
+    private static final boolean BIG_ENDIAN = ByteOrder.nativeOrder().equals(ByteOrder.BIG_ENDIAN);
+
+    private static final boolean UNALIGNED;
+    public static final boolean INVERTED_ORDER;
+
+    static
+    {
+        String arch = System.getProperty("os.arch");
+        UNALIGNED = arch.equals("i386") || arch.equals("x86")
+                || arch.equals("amd64") || arch.equals("x86_64");
+        INVERTED_ORDER = UNALIGNED && !BIG_ENDIAN;
+        try
+        {
+            Field field = sun.misc.Unsafe.class.getDeclaredField("theUnsafe");
+            field.setAccessible(true);
+            unsafe = (sun.misc.Unsafe) field.get(null);
+            Class<?> clazz = ByteBuffer.allocateDirect(0).getClass();
+            DIRECT_BYTE_BUFFER_ADDRESS_OFFSET = unsafe.objectFieldOffset(Buffer.class.getDeclaredField("address"));
+            DIRECT_BYTE_BUFFER_CAPACITY_OFFSET = unsafe.objectFieldOffset(Buffer.class.getDeclaredField("capacity"));
+            DIRECT_BYTE_BUFFER_LIMIT_OFFSET = unsafe.objectFieldOffset(Buffer.class.getDeclaredField("limit"));
+            DIRECT_BYTE_BUFFER_CLASS = clazz;
+            BYTE_ARRAY_BASE_OFFSET = unsafe.arrayBaseOffset(byte[].class);
+        }
+        catch (Exception e)
+        {
+            throw new AssertionError(e);
+        }
+    }
+
+    public static void setByte(long address, byte b)
+    {
+        unsafe.putByte(address, b);
+    }
+
+    public static void setShort(long address, short s)
+    {
+        unsafe.putShort(address, s);
+    }
+
+    public static void setInt(long address, int l)
+    {
+        if (UNALIGNED)
+            unsafe.putInt(address, l);
+        else
+            putIntByByte(address, l);
+    }
+
+    public static void setLong(long address, long l)
+    {
+        if (UNALIGNED)
+            unsafe.putLong(address, l);
+        else
+            putLongByByte(address, l);
+    }
+
+    public static byte getByte(long address)
+    {
+        return unsafe.getByte(address);
+    }
+
+    public static int getShort(long address)
+    {
+        return UNALIGNED ? unsafe.getShort(address) : getShortByByte(address);
+    }
+
+    public static int getInt(long address)
+    {
+        return UNALIGNED ? unsafe.getInt(address) : getIntByByte(address);
+    }
+
+    public static long getLong(long address)
+    {
+        return UNALIGNED ? unsafe.getLong(address) : getLongByByte(address);
+    }
+
+    public static ByteBuffer getByteBuffer(long address, int length)
+    {
+        ByteBuffer instance;
+        try
+        {
+            instance = (ByteBuffer) unsafe.allocateInstance(DIRECT_BYTE_BUFFER_CLASS);
+        }
+        catch (InstantiationException e)
+        {
+            throw new AssertionError(e);
+        }
+
+        unsafe.putLong(instance, DIRECT_BYTE_BUFFER_ADDRESS_OFFSET, address);
+        unsafe.putInt(instance, DIRECT_BYTE_BUFFER_CAPACITY_OFFSET, length);
+        unsafe.putInt(instance, DIRECT_BYTE_BUFFER_LIMIT_OFFSET, length);
+        return instance;
+    }
+
+    public static long getLongByByte(long address)
+    {
+        if (BIG_ENDIAN)
+        {
+            return  (((long) unsafe.getByte(address    )       ) << 56) |
+                    (((long) unsafe.getByte(address + 1) & 0xff) << 48) |
+                    (((long) unsafe.getByte(address + 2) & 0xff) << 40) |
+                    (((long) unsafe.getByte(address + 3) & 0xff) << 32) |
+                    (((long) unsafe.getByte(address + 4) & 0xff) << 24) |
+                    (((long) unsafe.getByte(address + 5) & 0xff) << 16) |
+                    (((long) unsafe.getByte(address + 6) & 0xff) <<  8) |
+                    (((long) unsafe.getByte(address + 7) & 0xff)      );
+        }
+        else
+        {
+            return  (((long) unsafe.getByte(address + 7)       ) << 56) |
+                    (((long) unsafe.getByte(address + 6) & 0xff) << 48) |
+                    (((long) unsafe.getByte(address + 5) & 0xff) << 40) |
+                    (((long) unsafe.getByte(address + 4) & 0xff) << 32) |
+                    (((long) unsafe.getByte(address + 3) & 0xff) << 24) |
+                    (((long) unsafe.getByte(address + 2) & 0xff) << 16) |
+                    (((long) unsafe.getByte(address + 1) & 0xff) <<  8) |
+                    (((long) unsafe.getByte(address    ) & 0xff)      );
+        }
+    }
+
+    public static int getIntByByte(long address)
+    {
+        if (BIG_ENDIAN)
+        {
+            return  (((int) unsafe.getByte(address    )       ) << 24) |
+                    (((int) unsafe.getByte(address + 1) & 0xff) << 16) |
+                    (((int) unsafe.getByte(address + 2) & 0xff) << 8 ) |
+                    (((int) unsafe.getByte(address + 3) & 0xff)      );
+        }
+        else
+        {
+            return  (((int) unsafe.getByte(address + 3)       ) << 24) |
+                    (((int) unsafe.getByte(address + 2) & 0xff) << 16) |
+                    (((int) unsafe.getByte(address + 1) & 0xff) <<  8) |
+                    (((int) unsafe.getByte(address    ) & 0xff)      );
+        }
+    }
+
+
+    public static int getShortByByte(long address)
+    {
+        if (BIG_ENDIAN)
+        {
+            return  (((int) unsafe.getByte(address    )       ) << 8) |
+                    (((int) unsafe.getByte(address + 1) & 0xff)     );
+        }
+        else
+        {
+            return  (((int) unsafe.getByte(address + 1)       ) <<  8) |
+                    (((int) unsafe.getByte(address    ) & 0xff)      );
+        }
+    }
+
+    public static void putLongByByte(long address, long value)
+    {
+        if (BIG_ENDIAN)
+        {
+            unsafe.putByte(address, (byte) (value >> 56));
+            unsafe.putByte(address + 1, (byte) (value >> 48));
+            unsafe.putByte(address + 2, (byte) (value >> 40));
+            unsafe.putByte(address + 3, (byte) (value >> 32));
+            unsafe.putByte(address + 4, (byte) (value >> 24));
+            unsafe.putByte(address + 5, (byte) (value >> 16));
+            unsafe.putByte(address + 6, (byte) (value >> 8));
+            unsafe.putByte(address + 7, (byte) (value));
+        }
+        else
+        {
+            unsafe.putByte(address + 7, (byte) (value >> 56));
+            unsafe.putByte(address + 6, (byte) (value >> 48));
+            unsafe.putByte(address + 5, (byte) (value >> 40));
+            unsafe.putByte(address + 4, (byte) (value >> 32));
+            unsafe.putByte(address + 3, (byte) (value >> 24));
+            unsafe.putByte(address + 2, (byte) (value >> 16));
+            unsafe.putByte(address + 1, (byte) (value >> 8));
+            unsafe.putByte(address, (byte) (value));
+        }
+    }
+
+    public static void putIntByByte(long address, int value)
+    {
+        if (BIG_ENDIAN)
+        {
+            unsafe.putByte(address, (byte) (value >> 24));
+            unsafe.putByte(address + 1, (byte) (value >> 16));
+            unsafe.putByte(address + 2, (byte) (value >> 8));
+            unsafe.putByte(address + 3, (byte) (value));
+        }
+        else
+        {
+            unsafe.putByte(address + 3, (byte) (value >> 24));
+            unsafe.putByte(address + 2, (byte) (value >> 16));
+            unsafe.putByte(address + 1, (byte) (value >> 8));
+            unsafe.putByte(address, (byte) (value));
+        }
+    }
+
+    public static void setBytes(long address, ByteBuffer buffer)
+    {
+        int start = buffer.position();
+        int count = buffer.limit() - start;
+        if (count == 0)
+            return;
+
+        if (buffer.isDirect())
+            setBytes(unsafe.getLong(buffer, DIRECT_BYTE_BUFFER_ADDRESS_OFFSET) + start, address, count);
+        else
+            setBytes(address, buffer.array(), start, count);
+    }
+
+    /**
+     * Transfers count bytes from buffer to Memory
+     *
+     * @param address start offset in the memory
+     * @param buffer the data buffer
+     * @param bufferOffset start offset of the buffer
+     * @param count number of bytes to transfer
+     */
+    public static void setBytes(long address, byte[] buffer, int bufferOffset, int count)
+    {
+        assert buffer != null;
+        assert !(bufferOffset < 0 || count < 0 || bufferOffset + count > buffer.length);
+        setBytes(buffer, bufferOffset, address, count);
+    }
+
+    public static void setBytes(long src, long trg, long count)
+    {
+        while (count > 0)
+        {
+            long size = (count> UNSAFE_COPY_THRESHOLD) ? UNSAFE_COPY_THRESHOLD : count;
+            unsafe.copyMemory(src, trg, size);
+            count -= size;
+            src += size;
+            trg+= size;
+        }
+    }
+
+    public static void setBytes(byte[] src, int offset, long trg, long count)
+    {
+        while (count > 0)
+        {
+            long size = (count> UNSAFE_COPY_THRESHOLD) ? UNSAFE_COPY_THRESHOLD : count;
+            unsafe.copyMemory(src, BYTE_ARRAY_BASE_OFFSET + offset, null, trg, size);
+            count -= size;
+            offset += size;
+            trg += size;
+        }
+    }
+
+    /**
+     * Transfers count bytes from Memory starting at memoryOffset to buffer starting at bufferOffset
+     *
+     * @param address start offset in the memory
+     * @param buffer the data buffer
+     * @param bufferOffset start offset of the buffer
+     * @param count number of bytes to transfer
+     */
+    public static void getBytes(long address, byte[] buffer, int bufferOffset, int count)
+    {
+        if (buffer == null)
+            throw new NullPointerException();
+        else if (bufferOffset < 0 || count < 0 || count > buffer.length - bufferOffset)
+            throw new IndexOutOfBoundsException();
+        else if (count == 0)
+            return;
+
+        unsafe.copyMemory(null, address, buffer, BYTE_ARRAY_BASE_OFFSET + bufferOffset, count);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/utils/memory/MemtableAllocator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/memory/MemtableAllocator.java b/src/java/org/apache/cassandra/utils/memory/MemtableAllocator.java
new file mode 100644
index 0000000..e814b4d
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/memory/MemtableAllocator.java
@@ -0,0 +1,245 @@
+/*
+ * 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.utils.memory;
+
+import java.util.concurrent.atomic.AtomicLongFieldUpdater;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.db.*;
+import org.apache.cassandra.utils.concurrent.OpOrder;
+import org.apache.cassandra.utils.concurrent.WaitQueue;
+
+public abstract class MemtableAllocator
+{
+    private final SubAllocator onHeap;
+    private final SubAllocator offHeap;
+    volatile LifeCycle state = LifeCycle.LIVE;
+
+    static enum LifeCycle
+    {
+        LIVE, DISCARDING, DISCARDED;
+        LifeCycle transition(LifeCycle targetState)
+        {
+            switch (targetState)
+            {
+                case DISCARDING:
+                    assert this == LifeCycle.LIVE;
+                    return LifeCycle.DISCARDING;
+
+                case DISCARDED:
+                    assert this == LifeCycle.DISCARDING;
+                    return LifeCycle.DISCARDED;
+
+                default:
+                    throw new IllegalStateException();
+            }
+        }
+    }
+
+    MemtableAllocator(SubAllocator onHeap, SubAllocator offHeap)
+    {
+        this.onHeap = onHeap;
+        this.offHeap = offHeap;
+    }
+
+    public abstract Cell clone(Cell cell, CFMetaData cfm, OpOrder.Group writeOp);
+    public abstract CounterCell clone(CounterCell cell, CFMetaData cfm, OpOrder.Group writeOp);
+    public abstract DeletedCell clone(DeletedCell cell, CFMetaData cfm, OpOrder.Group writeOp);
+    public abstract ExpiringCell clone(ExpiringCell cell, CFMetaData cfm, OpOrder.Group writeOp);
+    public abstract DecoratedKey clone(DecoratedKey key, OpOrder.Group opGroup);
+    public abstract DataReclaimer reclaimer();
+
+    public SubAllocator onHeap()
+    {
+        return onHeap;
+    }
+
+    public SubAllocator offHeap()
+    {
+        return offHeap;
+    }
+
+    /**
+     * Mark this allocator reclaiming; this will permit any outstanding allocations to temporarily
+     * overshoot the maximum memory limit so that flushing can begin immediately
+     */
+    public void setDiscarding()
+    {
+        state = state.transition(LifeCycle.DISCARDING);
+        // mark the memory owned by this allocator as reclaiming
+        onHeap.markAllReclaiming();
+        offHeap.markAllReclaiming();
+    }
+
+    /**
+     * Indicate the memory and resources owned by this allocator are no longer referenced,
+     * and can be reclaimed/reused.
+     */
+    public void setDiscarded()
+    {
+        state = state.transition(LifeCycle.DISCARDED);
+        // release any memory owned by this allocator; automatically signals waiters
+        onHeap.releaseAll();
+        offHeap.releaseAll();
+    }
+
+    public boolean isLive()
+    {
+        return state == LifeCycle.LIVE;
+    }
+
+    public static interface DataReclaimer
+    {
+        public DataReclaimer reclaim(Cell cell);
+        public DataReclaimer reclaimImmediately(Cell cell);
+        public DataReclaimer reclaimImmediately(DecoratedKey key);
+        public void cancel();
+        public void commit();
+    }
+
+    public static final DataReclaimer NO_OP = new DataReclaimer()
+    {
+        public DataReclaimer reclaim(Cell cell)
+        {
+            return this;
+        }
+
+        public DataReclaimer reclaimImmediately(Cell cell)
+        {
+            return this;
+        }
+
+        public DataReclaimer reclaimImmediately(DecoratedKey key)
+        {
+            return this;
+        }
+
+        @Override
+        public void cancel()
+        {}
+
+        @Override
+        public void commit()
+        {}
+    };
+
+    /** Mark the BB as unused, permitting it to be reclaimed */
+    public static final class SubAllocator
+    {
+        // the tracker we are owning memory from
+        private final MemtablePool.SubPool parent;
+
+        // the amount of memory/resource owned by this object
+        private volatile long owns;
+        // the amount of memory we are reporting to collect; this may be inaccurate, but is close
+        // and is used only to ensure that once we have reclaimed we mark the tracker with the same amount
+        private volatile long reclaiming;
+
+        SubAllocator(MemtablePool.SubPool parent)
+        {
+            this.parent = parent;
+        }
+
+        // should only be called once we know we will never allocate to the object again.
+        // currently no corroboration/enforcement of this is performed.
+        void releaseAll()
+        {
+            parent.adjustAcquired(-ownsUpdater.getAndSet(this, 0), false);
+            parent.adjustReclaiming(-reclaimingUpdater.getAndSet(this, 0));
+        }
+
+        // allocate memory in the tracker, and mark ourselves as owning it
+        public void allocate(long size, OpOrder.Group opGroup)
+        {
+            while (true)
+            {
+                if (parent.tryAllocate(size))
+                {
+                    acquired(size);
+                    return;
+                }
+                WaitQueue.Signal signal = opGroup.isBlockingSignal(parent.hasRoom().register());
+                boolean allocated = parent.tryAllocate(size);
+                if (allocated || opGroup.isBlocking())
+                {
+                    signal.cancel();
+                    if (allocated) // if we allocated, take ownership
+                        acquired(size);
+                    else // otherwise we're blocking so we're permitted to overshoot our constraints, to just allocate without blocking
+                        allocated(size);
+                    return;
+                }
+                else
+                    signal.awaitUninterruptibly();
+            }
+        }
+
+        // retroactively mark an amount allocated amd acquired in the tracker, and owned by us
+        void allocated(long size)
+        {
+            parent.adjustAcquired(size, true);
+            ownsUpdater.addAndGet(this, size);
+        }
+
+        // retroactively mark an amount acquired in the tracker, and owned by us
+        void acquired(long size)
+        {
+            parent.adjustAcquired(size, false);
+            ownsUpdater.addAndGet(this, size);
+        }
+
+        void release(long size)
+        {
+            parent.adjustAcquired(-size, false);
+            ownsUpdater.addAndGet(this, -size);
+        }
+
+        // mark everything we currently own as reclaiming, both here and in our parent
+        void markAllReclaiming()
+        {
+            while (true)
+            {
+                long cur = owns;
+                long prev = reclaiming;
+                if (reclaimingUpdater.compareAndSet(this, prev, cur))
+                {
+                    parent.adjustReclaiming(cur - prev);
+                    return;
+                }
+            }
+        }
+
+        public long owns()
+        {
+            return owns;
+        }
+
+        public float ownershipRatio()
+        {
+            float r = owns / (float) parent.limit;
+            if (Float.isNaN(r))
+                return 0;
+            return r;
+        }
+
+        private static final AtomicLongFieldUpdater<SubAllocator> ownsUpdater = AtomicLongFieldUpdater.newUpdater(SubAllocator.class, "owns");
+        private static final AtomicLongFieldUpdater<SubAllocator> reclaimingUpdater = AtomicLongFieldUpdater.newUpdater(SubAllocator.class, "reclaiming");
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/utils/memory/MemtableBufferAllocator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/memory/MemtableBufferAllocator.java b/src/java/org/apache/cassandra/utils/memory/MemtableBufferAllocator.java
new file mode 100644
index 0000000..7034d76
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/memory/MemtableBufferAllocator.java
@@ -0,0 +1,70 @@
+/*
+ * 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.utils.memory;
+
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.db.BufferDecoratedKey;
+import org.apache.cassandra.db.Cell;
+import org.apache.cassandra.db.CounterCell;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.DeletedCell;
+import org.apache.cassandra.db.ExpiringCell;
+import org.apache.cassandra.utils.concurrent.OpOrder;
+
+public abstract class MemtableBufferAllocator extends MemtableAllocator
+{
+
+    protected MemtableBufferAllocator(SubAllocator onHeap, SubAllocator offHeap)
+    {
+        super(onHeap, offHeap);
+    }
+
+    public Cell clone(Cell cell, CFMetaData cfm, OpOrder.Group writeOp)
+    {
+        return cell.localCopy(cfm, allocator(writeOp));
+    }
+
+    public CounterCell clone(CounterCell cell, CFMetaData cfm, OpOrder.Group writeOp)
+    {
+        return cell.localCopy(cfm, allocator(writeOp));
+    }
+
+    public DeletedCell clone(DeletedCell cell, CFMetaData cfm, OpOrder.Group writeOp)
+    {
+        return cell.localCopy(cfm, allocator(writeOp));
+    }
+
+    public ExpiringCell clone(ExpiringCell cell, CFMetaData cfm, OpOrder.Group writeOp)
+    {
+        return cell.localCopy(cfm, allocator(writeOp));
+    }
+
+    public DecoratedKey clone(DecoratedKey key, OpOrder.Group writeOp)
+    {
+        return new BufferDecoratedKey(key.getToken(), allocator(writeOp).clone(key.getKey()));
+    }
+
+    public abstract ByteBuffer allocate(int size, OpOrder.Group opGroup);
+
+    protected AbstractAllocator allocator(OpOrder.Group writeOp)
+    {
+        return new ContextAllocator(writeOp, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/utils/memory/MemtableCleanerThread.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/memory/MemtableCleanerThread.java b/src/java/org/apache/cassandra/utils/memory/MemtableCleanerThread.java
new file mode 100644
index 0000000..9950e49
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/memory/MemtableCleanerThread.java
@@ -0,0 +1,73 @@
+/*
+ * 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.utils.memory;
+
+import org.apache.cassandra.utils.concurrent.WaitQueue;
+
+/**
+ * A thread that reclaims memory from a MemtablePool on demand.  The actual reclaiming work is delegated to the
+ * cleaner Runnable, e.g., FlushLargestColumnFamily
+ */
+class MemtableCleanerThread<P extends MemtablePool> extends Thread
+{
+    /** The pool we're cleaning */
+    final P pool;
+
+    /** should ensure that at least some memory has been marked reclaiming after completion */
+    final Runnable cleaner;
+
+    /** signalled whenever needsCleaning() may return true */
+    final WaitQueue wait = new WaitQueue();
+
+    MemtableCleanerThread(P pool, Runnable cleaner)
+    {
+        super(pool.getClass().getSimpleName() + "Cleaner");
+        this.pool = pool;
+        this.cleaner = cleaner;
+    }
+
+    boolean needsCleaning()
+    {
+        return pool.onHeap.needsCleaning() || pool.offHeap.needsCleaning();
+    }
+
+    // should ONLY be called when we really think it already needs cleaning
+    void trigger()
+    {
+        wait.signal();
+    }
+
+    @Override
+    public void run()
+    {
+        while (true)
+        {
+            while (!needsCleaning())
+            {
+                final WaitQueue.Signal signal = wait.register();
+                if (!needsCleaning())
+                    signal.awaitUninterruptibly();
+                else
+                    signal.cancel();
+            }
+
+            cleaner.run();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/utils/memory/MemtablePool.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/memory/MemtablePool.java b/src/java/org/apache/cassandra/utils/memory/MemtablePool.java
new file mode 100644
index 0000000..1d219bb
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/memory/MemtablePool.java
@@ -0,0 +1,199 @@
+/*
+ * 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.utils.memory;
+
+import java.util.concurrent.atomic.AtomicLongFieldUpdater;
+
+import org.apache.cassandra.utils.concurrent.WaitQueue;
+
+
+/**
+ * Represents an amount of memory used for a given purpose, that can be allocated to specific tasks through
+ * child MemtableAllocator objects.
+ */
+public abstract class MemtablePool
+{
+    final MemtableCleanerThread<?> cleaner;
+
+    // the total memory used by this pool
+    public final SubPool onHeap;
+    public final SubPool offHeap;
+
+    final WaitQueue hasRoom = new WaitQueue();
+
+    MemtablePool(long maxOnHeapMemory, long maxOffHeapMemory, float cleanThreshold, Runnable cleaner)
+    {
+        this.onHeap = getSubPool(maxOnHeapMemory, cleanThreshold);
+        this.offHeap = getSubPool(maxOffHeapMemory, cleanThreshold);
+        this.cleaner = getCleaner(cleaner);
+        if (this.cleaner != null)
+            this.cleaner.start();
+    }
+
+    SubPool getSubPool(long limit, float cleanThreshold)
+    {
+        return new SubPool(limit, cleanThreshold);
+    }
+
+    MemtableCleanerThread<?> getCleaner(Runnable cleaner)
+    {
+        return cleaner == null ? null : new MemtableCleanerThread<>(this, cleaner);
+    }
+
+    public abstract boolean needToCopyOnHeap();
+    public abstract MemtableAllocator newAllocator();
+
+    /**
+     * Note the difference between acquire() and allocate(); allocate() makes more resources available to all owners,
+     * and acquire() makes shared resources unavailable but still recorded. An Owner must always acquire resources,
+     * but only needs to allocate if there are none already available. This distinction is not always meaningful.
+     */
+    public class SubPool
+    {
+
+        // total memory/resource permitted to allocate
+        public final long limit;
+
+        // ratio of used to spare (both excluding 'reclaiming') at which to trigger a clean
+        public final float cleanThreshold;
+
+        // total bytes allocated and reclaiming
+        volatile long allocated;
+        volatile long reclaiming;
+
+        // a cache of the calculation determining at what allocation threshold we should next clean
+        volatile long nextClean;
+
+        public SubPool(long limit, float cleanThreshold)
+        {
+            this.limit = limit;
+            this.cleanThreshold = cleanThreshold;
+        }
+
+        /** Methods for tracking and triggering a clean **/
+
+        boolean needsCleaning()
+        {
+            // use strictly-greater-than so we don't clean when limit is 0
+            return used() > nextClean && updateNextClean();
+        }
+
+        void maybeClean()
+        {
+            if (needsCleaning() && cleaner != null)
+                cleaner.trigger();
+        }
+
+        private boolean updateNextClean()
+        {
+            while (true)
+            {
+                long current = nextClean;
+                long reclaiming = this.reclaiming;
+                long next =  reclaiming + (long) (this.limit * cleanThreshold);
+                if (current == next || nextCleanUpdater.compareAndSet(this, current, next))
+                    return used() > next;
+            }
+        }
+
+        /** Methods to allocate space **/
+
+        boolean tryAllocate(long size)
+        {
+            while (true)
+            {
+                long cur;
+                if ((cur = allocated) + size > limit)
+                    return false;
+                if (allocatedUpdater.compareAndSet(this, cur, cur + size))
+                    return true;
+            }
+        }
+
+        /**
+         * apply the size adjustment to allocated, bypassing any limits or constraints. If this reduces the
+         * allocated total, we will signal waiters
+         */
+        void adjustAllocated(long size)
+        {
+            if (size == 0)
+                return;
+            while (true)
+            {
+                long cur = allocated;
+                if (allocatedUpdater.compareAndSet(this, cur, cur + size))
+                    return;
+            }
+        }
+
+        // 'acquires' an amount of memory, and maybe also marks it allocated. This method is meant to be overridden
+        // by implementations with a separate concept of acquired/allocated. As this method stands, an acquire
+        // without an allocate is a no-op (acquisition is achieved through allocation), however a release (where size < 0)
+        // is always processed and accounted for in allocated.
+        void adjustAcquired(long size, boolean alsoAllocated)
+        {
+            if (size > 0 || alsoAllocated)
+            {
+                if (alsoAllocated)
+                    adjustAllocated(size);
+                maybeClean();
+            }
+            else if (size < 0)
+            {
+                adjustAllocated(size);
+                hasRoom.signalAll();
+            }
+        }
+
+        // space reclaimed should be released prior to calling this, to avoid triggering unnecessary cleans
+        void adjustReclaiming(long reclaiming)
+        {
+            if (reclaiming == 0)
+                return;
+            reclaimingUpdater.addAndGet(this, reclaiming);
+            if (reclaiming < 0 && updateNextClean() && cleaner != null)
+                cleaner.trigger();
+        }
+
+        public long allocated()
+        {
+            return allocated;
+        }
+
+        public long used()
+        {
+            return allocated;
+        }
+
+        public MemtableAllocator.SubAllocator newAllocator()
+        {
+            return new MemtableAllocator.SubAllocator(this);
+        }
+
+        public WaitQueue hasRoom()
+        {
+            return hasRoom;
+        }
+    }
+
+    private static final AtomicLongFieldUpdater<SubPool> reclaimingUpdater = AtomicLongFieldUpdater.newUpdater(SubPool.class, "reclaiming");
+    private static final AtomicLongFieldUpdater<SubPool> allocatedUpdater = AtomicLongFieldUpdater.newUpdater(SubPool.class, "allocated");
+    private static final AtomicLongFieldUpdater<SubPool> nextCleanUpdater = AtomicLongFieldUpdater.newUpdater(SubPool.class, "nextClean");
+
+}


[2/7] Push more of memtable data off-heap

Posted by sl...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/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 0ef3333..12e6d9f 100644
--- a/test/unit/org/apache/cassandra/db/KeyspaceTest.java
+++ b/test/unit/org/apache/cassandra/db/KeyspaceTest.java
@@ -64,7 +64,7 @@ public class KeyspaceTest extends SchemaLoader
 
         ColumnFamily cf = ArrayBackedSortedColumns.factory.create("Keyspace2", "Standard3");
         cf.addColumn(column("col1","val1", 1L));
-        Mutation rm = new Mutation("Keyspace2", TEST_KEY.key, cf);
+        Mutation rm = new Mutation("Keyspace2", TEST_KEY.getKey(), cf);
         rm.apply();
 
         Runnable verify = new WrappedRunnable()
@@ -96,7 +96,7 @@ public class KeyspaceTest extends SchemaLoader
         cf.addColumn(column("col1","val1", 1L));
         cf.addColumn(column("col2","val2", 1L));
         cf.addColumn(column("col3","val3", 1L));
-        Mutation rm = new Mutation("Keyspace1", TEST_KEY.key, cf);
+        Mutation rm = new Mutation("Keyspace1", TEST_KEY.getKey(), cf);
         rm.apply();
 
         Runnable verify = new WrappedRunnable()
@@ -126,7 +126,7 @@ public class KeyspaceTest extends SchemaLoader
         cf.addColumn(column("a", "val1", 1L));
         cf.addColumn(column("b", "val2", 1L));
         cf.addColumn(column("c", "val3", 1L));
-        Mutation rm = new Mutation("Keyspace1", key.key, cf);
+        Mutation rm = new Mutation("Keyspace1", key.getKey(), cf);
         rm.apply();
 
         cf = cfStore.getColumnFamily(key, cellname("b"), cellname("c"), false, 100, System.currentTimeMillis());
@@ -172,7 +172,7 @@ public class KeyspaceTest extends SchemaLoader
         // so if we go to 300, we'll get at least 4 blocks, which is plenty for testing.
         for (int i = 0; i < 300; i++)
             cf.addColumn(column("col" + fmt.format(i), "omg!thisisthevalue!"+i, 1L));
-        Mutation rm = new Mutation("Keyspace1", ROW.key, cf);
+        Mutation rm = new Mutation("Keyspace1", ROW.getKey(), cf);
         rm.apply();
 
         Runnable verify = new WrappedRunnable()
@@ -226,8 +226,8 @@ public class KeyspaceTest extends SchemaLoader
         for (int i = 0; i < 10; i++)
         {
             ColumnFamily cf = ArrayBackedSortedColumns.factory.create("Keyspace1", "StandardLong1");
-            cf.addColumn(new Cell(cellname((long)i), ByteBufferUtil.EMPTY_BYTE_BUFFER, 0));
-            Mutation rm = new Mutation("Keyspace1", ROW.key, cf);
+            cf.addColumn(new BufferCell(cellname((long)i), ByteBufferUtil.EMPTY_BYTE_BUFFER, 0));
+            Mutation rm = new Mutation("Keyspace1", ROW.getKey(), cf);
             rm.apply();
         }
 
@@ -236,8 +236,8 @@ public class KeyspaceTest extends SchemaLoader
         for (int i = 10; i < 20; i++)
         {
             ColumnFamily cf = ArrayBackedSortedColumns.factory.create("Keyspace1", "StandardLong1");
-            cf.addColumn(new Cell(cellname((long)i), ByteBufferUtil.EMPTY_BYTE_BUFFER, 0));
-            Mutation rm = new Mutation("Keyspace1", ROW.key, cf);
+            cf.addColumn(new BufferCell(cellname((long)i), ByteBufferUtil.EMPTY_BYTE_BUFFER, 0));
+            Mutation rm = new Mutation("Keyspace1", ROW.getKey(), cf);
             rm.apply();
 
             cf = cfs.getColumnFamily(ROW, Composites.EMPTY, Composites.EMPTY, true, 1, System.currentTimeMillis());
@@ -275,10 +275,10 @@ public class KeyspaceTest extends SchemaLoader
         cf.addColumn(column("col5", "val5", 1L));
         cf.addColumn(column("col7", "val7", 1L));
         cf.addColumn(column("col9", "val9", 1L));
-        Mutation rm = new Mutation("Keyspace1", ROW.key, cf);
+        Mutation rm = new Mutation("Keyspace1", ROW.getKey(), cf);
         rm.apply();
 
-        rm = new Mutation("Keyspace1", ROW.key);
+        rm = new Mutation("Keyspace1", ROW.getKey());
         rm.delete("Standard1", cellname("col4"), 2L);
         rm.apply();
 
@@ -327,7 +327,7 @@ public class KeyspaceTest extends SchemaLoader
         cf.addColumn(column("col1", "val1", 1L));
         cf.addColumn(expiringColumn("col2", "val2", 1L, 60)); // long enough not to be tombstoned
         cf.addColumn(column("col3", "val3", 1L));
-        Mutation rm = new Mutation("Keyspace1", ROW.key, cf);
+        Mutation rm = new Mutation("Keyspace1", ROW.getKey(), cf);
         rm.apply();
 
         Runnable verify = new WrappedRunnable()
@@ -364,7 +364,7 @@ public class KeyspaceTest extends SchemaLoader
         cf.addColumn(column("col4", "val4", 1L));
         cf.addColumn(column("col5", "val5", 1L));
         cf.addColumn(column("col6", "val6", 1L));
-        Mutation rm = new Mutation("Keyspace1", ROW.key, cf);
+        Mutation rm = new Mutation("Keyspace1", ROW.getKey(), cf);
         rm.apply();
         cfStore.forceBlockingFlush();
 
@@ -372,7 +372,7 @@ public class KeyspaceTest extends SchemaLoader
         cf.addColumn(column("col1", "valx", 2L));
         cf.addColumn(column("col2", "valx", 2L));
         cf.addColumn(column("col3", "valx", 2L));
-        rm = new Mutation("Keyspace1", ROW.key, cf);
+        rm = new Mutation("Keyspace1", ROW.getKey(), cf);
         rm.apply();
 
         Runnable verify = new WrappedRunnable()
@@ -408,7 +408,7 @@ public class KeyspaceTest extends SchemaLoader
         ColumnFamily cf = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
         for (int i = 1000; i < 2000; i++)
             cf.addColumn(column("col" + i, ("v" + i), 1L));
-        Mutation rm = new Mutation("Keyspace1", key.key, cf);
+        Mutation rm = new Mutation("Keyspace1", key.getKey(), cf);
         rm.apply();
         cfStore.forceBlockingFlush();
 
@@ -441,7 +441,7 @@ public class KeyspaceTest extends SchemaLoader
             {
                 cf.addColumn(column("col" + i, ("v" + i), i));
             }
-            Mutation rm = new Mutation("Keyspace1", key.key, cf);
+            Mutation rm = new Mutation("Keyspace1", key.getKey(), cf);
             rm.apply();
             cfStore.forceBlockingFlush();
         }
@@ -451,7 +451,7 @@ public class KeyspaceTest extends SchemaLoader
         int i = 0;
         for (Cell c : cf.getSortedColumns())
         {
-            assertEquals(ByteBufferUtil.string(c.name.toByteBuffer()), "col" + (1000 + i++));
+            assertEquals(ByteBufferUtil.string(c.name().toByteBuffer()), "col" + (1000 + i++));
         }
         assertEquals(i, 500);
         cfStore.metric.sstablesPerReadHistogram.clear();
@@ -460,7 +460,7 @@ public class KeyspaceTest extends SchemaLoader
 
         for (Cell c : cf.getSortedColumns())
         {
-            assertEquals(ByteBufferUtil.string(c.name.toByteBuffer()), "col"+(1000 + i++));
+            assertEquals(ByteBufferUtil.string(c.name().toByteBuffer()), "col"+(1000 + i++));
         }
         assertEquals(i, 1000);
 
@@ -471,7 +471,7 @@ public class KeyspaceTest extends SchemaLoader
         i = 500;
         for (Cell c : cf.getSortedColumns())
         {
-            assertEquals(ByteBufferUtil.string(c.name.toByteBuffer()), "col"+(1000 + i++));
+            assertEquals(ByteBufferUtil.string(c.name().toByteBuffer()), "col"+(1000 + i++));
         }
         assertEquals(i, 1000);
 
@@ -505,7 +505,7 @@ public class KeyspaceTest extends SchemaLoader
         {
             for (int i = 0; i < 10; i++)
             {
-                Mutation rm = new Mutation("Keyspace1", key.key);
+                Mutation rm = new Mutation("Keyspace1", key.getKey());
                 CellName colName = type.makeCellName(ByteBufferUtil.bytes("a" + i), ByteBufferUtil.bytes(j*10 + i));
                 rm.add("StandardComposite2", colName, ByteBufferUtil.EMPTY_BYTE_BUFFER, 0);
                 rm.apply();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/test/unit/org/apache/cassandra/db/MultitableTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/MultitableTest.java b/test/unit/org/apache/cassandra/db/MultitableTest.java
index 5d2c467..cc11163 100644
--- a/test/unit/org/apache/cassandra/db/MultitableTest.java
+++ b/test/unit/org/apache/cassandra/db/MultitableTest.java
@@ -41,12 +41,12 @@ public class MultitableTest extends SchemaLoader
 
         cf = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
         cf.addColumn(column("col1", "val1", 1L));
-        rm = new Mutation("Keyspace1", dk.key, cf);
+        rm = new Mutation("Keyspace1", dk.getKey(), cf);
         rm.apply();
 
         cf = ArrayBackedSortedColumns.factory.create("Keyspace2", "Standard1");
         cf.addColumn(column("col2", "val2", 1L));
-        rm = new Mutation("Keyspace2", dk.key, cf);
+        rm = new Mutation("Keyspace2", dk.getKey(), cf);
         rm.apply();
 
         keyspace1.getColumnFamilyStore("Standard1").forceBlockingFlush();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/test/unit/org/apache/cassandra/db/NativeCellTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/NativeCellTest.java b/test/unit/org/apache/cassandra/db/NativeCellTest.java
new file mode 100644
index 0000000..0d03f55
--- /dev/null
+++ b/test/unit/org/apache/cassandra/db/NativeCellTest.java
@@ -0,0 +1,171 @@
+/*
+ * 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.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.security.MessageDigest;
+import java.security.NoSuchAlgorithmException;
+import java.util.Arrays;
+import java.util.Random;
+import java.util.concurrent.ThreadLocalRandom;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.db.composites.CellName;
+import org.apache.cassandra.db.composites.CellNameType;
+import org.apache.cassandra.db.composites.CompoundDenseCellNameType;
+import org.apache.cassandra.db.composites.CompoundSparseCellNameType;
+import org.apache.cassandra.db.composites.SimpleDenseCellNameType;
+import org.apache.cassandra.db.composites.SimpleSparseCellNameType;
+import org.apache.cassandra.db.context.CounterContext;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.io.util.DataOutputBuffer;
+import org.apache.cassandra.utils.concurrent.OpOrder;
+import org.apache.cassandra.utils.memory.NativeAllocator;
+import org.apache.cassandra.utils.memory.NativePool;
+
+import static org.apache.cassandra.db.composites.CellNames.compositeDense;
+import static org.apache.cassandra.db.composites.CellNames.compositeSparse;
+import static org.apache.cassandra.db.composites.CellNames.simpleDense;
+import static org.apache.cassandra.db.composites.CellNames.simpleSparse;
+import static org.apache.cassandra.utils.ByteBufferUtil.bytes;
+
+public class NativeCellTest
+{
+
+    private static final NativeAllocator nativeAllocator = new NativePool(Integer.MAX_VALUE, Integer.MAX_VALUE, 1f, null).newAllocator();
+    private static final OpOrder.Group group = new OpOrder().start();
+
+    static class Name
+    {
+        final CellName name;
+        final CellNameType type;
+        Name(CellName name, CellNameType type)
+        {
+            this.name = name;
+            this.type = type;
+        }
+    }
+
+    static ByteBuffer[] bytess(String ... strings)
+    {
+        ByteBuffer[] r = new ByteBuffer[strings.length];
+        for (int i = 0 ; i < r.length ; i++)
+            r[i] = bytes(strings[i]);
+        return r;
+    }
+
+    final static Name[] TESTS = new Name[]
+                          {
+                              new Name(simpleDense(bytes("a")), new SimpleDenseCellNameType(UTF8Type.instance)),
+                              new Name(simpleSparse(new ColumnIdentifier("a", true)), new SimpleSparseCellNameType(UTF8Type.instance)),
+                              new Name(compositeDense(bytes("a"), bytes("b")), new CompoundDenseCellNameType(Arrays.<AbstractType<?>>asList(UTF8Type.instance, UTF8Type.instance))),
+                              new Name(compositeSparse(bytess("b", "c"), new ColumnIdentifier("a", true), false), new CompoundSparseCellNameType(Arrays.<AbstractType<?>>asList(UTF8Type.instance, UTF8Type.instance))),
+                              new Name(compositeSparse(bytess("b", "c"), new ColumnIdentifier("a", true), true), new CompoundSparseCellNameType(Arrays.<AbstractType<?>>asList(UTF8Type.instance, UTF8Type.instance)))
+                          };
+
+    private static final CFMetaData metadata = new CFMetaData("", "", ColumnFamilyType.Standard, null);
+    static
+    {
+        try
+        {
+            metadata.addColumnDefinition(new ColumnDefinition(null, null, new ColumnIdentifier("a", true), UTF8Type.instance, null, null, null, null, null));
+        }
+        catch (ConfigurationException e)
+        {
+            throw new AssertionError();
+        }
+    }
+
+    @Test
+    public void testCells() throws IOException
+    {
+        Random rand = ThreadLocalRandom.current();
+        for (Name test : TESTS)
+        {
+            byte[] bytes = new byte[16];
+            rand.nextBytes(bytes);
+
+            // test regular Cell
+            Cell buf, nat;
+            buf = new BufferCell(test.name, ByteBuffer.wrap(bytes), rand.nextLong());
+            nat = buf.localCopy(metadata, nativeAllocator, group);
+            test(test, buf, nat);
+
+            // test DeletedCell
+            buf = new BufferDeletedCell(test.name, rand.nextInt(100000), rand.nextLong());
+            nat = buf.localCopy(metadata, nativeAllocator, group);
+            test(test, buf, nat);
+
+            // test ExpiringCell
+            buf = new BufferExpiringCell(test.name, ByteBuffer.wrap(bytes), rand.nextLong(),  rand.nextInt(100000));
+            nat = buf.localCopy(metadata, nativeAllocator, group);
+            test(test, buf, nat);
+
+            // test CounterCell
+            buf = new BufferCounterCell(test.name, CounterContext.instance().createLocal(rand.nextLong()), rand.nextLong(),  rand.nextInt(100000));
+            nat = buf.localCopy(metadata, nativeAllocator, group);
+            test(test, buf, nat);
+        }
+    }
+
+    static void test(Name test, Cell buf, Cell nat) throws IOException
+    {
+        Assert.assertTrue(buf.equals(nat));
+        Assert.assertTrue(nat.equals(buf));
+        Assert.assertTrue(buf.equals(buf));
+        Assert.assertTrue(nat.equals(nat));
+
+        try
+        {
+            MessageDigest d1 = MessageDigest.getInstance("MD5");
+            MessageDigest d2 = MessageDigest.getInstance("MD5");
+            buf.updateDigest(d1);
+            nat.updateDigest(d2);
+            Assert.assertArrayEquals(d1.digest(), d2.digest());
+        }
+        catch (NoSuchAlgorithmException e)
+        {
+            throw new IllegalStateException(e);
+        }
+
+        byte[] serialized;
+        try (DataOutputBuffer bufOut = new DataOutputBuffer())
+        {
+            test.type.columnSerializer().serialize(nat, bufOut);
+            serialized = bufOut.getData();
+        }
+
+        ByteArrayInputStream bufIn = new ByteArrayInputStream(serialized, 0, serialized.length);
+        Cell deserialized = test.type.columnSerializer().deserialize(new DataInputStream(bufIn));
+        Assert.assertTrue(buf.equals(deserialized));
+
+    }
+
+
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/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 4f33dae..6ac5f13 100644
--- a/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java
+++ b/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java
@@ -52,7 +52,7 @@ import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.sstable.SSTableReader;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.concurrent.OpOrder;
-import org.apache.cassandra.utils.memory.PoolAllocator;
+import org.apache.cassandra.utils.memory.MemtableAllocator;
 
 import static org.apache.cassandra.Util.dk;
 import static org.junit.Assert.assertEquals;
@@ -534,13 +534,6 @@ public class RangeTombstoneTest extends SchemaLoader
 
         public void forceBlockingFlush(){}
 
-        @Override
-        public PoolAllocator getAllocator()
-        {
-            return null;
-        }
-
-
         public ColumnFamilyStore getIndexCfs(){ return null; }
 
         public void removeIndex(ByteBuffer columnName){}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/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 61ef912..6d19d3a 100644
--- a/test/unit/org/apache/cassandra/db/ReadMessageTest.java
+++ b/test/unit/org/apache/cassandra/db/ReadMessageTest.java
@@ -54,15 +54,15 @@ public class ReadMessageTest extends SchemaLoader
         DecoratedKey dk = Util.dk("row1");
         long ts = System.currentTimeMillis();
 
-        rm = new SliceByNamesReadCommand("Keyspace1", dk.key, "Standard1", ts, new NamesQueryFilter(colList));
+        rm = new SliceByNamesReadCommand("Keyspace1", dk.getKey(), "Standard1", ts, new NamesQueryFilter(colList));
         rm2 = serializeAndDeserializeReadMessage(rm);
         assert rm2.toString().equals(rm.toString());
 
-        rm = new SliceFromReadCommand("Keyspace1", dk.key, "Standard1", ts, new SliceQueryFilter(Composites.EMPTY, Composites.EMPTY, true, 2));
+        rm = new SliceFromReadCommand("Keyspace1", dk.getKey(), "Standard1", ts, new SliceQueryFilter(Composites.EMPTY, Composites.EMPTY, true, 2));
         rm2 = serializeAndDeserializeReadMessage(rm);
         assert rm2.toString().equals(rm.toString());
 
-        rm = new SliceFromReadCommand("Keyspace1", dk.key, "Standard1", ts, new SliceQueryFilter(Util.cellname("a"), Util.cellname("z"), true, 5));
+        rm = new SliceFromReadCommand("Keyspace1", dk.getKey(), "Standard1", ts, new SliceQueryFilter(Util.cellname("a"), Util.cellname("z"), true, 5));
         rm2 = serializeAndDeserializeReadMessage(rm);
         assert rm2.toString().equals(rm.toString());
     }
@@ -87,11 +87,11 @@ public class ReadMessageTest extends SchemaLoader
         DecoratedKey dk = Util.dk("key1");
 
         // add data
-        rm = new Mutation("Keyspace1", dk.key);
+        rm = new Mutation("Keyspace1", dk.getKey());
         rm.add("Standard1", Util.cellname("Column1"), ByteBufferUtil.bytes("abcd"), 0);
         rm.apply();
 
-        ReadCommand command = new SliceByNamesReadCommand("Keyspace1", dk.key, "Standard1", System.currentTimeMillis(), new NamesQueryFilter(FBUtilities.singleton(Util.cellname("Column1"), type)));
+        ReadCommand command = new SliceByNamesReadCommand("Keyspace1", dk.getKey(), "Standard1", System.currentTimeMillis(), new NamesQueryFilter(FBUtilities.singleton(Util.cellname("Column1"), type)));
         Row row = command.getRow(keyspace);
         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/8541cca7/test/unit/org/apache/cassandra/db/RecoveryManager3Test.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RecoveryManager3Test.java b/test/unit/org/apache/cassandra/db/RecoveryManager3Test.java
index 5432a21..c9bc86a 100644
--- a/test/unit/org/apache/cassandra/db/RecoveryManager3Test.java
+++ b/test/unit/org/apache/cassandra/db/RecoveryManager3Test.java
@@ -49,12 +49,12 @@ public class RecoveryManager3Test extends SchemaLoader
 
         cf = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
         cf.addColumn(column("col1", "val1", 1L));
-        rm = new Mutation("Keyspace1", dk.key, cf);
+        rm = new Mutation("Keyspace1", dk.getKey(), cf);
         rm.apply();
 
         cf = ArrayBackedSortedColumns.factory.create("Keyspace2", "Standard3");
         cf.addColumn(column("col2", "val2", 1L));
-        rm = new Mutation("Keyspace2", dk.key, cf);
+        rm = new Mutation("Keyspace2", dk.getKey(), cf);
         rm.apply();
 
         keyspace1.getColumnFamilyStore("Standard1").clearUnsafe();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/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 004eb99..e812dfb 100644
--- a/test/unit/org/apache/cassandra/db/RecoveryManagerTest.java
+++ b/test/unit/org/apache/cassandra/db/RecoveryManagerTest.java
@@ -53,12 +53,12 @@ public class RecoveryManagerTest extends SchemaLoader
 
         cf = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
         cf.addColumn(column("col1", "val1", 1L));
-        rm = new Mutation("Keyspace1", dk.key, cf);
+        rm = new Mutation("Keyspace1", dk.getKey(), cf);
         rm.apply();
 
         cf = ArrayBackedSortedColumns.factory.create("Keyspace2", "Standard3");
         cf.addColumn(column("col2", "val2", 1L));
-        rm = new Mutation("Keyspace2", dk.key, cf);
+        rm = new Mutation("Keyspace2", dk.getKey(), cf);
         rm.apply();
 
         keyspace1.getColumnFamilyStore("Standard1").clearUnsafe();
@@ -83,8 +83,8 @@ public class RecoveryManagerTest extends SchemaLoader
         for (int i = 0; i < 10; ++i)
         {
             cf = ArrayBackedSortedColumns.factory.create("Keyspace1", "Counter1");
-            cf.addColumn(CounterCell.createLocal(cellname("col"), 1L, 1L, Long.MIN_VALUE));
-            rm = new Mutation("Keyspace1", dk.key, cf);
+            cf.addColumn(BufferCounterCell.createLocal(cellname("col"), 1L, 1L, Long.MIN_VALUE));
+            rm = new Mutation("Keyspace1", dk.getKey(), cf);
             rm.apply();
         }
 
@@ -115,7 +115,7 @@ public class RecoveryManagerTest extends SchemaLoader
             long ts = TimeUnit.MILLISECONDS.toMicros(timeMS + (i * 1000));
             ColumnFamily cf = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
             cf.addColumn(column("name-" + i, "value", ts));
-            Mutation rm = new Mutation("Keyspace1", dk.key, cf);
+            Mutation rm = new Mutation("Keyspace1", dk.getKey(), cf);
             rm.apply();
         }
         keyspace1.getColumnFamilyStore("Standard1").clearUnsafe();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/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
index 629b40d..1624913 100644
--- a/test/unit/org/apache/cassandra/db/RemoveCellTest.java
+++ b/test/unit/org/apache/cassandra/db/RemoveCellTest.java
@@ -41,13 +41,13 @@ public class RemoveCellTest extends SchemaLoader
         DecoratedKey dk = Util.dk("key1");
 
         // add data
-        rm = new Mutation("Keyspace1", dk.key);
+        rm = new Mutation("Keyspace1", dk.getKey());
         rm.add("Standard1", Util.cellname("Column1"), ByteBufferUtil.bytes("asdf"), 0);
         rm.apply();
         store.forceBlockingFlush();
 
         // remove
-        rm = new Mutation("Keyspace1", dk.key);
+        rm = new Mutation("Keyspace1", dk.getKey());
         rm.delete("Standard1", Util.cellname("Column1"), 1);
         rm.apply();
 
@@ -60,9 +60,9 @@ public class RemoveCellTest extends SchemaLoader
                                               Integer.MAX_VALUE));
     }
 
-    private static DeletedCell dc(String name, int ldt, long timestamp)
+    private static BufferDeletedCell dc(String name, int ldt, long timestamp)
     {
-        return new DeletedCell(Util.cellname(name), ldt, timestamp);
+        return new BufferDeletedCell(Util.cellname(name), ldt, timestamp);
     }
 
     @Test

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/test/unit/org/apache/cassandra/db/RemoveColumnFamilyTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RemoveColumnFamilyTest.java b/test/unit/org/apache/cassandra/db/RemoveColumnFamilyTest.java
index b80b9d1..09eed71 100644
--- a/test/unit/org/apache/cassandra/db/RemoveColumnFamilyTest.java
+++ b/test/unit/org/apache/cassandra/db/RemoveColumnFamilyTest.java
@@ -39,12 +39,12 @@ public class RemoveColumnFamilyTest extends SchemaLoader
         DecoratedKey dk = Util.dk("key1");
 
         // add data
-        rm = new Mutation("Keyspace1", dk.key);
+        rm = new Mutation("Keyspace1", dk.getKey());
         rm.add("Standard1", Util.cellname("Column1"), ByteBufferUtil.bytes("asdf"), 0);
         rm.apply();
 
         // remove
-        rm = new Mutation("Keyspace1", dk.key);
+        rm = new Mutation("Keyspace1", dk.getKey());
         rm.delete("Standard1", 1);
         rm.apply();
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush1Test.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush1Test.java b/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush1Test.java
index c95ab97..f898f16 100644
--- a/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush1Test.java
+++ b/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush1Test.java
@@ -39,14 +39,14 @@ public class RemoveColumnFamilyWithFlush1Test extends SchemaLoader
         DecoratedKey dk = Util.dk("key1");
 
         // add data
-        rm = new Mutation("Keyspace1", dk.key);
+        rm = new Mutation("Keyspace1", dk.getKey());
         rm.add("Standard1", Util.cellname("Column1"), ByteBufferUtil.bytes("asdf"), 0);
         rm.add("Standard1", Util.cellname("Column2"), ByteBufferUtil.bytes("asdf"), 0);
         rm.apply();
         store.forceBlockingFlush();
 
         // remove
-        rm = new Mutation("Keyspace1", dk.key);
+        rm = new Mutation("Keyspace1", dk.getKey());
         rm.delete("Standard1", 1);
         rm.apply();
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush2Test.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush2Test.java b/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush2Test.java
index df81d58..1e910ad 100644
--- a/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush2Test.java
+++ b/test/unit/org/apache/cassandra/db/RemoveColumnFamilyWithFlush2Test.java
@@ -39,11 +39,11 @@ public class RemoveColumnFamilyWithFlush2Test extends SchemaLoader
         DecoratedKey dk = Util.dk("key1");
 
         // add data
-        rm = new Mutation("Keyspace1", dk.key);
+        rm = new Mutation("Keyspace1", dk.getKey());
         rm.add("Standard1", Util.cellname("Column1"), ByteBufferUtil.bytes("asdf"), 0);
         rm.apply();
         // remove
-        rm = new Mutation("Keyspace1", dk.key);
+        rm = new Mutation("Keyspace1", dk.getKey());
         rm.delete("Standard1", 1);
         rm.apply();
         store.forceBlockingFlush();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/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
index 3026897..6eee4a6 100644
--- a/test/unit/org/apache/cassandra/db/RemoveSubCellTest.java
+++ b/test/unit/org/apache/cassandra/db/RemoveSubCellTest.java
@@ -46,14 +46,14 @@ public class RemoveSubCellTest extends SchemaLoader
         DecoratedKey dk = Util.dk("key1");
 
         // add data
-        rm = new Mutation("Keyspace1", dk.key);
+        rm = new Mutation("Keyspace1", dk.getKey());
         Util.addMutation(rm, "Super1", "SC1", 1, "asdf", 0);
         rm.apply();
         store.forceBlockingFlush();
 
         CellName cname = CellNames.compositeDense(ByteBufferUtil.bytes("SC1"), getBytes(1L));
         // remove
-        rm = new Mutation("Keyspace1", dk.key);
+        rm = new Mutation("Keyspace1", dk.getKey());
         rm.delete("Super1", cname, 1);
         rm.apply();
 
@@ -71,7 +71,7 @@ public class RemoveSubCellTest extends SchemaLoader
         DecoratedKey dk = Util.dk("key2");
 
         // add data
-        rm = new Mutation("Keyspace1", dk.key);
+        rm = new Mutation("Keyspace1", dk.getKey());
         Util.addMutation(rm, "Super1", "SC1", 1, "asdf", 0);
         rm.apply();
         store.forceBlockingFlush();
@@ -79,7 +79,7 @@ public class RemoveSubCellTest extends SchemaLoader
         // remove the SC
         ByteBuffer scName = ByteBufferUtil.bytes("SC1");
         CellName cname = CellNames.compositeDense(scName, getBytes(1L));
-        rm = new Mutation("Keyspace1", dk.key);
+        rm = new Mutation("Keyspace1", dk.getKey());
         rm.deleteRange("Super1", SuperColumns.startOf(scName), SuperColumns.endOf(scName), 1);
         rm.apply();
 
@@ -89,7 +89,7 @@ public class RemoveSubCellTest extends SchemaLoader
         Uninterruptibles.sleepUninterruptibly(1, TimeUnit.SECONDS);
 
         // remove the column itself
-        rm = new Mutation("Keyspace1", dk.key);
+        rm = new Mutation("Keyspace1", dk.getKey());
         rm.delete("Super1", cname, 2);
         rm.apply();
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/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 3de1e7d..7b6ff99 100644
--- a/test/unit/org/apache/cassandra/db/RowCacheTest.java
+++ b/test/unit/org/apache/cassandra/db/RowCacheTest.java
@@ -226,7 +226,7 @@ public class RowCacheTest extends SchemaLoader
         int i = 0;
         for(Cell c : cachedCf)
         {
-            assertEquals(c.name, Util.cellname(i++));
+            assertEquals(c.name(), Util.cellname(i++));
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/test/unit/org/apache/cassandra/db/RowIterationTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RowIterationTest.java b/test/unit/org/apache/cassandra/db/RowIterationTest.java
index bfba858..245a6a7 100644
--- a/test/unit/org/apache/cassandra/db/RowIterationTest.java
+++ b/test/unit/org/apache/cassandra/db/RowIterationTest.java
@@ -49,7 +49,7 @@ public class RowIterationTest extends SchemaLoader
         Set<DecoratedKey> inserted = new HashSet<DecoratedKey>();
         for (int i = 0; i < ROWS_PER_SSTABLE; i++) {
             DecoratedKey key = Util.dk(String.valueOf(i));
-            Mutation rm = new Mutation(KEYSPACE1, key.key);
+            Mutation rm = new Mutation(KEYSPACE1, key.getKey());
             rm.add("Super3", CellNames.compositeDense(ByteBufferUtil.bytes("sc"), ByteBufferUtil.bytes(String.valueOf(i))), ByteBuffer.wrap(new byte[ROWS_PER_SSTABLE * 10 - i * 2]), i);
             rm.apply();
             inserted.add(key);
@@ -67,14 +67,14 @@ public class RowIterationTest extends SchemaLoader
         DecoratedKey key = Util.dk("key");
 
         // Delete row in first sstable
-        Mutation rm = new Mutation(KEYSPACE1, key.key);
+        Mutation rm = new Mutation(KEYSPACE1, key.getKey());
         rm.delete(CF_NAME, 0);
         rm.add(CF_NAME, Util.cellname("c"), ByteBufferUtil.bytes("values"), 0L);
         rm.apply();
         store.forceBlockingFlush();
 
         // Delete row in second sstable with higher timestamp
-        rm = new Mutation(KEYSPACE1, key.key);
+        rm = new Mutation(KEYSPACE1, key.getKey());
         rm.delete(CF_NAME, 1);
         rm.add(CF_NAME, Util.cellname("c"), ByteBufferUtil.bytes("values"), 1L);
         DeletionInfo delInfo2 = rm.getColumnFamilies().iterator().next().deletionInfo();
@@ -95,7 +95,7 @@ public class RowIterationTest extends SchemaLoader
         DecoratedKey key = Util.dk("key");
 
         // Delete a row in first sstable
-        Mutation rm = new Mutation(KEYSPACE1, key.key);
+        Mutation rm = new Mutation(KEYSPACE1, key.getKey());
         rm.delete(CF_NAME, 0);
         rm.apply();
         store.forceBlockingFlush();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/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 3de7fdc..653c9f5 100644
--- a/test/unit/org/apache/cassandra/db/RowTest.java
+++ b/test/unit/org/apache/cassandra/db/RowTest.java
@@ -100,7 +100,7 @@ public class RowTest extends SchemaLoader
     @Test
     public void testExpiringColumnExpiration()
     {
-        Cell c = new ExpiringCell(CellNames.simpleDense(ByteBufferUtil.bytes("one")), ByteBufferUtil.bytes("A"), 0, 1);
+        Cell c = new BufferExpiringCell(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/8541cca7/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 e820fc2..220e2a4 100644
--- a/test/unit/org/apache/cassandra/db/ScrubTest.java
+++ b/test/unit/org/apache/cassandra/db/ScrubTest.java
@@ -103,8 +103,8 @@ public class ScrubTest extends SchemaLoader
         SSTableReader sstable = cfs.getSSTables().iterator().next();
 
         // overwrite one row with garbage
-        long row0Start = sstable.getPosition(RowPosition.forKey(ByteBufferUtil.bytes("0"), sstable.partitioner), SSTableReader.Operator.EQ).position;
-        long row1Start = sstable.getPosition(RowPosition.forKey(ByteBufferUtil.bytes("1"), sstable.partitioner), SSTableReader.Operator.EQ).position;
+        long row0Start = sstable.getPosition(RowPosition.ForKey.get(ByteBufferUtil.bytes("0"), sstable.partitioner), SSTableReader.Operator.EQ).position;
+        long row1Start = sstable.getPosition(RowPosition.ForKey.get(ByteBufferUtil.bytes("1"), sstable.partitioner), SSTableReader.Operator.EQ).position;
         long startPosition = row0Start < row1Start ? row0Start : row1Start;
         long endPosition = row0Start < row1Start ? row1Start : row0Start;
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/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
index 9e911b4..45f9e05 100644
--- a/test/unit/org/apache/cassandra/db/SecondaryIndexCellSizeTest.java
+++ b/test/unit/org/apache/cassandra/db/SecondaryIndexCellSizeTest.java
@@ -31,7 +31,7 @@ import org.apache.cassandra.db.index.SecondaryIndexSearcher;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.concurrent.OpOrder;
-import org.apache.cassandra.utils.memory.PoolAllocator;
+import org.apache.cassandra.utils.memory.MemtableAllocator;
 
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
@@ -51,7 +51,7 @@ public class SecondaryIndexCellSizeTest
 
         // for read
         buffer.flip();
-        Cell cell = new Cell(CellNames.simpleDense(ByteBufferUtil.bytes("test")), buffer, 0);
+        Cell cell = new BufferCell(CellNames.simpleDense(ByteBufferUtil.bytes("test")), buffer, 0);
 
         SecondaryIndexCellSizeTest.MockRowIndex mockRowIndex = new SecondaryIndexCellSizeTest.MockRowIndex();
         SecondaryIndexCellSizeTest.MockColumnIndex mockColumnIndex = new SecondaryIndexCellSizeTest.MockColumnIndex();
@@ -93,11 +93,6 @@ public class SecondaryIndexCellSizeTest
         {
         }
 
-        public PoolAllocator getAllocator()
-        {
-            return null;
-        }
-
         public ColumnFamilyStore getIndexCfs()
         {
             return null;
@@ -173,12 +168,6 @@ public class SecondaryIndexCellSizeTest
         }
 
         @Override
-        public PoolAllocator getAllocator()
-        {
-            return null;
-        }
-
-        @Override
         public ColumnFamilyStore getIndexCfs()
         {
             return null;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/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 45ce347..0dd6b8f 100644
--- a/test/unit/org/apache/cassandra/db/SerializationsTest.java
+++ b/test/unit/org/apache/cassandra/db/SerializationsTest.java
@@ -39,7 +39,6 @@ import org.apache.cassandra.utils.ByteBufferUtil;
 import org.junit.Test;
 
 import java.io.DataInputStream;
-import java.io.DataOutputStream;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.*;
@@ -369,21 +368,21 @@ public class SerializationsTest extends AbstractSerializationsTester
 
         private Statics()
         {
-            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));
+            StandardCf.addColumn(new BufferCell(cn("aaaa")));
+            StandardCf.addColumn(new BufferCell(cn("bbbb"), bb("bbbbb-value")));
+            StandardCf.addColumn(new BufferCell(cn("cccc"), bb("ccccc-value"), 1000L));
+            StandardCf.addColumn(new BufferDeletedCell(cn("dddd"), 500, 1000));
+            StandardCf.addColumn(new BufferDeletedCell(cn("eeee"), bb("eeee-value"), 1001));
+            StandardCf.addColumn(new BufferExpiringCell(cn("ffff"), bb("ffff-value"), 2000, 1000));
+            StandardCf.addColumn(new BufferExpiringCell(cn("gggg"), bb("gggg-value"), 2001, 1000, 2002));
+
+            SuperCf.addColumn(new BufferCell(CellNames.compositeDense(SC, bb("aaaa"))));
+            SuperCf.addColumn(new BufferCell(CellNames.compositeDense(SC, bb("bbbb")), bb("bbbbb-value")));
+            SuperCf.addColumn(new BufferCell(CellNames.compositeDense(SC, bb("cccc")), bb("ccccc-value"), 1000L));
+            SuperCf.addColumn(new BufferDeletedCell(CellNames.compositeDense(SC, bb("dddd")), 500, 1000));
+            SuperCf.addColumn(new BufferDeletedCell(CellNames.compositeDense(SC, bb("eeee")), bb("eeee-value"), 1001));
+            SuperCf.addColumn(new BufferExpiringCell(CellNames.compositeDense(SC, bb("ffff")), bb("ffff-value"), 2000, 1000));
+            SuperCf.addColumn(new BufferExpiringCell(CellNames.compositeDense(SC, bb("gggg")), bb("gggg-value"), 2001, 1000, 2002));
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/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 1b02ea5..80c0ff1 100644
--- a/test/unit/org/apache/cassandra/db/TimeSortTest.java
+++ b/test/unit/org/apache/cassandra/db/TimeSortTest.java
@@ -44,12 +44,12 @@ public class TimeSortTest extends SchemaLoader
         Mutation rm;
         DecoratedKey key = Util.dk("key0");
 
-        rm = new Mutation("Keyspace1", key.key);
+        rm = new Mutation("Keyspace1", key.getKey());
         rm.add("StandardLong1", cellname(100), ByteBufferUtil.bytes("a"), 100);
         rm.apply();
         cfStore.forceBlockingFlush();
 
-        rm = new Mutation("Keyspace1", key.key);
+        rm = new Mutation("Keyspace1", key.getKey());
         rm.add("StandardLong1", cellname(0), ByteBufferUtil.bytes("b"), 0);
         rm.apply();
 
@@ -81,14 +81,14 @@ public class TimeSortTest extends SchemaLoader
 
         // interleave some new data to test memtable + sstable
         DecoratedKey key = Util.dk("900");
-        Mutation rm = new Mutation("Keyspace1", key.key);
+        Mutation rm = new Mutation("Keyspace1", key.getKey());
         for (int j = 0; j < 4; ++j)
         {
             rm.add("StandardLong1", cellname(j * 2 + 1), ByteBufferUtil.bytes("b"), j * 2 + 1);
         }
         rm.apply();
         // and some overwrites
-        rm = new Mutation("Keyspace1", key.key);
+        rm = new Mutation("Keyspace1", key.getKey());
         rm.add("StandardLong1", cellname(0), ByteBufferUtil.bytes("c"), 100);
         rm.add("StandardLong1", cellname(10), ByteBufferUtil.bytes("c"), 100);
         rm.apply();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/test/unit/org/apache/cassandra/db/compaction/AntiCompactionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/AntiCompactionTest.java b/test/unit/org/apache/cassandra/db/compaction/AntiCompactionTest.java
index fb87f80..b8637a8 100644
--- a/test/unit/org/apache/cassandra/db/compaction/AntiCompactionTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/AntiCompactionTest.java
@@ -56,7 +56,7 @@ public class AntiCompactionTest extends SchemaLoader
         for (int i = 0; i < 10; i++)
         {
             DecoratedKey key = Util.dk(Integer.toString(i));
-            Mutation rm = new Mutation(KEYSPACE1, key.key);
+            Mutation rm = new Mutation(KEYSPACE1, key.getKey());
             for (int j = 0; j < 10; j++)
                 rm.add("Standard1", Util.cellname(Integer.toString(j)),
                        ByteBufferUtil.EMPTY_BYTE_BUFFER,
@@ -85,12 +85,12 @@ public class AntiCompactionTest extends SchemaLoader
                 SSTableIdentityIterator row = (SSTableIdentityIterator) scanner.next();
                 if (sstable.isRepaired())
                 {
-                    assertTrue(range.contains(row.getKey().token));
+                    assertTrue(range.contains(row.getKey().getToken()));
                     repairedKeys++;
                 }
                 else
                 {
-                    assertFalse(range.contains(row.getKey().token));
+                    assertFalse(range.contains(row.getKey().getToken()));
                     nonRepairedKeys++;
                 }
             }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/test/unit/org/apache/cassandra/db/compaction/BlacklistingCompactionsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/BlacklistingCompactionsTest.java b/test/unit/org/apache/cassandra/db/compaction/BlacklistingCompactionsTest.java
index d462d96..e6626ea 100644
--- a/test/unit/org/apache/cassandra/db/compaction/BlacklistingCompactionsTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/BlacklistingCompactionsTest.java
@@ -89,7 +89,7 @@ public class BlacklistingCompactionsTest extends SchemaLoader
             for (int i = 0; i < ROWS_PER_SSTABLE; i++)
             {
                 DecoratedKey key = Util.dk(String.valueOf(i % 2));
-                Mutation rm = new Mutation(KEYSPACE, key.key);
+                Mutation rm = new Mutation(KEYSPACE, key.getKey());
                 long timestamp = j * ROWS_PER_SSTABLE + i;
                 rm.add("Standard1", cellname(i / 2), ByteBufferUtil.EMPTY_BYTE_BUFFER, timestamp);
                 maxTimestampExpected = Math.max(timestamp, maxTimestampExpected);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/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 42445b5..f9cfb2c 100644
--- a/test/unit/org/apache/cassandra/db/compaction/CompactionsPurgeTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/CompactionsPurgeTest.java
@@ -61,7 +61,7 @@ public class CompactionsPurgeTest extends SchemaLoader
         Mutation rm;
 
         // inserts
-        rm = new Mutation(KEYSPACE1, key.key);
+        rm = new Mutation(KEYSPACE1, key.getKey());
         for (int i = 0; i < 10; i++)
         {
             rm.add(cfName, cellname(String.valueOf(i)), ByteBufferUtil.EMPTY_BYTE_BUFFER, 0);
@@ -72,14 +72,14 @@ public class CompactionsPurgeTest extends SchemaLoader
         // deletes
         for (int i = 0; i < 10; i++)
         {
-            rm = new Mutation(KEYSPACE1, key.key);
+            rm = new Mutation(KEYSPACE1, key.getKey());
             rm.delete(cfName, cellname(String.valueOf(i)), 1);
             rm.apply();
         }
         cfs.forceBlockingFlush();
 
         // resurrect one column
-        rm = new Mutation(KEYSPACE1, key.key);
+        rm = new Mutation(KEYSPACE1, key.getKey());
         rm.add(cfName, cellname(String.valueOf(5)), ByteBufferUtil.EMPTY_BYTE_BUFFER, 2);
         rm.apply();
         cfs.forceBlockingFlush();
@@ -106,7 +106,7 @@ public class CompactionsPurgeTest extends SchemaLoader
             DecoratedKey key = Util.dk("key" + k);
 
             // inserts
-            rm = new Mutation(KEYSPACE2, key.key);
+            rm = new Mutation(KEYSPACE2, key.getKey());
             for (int i = 0; i < 10; i++)
             {
                 rm.add(cfName, cellname(String.valueOf(i)), ByteBufferUtil.EMPTY_BYTE_BUFFER, 0);
@@ -117,7 +117,7 @@ public class CompactionsPurgeTest extends SchemaLoader
             // deletes
             for (int i = 0; i < 10; i++)
             {
-                rm = new Mutation(KEYSPACE2, key.key);
+                rm = new Mutation(KEYSPACE2, key.getKey());
                 rm.delete(cfName, cellname(String.valueOf(i)), 1);
                 rm.apply();
             }
@@ -131,7 +131,7 @@ public class CompactionsPurgeTest extends SchemaLoader
         // for first key. Then submit minor compaction on remembered sstables.
         cfs.forceBlockingFlush();
         Collection<SSTableReader> sstablesIncomplete = cfs.getSSTables();
-        rm = new Mutation(KEYSPACE2, key1.key);
+        rm = new Mutation(KEYSPACE2, key1.getKey());
         rm.add(cfName, cellname(String.valueOf(5)), ByteBufferUtil.EMPTY_BYTE_BUFFER, 2);
         rm.apply();
         cfs.forceBlockingFlush();
@@ -163,20 +163,20 @@ public class CompactionsPurgeTest extends SchemaLoader
         DecoratedKey key3 = Util.dk("key3");
 
         // inserts
-        rm = new Mutation(KEYSPACE2, key3.key);
+        rm = new Mutation(KEYSPACE2, key3.getKey());
         rm.add(cfName, cellname("c1"), ByteBufferUtil.EMPTY_BYTE_BUFFER, 8);
         rm.add(cfName, cellname("c2"), ByteBufferUtil.EMPTY_BYTE_BUFFER, 8);
         rm.apply();
         cfs.forceBlockingFlush();
         // delete c1
-        rm = new Mutation(KEYSPACE2, key3.key);
+        rm = new Mutation(KEYSPACE2, key3.getKey());
         rm.delete(cfName, cellname("c1"), 10);
         rm.apply();
         cfs.forceBlockingFlush();
         Collection<SSTableReader> sstablesIncomplete = cfs.getSSTables();
 
         // delete c2 so we have new delete in a diffrent SSTable
-        rm = new Mutation(KEYSPACE2, key3.key);
+        rm = new Mutation(KEYSPACE2, key3.getKey());
         rm.delete(cfName, cellname("c2"), 9);
         rm.apply();
         cfs.forceBlockingFlush();
@@ -204,7 +204,7 @@ public class CompactionsPurgeTest extends SchemaLoader
         Mutation rm;
 
         // inserts
-        rm = new Mutation(KEYSPACE1, key.key);
+        rm = new Mutation(KEYSPACE1, key.getKey());
         for (int i = 0; i < 5; i++)
         {
             rm.add(cfName, cellname(String.valueOf(i)), ByteBufferUtil.EMPTY_BYTE_BUFFER, 0);
@@ -214,7 +214,7 @@ public class CompactionsPurgeTest extends SchemaLoader
         // deletes
         for (int i = 0; i < 5; i++)
         {
-            rm = new Mutation(KEYSPACE1, key.key);
+            rm = new Mutation(KEYSPACE1, key.getKey());
             rm.delete(cfName, cellname(String.valueOf(i)), 1);
             rm.apply();
         }
@@ -242,7 +242,7 @@ public class CompactionsPurgeTest extends SchemaLoader
         Mutation rm;
 
         // inserts
-        rm = new Mutation(keyspaceName, key.key);
+        rm = new Mutation(keyspaceName, key.getKey());
         for (int i = 0; i < 10; i++)
         {
             rm.add(cfName, cellname(String.valueOf(i)), ByteBufferUtil.EMPTY_BYTE_BUFFER, 0);
@@ -253,7 +253,7 @@ public class CompactionsPurgeTest extends SchemaLoader
         cfs.getColumnFamily(QueryFilter.getIdentityFilter(key, cfName, System.currentTimeMillis()));
 
         // deletes row
-        rm = new Mutation(keyspaceName, key.key);
+        rm = new Mutation(keyspaceName, key.getKey());
         rm.delete(cfName, 1);
         rm.apply();
 
@@ -262,7 +262,7 @@ public class CompactionsPurgeTest extends SchemaLoader
         Util.compactAll(cfs, Integer.MAX_VALUE).get();
 
         // re-inserts with timestamp lower than delete
-        rm = new Mutation(keyspaceName, key.key);
+        rm = new Mutation(keyspaceName, key.getKey());
         for (int i = 0; i < 10; i++)
         {
             rm.add(cfName, cellname(String.valueOf(i)), ByteBufferUtil.EMPTY_BYTE_BUFFER, 0);
@@ -290,13 +290,13 @@ public class CompactionsPurgeTest extends SchemaLoader
         QueryFilter filter = QueryFilter.getIdentityFilter(key, cfName, System.currentTimeMillis());
 
         // inserts
-        rm = new Mutation(keyspaceName, key.key);
+        rm = new Mutation(keyspaceName, key.getKey());
         for (int i = 0; i < 10; i++)
             rm.add(cfName, cellname(String.valueOf(i)), ByteBufferUtil.EMPTY_BYTE_BUFFER, i);
         rm.apply();
 
         // deletes row with timestamp such that not all columns are deleted
-        rm = new Mutation(keyspaceName, key.key);
+        rm = new Mutation(keyspaceName, key.getKey());
         rm.delete(cfName, 4);
         rm.apply();
         ColumnFamily cf = cfs.getColumnFamily(filter);
@@ -308,7 +308,7 @@ public class CompactionsPurgeTest extends SchemaLoader
         assertFalse(cfs.getColumnFamily(filter).isMarkedForDelete());
 
         // re-inserts with timestamp lower than delete
-        rm = new Mutation(keyspaceName, key.key);
+        rm = new Mutation(keyspaceName, key.getKey());
         for (int i = 0; i < 5; i++)
             rm.add(cfName, cellname(String.valueOf(i)), ByteBufferUtil.EMPTY_BYTE_BUFFER, i);
         rm.apply();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/test/unit/org/apache/cassandra/db/compaction/CompactionsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/CompactionsTest.java b/test/unit/org/apache/cassandra/db/compaction/CompactionsTest.java
index a1679f5..89b6ea5 100644
--- a/test/unit/org/apache/cassandra/db/compaction/CompactionsTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/CompactionsTest.java
@@ -68,7 +68,7 @@ public class CompactionsTest extends SchemaLoader
         for (int i = 0; i < 10; i++)
         {
             DecoratedKey key = Util.dk(Integer.toString(i));
-            Mutation rm = new Mutation(KEYSPACE1, key.key);
+            Mutation rm = new Mutation(KEYSPACE1, key.getKey());
             for (int j = 0; j < 10; j++)
                 rm.add("Standard1", Util.cellname(Integer.toString(j)),
                        ByteBufferUtil.EMPTY_BYTE_BUFFER,
@@ -129,7 +129,7 @@ public class CompactionsTest extends SchemaLoader
         ByteBuffer scName = ByteBufferUtil.bytes("TestSuperColumn");
 
         // a subcolumn
-        Mutation rm = new Mutation(KEYSPACE1, key.key);
+        Mutation rm = new Mutation(KEYSPACE1, key.getKey());
         rm.add("Super1", Util.cellname(scName, ByteBufferUtil.bytes(0)),
                ByteBufferUtil.EMPTY_BYTE_BUFFER,
                FBUtilities.timestampMicros());
@@ -137,7 +137,7 @@ public class CompactionsTest extends SchemaLoader
         cfs.forceBlockingFlush();
 
         // shadow the subcolumn with a supercolumn tombstone
-        rm = new Mutation(KEYSPACE1, key.key);
+        rm = new Mutation(KEYSPACE1, key.getKey());
         rm.deleteRange("Super1", SuperColumns.startOf(scName), SuperColumns.endOf(scName), FBUtilities.timestampMicros());
         rm.apply();
         cfs.forceBlockingFlush();
@@ -179,7 +179,7 @@ public class CompactionsTest extends SchemaLoader
         for (int i=1; i < 5; i++)
         {
             DecoratedKey key = Util.dk(String.valueOf(i));
-            Mutation rm = new Mutation(KEYSPACE1, key.key);
+            Mutation rm = new Mutation(KEYSPACE1, key.getKey());
             rm.add("Standard2", Util.cellname(String.valueOf(i)), ByteBufferUtil.EMPTY_BYTE_BUFFER, i);
             rm.apply();
 
@@ -194,7 +194,7 @@ public class CompactionsTest extends SchemaLoader
         for (int i=1; i < 5; i++)
         {
             DecoratedKey key = Util.dk(String.valueOf(i));
-            Mutation rm = new Mutation(KEYSPACE1, key.key);
+            Mutation rm = new Mutation(KEYSPACE1, key.getKey());
             rm.add("Standard2", Util.cellname(String.valueOf(i)), ByteBufferUtil.EMPTY_BYTE_BUFFER, i);
             rm.apply();
         }
@@ -239,7 +239,7 @@ public class CompactionsTest extends SchemaLoader
         final int ROWS_PER_SSTABLE = 10;
         for (int i = 0; i < ROWS_PER_SSTABLE; i++) {
             DecoratedKey key = Util.dk(String.valueOf(i));
-            Mutation rm = new Mutation(KEYSPACE1, key.key);
+            Mutation rm = new Mutation(KEYSPACE1, key.getKey());
             rm.add(cfname, Util.cellname("col"),
                    ByteBufferUtil.EMPTY_BYTE_BUFFER,
                    System.currentTimeMillis());
@@ -307,7 +307,7 @@ public class CompactionsTest extends SchemaLoader
 
         // Add test row
         DecoratedKey key = Util.dk(k);
-        Mutation rm = new Mutation(KEYSPACE1, key.key);
+        Mutation rm = new Mutation(KEYSPACE1, key.getKey());
         rm.add(cfname, Util.cellname(ByteBufferUtil.bytes("sc"), ByteBufferUtil.bytes("c")), ByteBufferUtil.EMPTY_BYTE_BUFFER, 0);
         rm.apply();
 
@@ -319,7 +319,7 @@ public class CompactionsTest extends SchemaLoader
         assertTrue(cfs.getColumnFamily(filter).hasColumns());
 
         // Remove key
-        rm = new Mutation(KEYSPACE1, key.key);
+        rm = new Mutation(KEYSPACE1, key.getKey());
         rm.delete(cfname, 2);
         rm.apply();
 
@@ -361,7 +361,7 @@ public class CompactionsTest extends SchemaLoader
     {
         long timestamp = System.currentTimeMillis();
         DecoratedKey decoratedKey = Util.dk(String.format("%03d", key));
-        Mutation rm = new Mutation(KEYSPACE1, decoratedKey.key);
+        Mutation rm = new Mutation(KEYSPACE1, decoratedKey.getKey());
         rm.add("Standard1", Util.cellname("col"), ByteBufferUtil.EMPTY_BYTE_BUFFER, timestamp, 1000);
         rm.apply();
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java b/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java
index 625beac..defb087 100644
--- a/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/LeveledCompactionStrategyTest.java
@@ -83,7 +83,7 @@ public class LeveledCompactionStrategyTest extends SchemaLoader
         for (int r = 0; r < rows; r++)
         {
             DecoratedKey key = Util.dk(String.valueOf(r));
-            Mutation rm = new Mutation(ksname, key.key);
+            Mutation rm = new Mutation(ksname, key.getKey());
             for (int c = 0; c < columns; c++)
             {
                 rm.add(cfname, Util.cellname("column" + c), value, 0);
@@ -128,7 +128,7 @@ public class LeveledCompactionStrategyTest extends SchemaLoader
         for (int r = 0; r < rows; r++)
         {
             DecoratedKey key = Util.dk(String.valueOf(r));
-            Mutation rm = new Mutation(ksname, key.key);
+            Mutation rm = new Mutation(ksname, key.getKey());
             for (int c = 0; c < columns; c++)
             {
                 rm.add(cfname, Util.cellname("column" + c), value, 0);
@@ -167,7 +167,7 @@ public class LeveledCompactionStrategyTest extends SchemaLoader
         for (int r = 0; r < rows; r++)
         {
             DecoratedKey key = Util.dk(String.valueOf(r));
-            Mutation rm = new Mutation(ksname, key.key);
+            Mutation rm = new Mutation(ksname, key.getKey());
             for (int c = 0; c < columns; c++)
             {
                 rm.add(cfname, Util.cellname("column" + c), value, 0);
@@ -213,7 +213,7 @@ public class LeveledCompactionStrategyTest extends SchemaLoader
         for (int r = 0; r < rows; r++)
         {
             DecoratedKey key = Util.dk(String.valueOf(r));
-            Mutation rm = new Mutation(ksname, key.key);
+            Mutation rm = new Mutation(ksname, key.getKey());
             for (int c = 0; c < columns; c++)
             {
                 rm.add(cfname, Util.cellname("column" + c), value, 0);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/test/unit/org/apache/cassandra/db/compaction/OneCompactionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/OneCompactionTest.java b/test/unit/org/apache/cassandra/db/compaction/OneCompactionTest.java
index abd3829..bc3fe35 100644
--- a/test/unit/org/apache/cassandra/db/compaction/OneCompactionTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/OneCompactionTest.java
@@ -46,7 +46,7 @@ public class OneCompactionTest extends SchemaLoader
         Set<DecoratedKey> inserted = new HashSet<DecoratedKey>();
         for (int j = 0; j < insertsPerTable; j++) {
             DecoratedKey key = Util.dk(String.valueOf(j));
-            Mutation rm = new Mutation("Keyspace1", key.key);
+            Mutation rm = new Mutation("Keyspace1", key.getKey());
             rm.add(columnFamilyName, Util.cellname("0"), ByteBufferUtil.EMPTY_BYTE_BUFFER, j);
             rm.apply();
             inserted.add(key);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/test/unit/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategyTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategyTest.java b/test/unit/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategyTest.java
index bd4e76d..6132dad 100644
--- a/test/unit/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategyTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/SizeTieredCompactionStrategyTest.java
@@ -160,7 +160,7 @@ public class SizeTieredCompactionStrategyTest extends SchemaLoader
         for (int r = 0; r < numSSTables; r++)
         {
             DecoratedKey key = Util.dk(String.valueOf(r));
-            Mutation rm = new Mutation(ksname, key.key);
+            Mutation rm = new Mutation(ksname, key.getKey());
             rm.add(cfname, Util.cellname("column"), value, 0);
             rm.apply();
             cfs.forceBlockingFlush();
@@ -204,7 +204,7 @@ public class SizeTieredCompactionStrategyTest extends SchemaLoader
         for (int r = 0; r < numSSTables; r++)
         {
             DecoratedKey key = Util.dk(String.valueOf(r));
-            Mutation rm = new Mutation(ksname, key.key);
+            Mutation rm = new Mutation(ksname, key.getKey());
             rm.add(cfname, Util.cellname("column"), value, 0);
             rm.apply();
             cfs.forceBlockingFlush();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/test/unit/org/apache/cassandra/db/compaction/TTLExpiryTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/compaction/TTLExpiryTest.java b/test/unit/org/apache/cassandra/db/compaction/TTLExpiryTest.java
index 73aab41..b98af68 100644
--- a/test/unit/org/apache/cassandra/db/compaction/TTLExpiryTest.java
+++ b/test/unit/org/apache/cassandra/db/compaction/TTLExpiryTest.java
@@ -44,7 +44,7 @@ public class TTLExpiryTest extends SchemaLoader
         cfs.disableAutoCompaction();
         cfs.metadata.gcGraceSeconds(0);
         long timestamp = System.currentTimeMillis();
-        Mutation rm = new Mutation("Keyspace1", Util.dk("ttl").key);
+        Mutation rm = new Mutation("Keyspace1", Util.dk("ttl").getKey());
         rm.add("Standard1", Util.cellname("col"),
                ByteBufferUtil.EMPTY_BYTE_BUFFER,
                timestamp,
@@ -57,21 +57,21 @@ public class TTLExpiryTest extends SchemaLoader
         rm.apply();
         cfs.forceBlockingFlush();
 
-        rm = new Mutation("Keyspace1", Util.dk("ttl").key);
+        rm = new Mutation("Keyspace1", Util.dk("ttl").getKey());
                 rm.add("Standard1", Util.cellname("col2"),
                        ByteBufferUtil.EMPTY_BYTE_BUFFER,
                        timestamp,
                        1);
                 rm.apply();
         cfs.forceBlockingFlush();
-        rm = new Mutation("Keyspace1", Util.dk("ttl").key);
+        rm = new Mutation("Keyspace1", Util.dk("ttl").getKey());
         rm.add("Standard1", Util.cellname("col3"),
                    ByteBufferUtil.EMPTY_BYTE_BUFFER,
                    timestamp,
                    1);
         rm.apply();
         cfs.forceBlockingFlush();
-        rm = new Mutation("Keyspace1", Util.dk("ttl").key);
+        rm = new Mutation("Keyspace1", Util.dk("ttl").getKey());
         rm.add("Standard1", Util.cellname("col311"),
                    ByteBufferUtil.EMPTY_BYTE_BUFFER,
                    timestamp,
@@ -92,7 +92,7 @@ public class TTLExpiryTest extends SchemaLoader
         cfs.disableAutoCompaction();
         cfs.metadata.gcGraceSeconds(0);
         long timestamp = System.currentTimeMillis();
-        Mutation rm = new Mutation("Keyspace1", Util.dk("ttl").key);
+        Mutation rm = new Mutation("Keyspace1", Util.dk("ttl").getKey());
         rm.add("Standard1", Util.cellname("col"),
                ByteBufferUtil.EMPTY_BYTE_BUFFER,
                timestamp,
@@ -105,14 +105,14 @@ public class TTLExpiryTest extends SchemaLoader
         rm.apply();
         cfs.forceBlockingFlush();
 
-        rm = new Mutation("Keyspace1", Util.dk("ttl").key);
+        rm = new Mutation("Keyspace1", Util.dk("ttl").getKey());
                 rm.add("Standard1", Util.cellname("col2"),
                        ByteBufferUtil.EMPTY_BYTE_BUFFER,
                        timestamp,
                        1);
                 rm.apply();
         cfs.forceBlockingFlush();
-        rm = new Mutation("Keyspace1", Util.dk("ttl").key);
+        rm = new Mutation("Keyspace1", Util.dk("ttl").getKey());
         rm.add("Standard1", Util.cellname("col3"),
                    ByteBufferUtil.EMPTY_BYTE_BUFFER,
                    timestamp,
@@ -120,7 +120,7 @@ public class TTLExpiryTest extends SchemaLoader
         rm.apply();
         cfs.forceBlockingFlush();
         DecoratedKey noTTLKey = Util.dk("nottl");
-        rm = new Mutation("Keyspace1", noTTLKey.key);
+        rm = new Mutation("Keyspace1", noTTLKey.getKey());
         rm.add("Standard1", Util.cellname("col311"),
                    ByteBufferUtil.EMPTY_BYTE_BUFFER,
                    timestamp);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/test/unit/org/apache/cassandra/db/context/CounterContextTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/context/CounterContextTest.java b/test/unit/org/apache/cassandra/db/context/CounterContextTest.java
index bc297ab..a72d30d 100644
--- a/test/unit/org/apache/cassandra/db/context/CounterContextTest.java
+++ b/test/unit/org/apache/cassandra/db/context/CounterContextTest.java
@@ -29,9 +29,6 @@ import org.apache.cassandra.db.ClockAndCount;
 import org.apache.cassandra.db.context.CounterContext.Relationship;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.CounterId;
-import org.apache.cassandra.utils.memory.AbstractAllocator;
-import org.apache.cassandra.utils.memory.Pool;
-import org.apache.cassandra.utils.memory.SlabPool;
 
 import static org.apache.cassandra.db.context.CounterContext.ContextState;
 import static org.junit.Assert.assertEquals;
@@ -51,16 +48,6 @@ public class CounterContextTest
     private static final int countLength = 8;
     private static final int stepLength = idLength + clockLength + countLength;
 
-    private static final Pool POOL = new SlabPool(Integer.MAX_VALUE, 0, 1f, null);
-
-    /** Allocates 1 byte from a new SlabAllocator and returns it. */
-    private AbstractAllocator bumpedSlab()
-    {
-        AbstractAllocator allocator = POOL.newAllocator();
-        allocator.allocate(1);
-        return allocator;
-    }
-
     @Test
     public void testAllocate()
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/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 c57ba05..7eb899f 100644
--- a/test/unit/org/apache/cassandra/db/index/PerRowSecondaryIndexTest.java
+++ b/test/unit/org/apache/cassandra/db/index/PerRowSecondaryIndexTest.java
@@ -38,7 +38,7 @@ import org.apache.cassandra.db.filter.QueryFilter;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.concurrent.OpOrder;
-import org.apache.cassandra.utils.memory.PoolAllocator;
+import org.apache.cassandra.utils.memory.MemtableAllocator;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
@@ -179,12 +179,6 @@ public class PerRowSecondaryIndexTest extends SchemaLoader
         }
 
         @Override
-        public PoolAllocator getAllocator()
-        {
-            return null;
-        }
-
-        @Override
         public ColumnFamilyStore getIndexCfs()
         {
             return null;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/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 9b2b492..5281449 100644
--- a/test/unit/org/apache/cassandra/io/sstable/IndexSummaryManagerTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/IndexSummaryManagerTest.java
@@ -135,7 +135,7 @@ public class IndexSummaryManagerTest extends SchemaLoader
             for (int row = 0; row < numRows; row++)
             {
                 DecoratedKey key = Util.dk(String.format("%3d", row));
-                Mutation rm = new Mutation(ksname, key.key);
+                Mutation rm = new Mutation(ksname, key.getKey());
                 rm.add(cfname, Util.cellname("column"), value, 0);
                 rm.applyUnsafe();
             }
@@ -408,7 +408,7 @@ public class IndexSummaryManagerTest extends SchemaLoader
         for (int row = 0; row < numRows; row++)
         {
             DecoratedKey key = Util.dk(String.valueOf(row));
-            Mutation rm = new Mutation(ksname, key.key);
+            Mutation rm = new Mutation(ksname, key.getKey());
             rm.add(cfname, Util.cellname("column"), value, 0);
             rm.apply();
         }
@@ -468,7 +468,7 @@ public class IndexSummaryManagerTest extends SchemaLoader
             for (int row = 0; row < numRows; row++)
             {
                 DecoratedKey key = Util.dk(String.valueOf(row));
-                Mutation rm = new Mutation(ksname, key.key);
+                Mutation rm = new Mutation(ksname, key.getKey());
                 rm.add(cfname, Util.cellname("column"), value, 0);
                 rm.apply();
             }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/test/unit/org/apache/cassandra/io/sstable/IndexSummaryTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/IndexSummaryTest.java b/test/unit/org/apache/cassandra/io/sstable/IndexSummaryTest.java
index edaa5ba..9aca66d 100644
--- a/test/unit/org/apache/cassandra/io/sstable/IndexSummaryTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/IndexSummaryTest.java
@@ -18,9 +18,7 @@
 package org.apache.cassandra.io.sstable;
 
 import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
 import java.io.DataInputStream;
-import java.io.DataOutputStream;
 import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.*;
@@ -50,7 +48,7 @@ public class IndexSummaryTest
     {
         Pair<List<DecoratedKey>, IndexSummary> random = generateRandomIndex(100, 1);
         for (int i = 0; i < 100; i++)
-            assertEquals(random.left.get(i).key, ByteBuffer.wrap(random.right.getKey(i)));
+            assertEquals(random.left.get(i).getKey(), ByteBuffer.wrap(random.right.getKey(i)));
     }
 
     @Test
@@ -163,7 +161,7 @@ public class IndexSummaryTest
 
         // sanity check on the original index summary
         for (int i = 0; i < ORIGINAL_NUM_ENTRIES; i++)
-            assertEquals(keys.get(i * INDEX_INTERVAL).key, ByteBuffer.wrap(original.getKey(i)));
+            assertEquals(keys.get(i * INDEX_INTERVAL).getKey(), ByteBuffer.wrap(original.getKey(i)));
 
         List<Integer> samplePattern = Downsampling.getSamplingPattern(BASE_SAMPLING_LEVEL);
 
@@ -180,7 +178,7 @@ public class IndexSummaryTest
             {
                 if (!shouldSkip(i, skipStartPoints))
                 {
-                    assertEquals(keys.get(i * INDEX_INTERVAL).key, ByteBuffer.wrap(downsampled.getKey(sampledCount)));
+                    assertEquals(keys.get(i * INDEX_INTERVAL).getKey(), ByteBuffer.wrap(downsampled.getKey(sampledCount)));
                     sampledCount++;
                 }
             }
@@ -201,7 +199,7 @@ public class IndexSummaryTest
             {
                 if (!shouldSkip(i, skipStartPoints))
                 {
-                    assertEquals(keys.get(i * INDEX_INTERVAL).key, ByteBuffer.wrap(downsampled.getKey(sampledCount)));
+                    assertEquals(keys.get(i * INDEX_INTERVAL).getKey(), ByteBuffer.wrap(downsampled.getKey(sampledCount)));
                     sampledCount++;
                 }
             }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/test/unit/org/apache/cassandra/io/sstable/SSTableLoaderTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableLoaderTest.java b/test/unit/org/apache/cassandra/io/sstable/SSTableLoaderTest.java
index 18beaa9..39beb94 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableLoaderTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableLoaderTest.java
@@ -61,7 +61,7 @@ public class SSTableLoaderTest extends SchemaLoader
                                                                              StorageService.getPartitioner(),
                                                                              1);
         DecoratedKey key = Util.dk("key1");
-        writer.newRow(key.key);
+        writer.newRow(key.getKey());
         writer.addColumn(ByteBufferUtil.bytes("col1"), ByteBufferUtil.bytes(100), 1);
         writer.close();
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/test/unit/org/apache/cassandra/io/sstable/SSTableMetadataTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableMetadataTest.java b/test/unit/org/apache/cassandra/io/sstable/SSTableMetadataTest.java
index 1624a6b..72307c5 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableMetadataTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableMetadataTest.java
@@ -52,7 +52,7 @@ public class SSTableMetadataTest extends SchemaLoader
         for(int i = 0; i < 10; i++)
         {
             DecoratedKey key = Util.dk(Integer.toString(i));
-            Mutation rm = new Mutation("Keyspace1", key.key);
+            Mutation rm = new Mutation("Keyspace1", key.getKey());
             for (int j = 0; j < 10; j++)
                 rm.add("Standard1", cellname(Integer.toString(j)),
                        ByteBufferUtil.EMPTY_BYTE_BUFFER,
@@ -60,7 +60,7 @@ public class SSTableMetadataTest extends SchemaLoader
                        10 + j);
             rm.apply();
         }
-        Mutation rm = new Mutation("Keyspace1", Util.dk("longttl").key);
+        Mutation rm = new Mutation("Keyspace1", Util.dk("longttl").getKey());
         rm.add("Standard1", cellname("col"),
                ByteBufferUtil.EMPTY_BYTE_BUFFER,
                timestamp,
@@ -76,7 +76,7 @@ public class SSTableMetadataTest extends SchemaLoader
             assertEquals(ttltimestamp + 10000, firstDelTime, 10);
 
         }
-        rm = new Mutation("Keyspace1", Util.dk("longttl2").key);
+        rm = new Mutation("Keyspace1", Util.dk("longttl2").getKey());
         rm.add("Standard1", cellname("col"),
                ByteBufferUtil.EMPTY_BYTE_BUFFER,
                timestamp,
@@ -123,7 +123,7 @@ public class SSTableMetadataTest extends SchemaLoader
         ColumnFamilyStore store = keyspace.getColumnFamilyStore("Standard2");
         long timestamp = System.currentTimeMillis();
         DecoratedKey key = Util.dk("deletetest");
-        Mutation rm = new Mutation("Keyspace1", key.key);
+        Mutation rm = new Mutation("Keyspace1", key.getKey());
         for (int i = 0; i<5; i++)
             rm.add("Standard2", cellname("deletecolumn" + i),
                        ByteBufferUtil.EMPTY_BYTE_BUFFER,
@@ -143,7 +143,7 @@ public class SSTableMetadataTest extends SchemaLoader
             firstMaxDelTime = sstable.getSSTableMetadata().maxLocalDeletionTime;
             assertEquals(ttltimestamp + 1000, firstMaxDelTime, 10);
         }
-        rm = new Mutation("Keyspace1", key.key);
+        rm = new Mutation("Keyspace1", key.getKey());
         rm.delete("Standard2", cellname("todelete"), timestamp + 1);
         rm.apply();
         store.forceBlockingFlush();
@@ -175,7 +175,7 @@ public class SSTableMetadataTest extends SchemaLoader
         for (int j = 0; j < 8; j++)
         {
             DecoratedKey key = Util.dk("row"+j);
-            Mutation rm = new Mutation("Keyspace1", key.key);
+            Mutation rm = new Mutation("Keyspace1", key.getKey());
             for (int i = 100; i<150; i++)
             {
                 rm.add("Standard3", cellname(j + "col" + i), ByteBufferUtil.EMPTY_BYTE_BUFFER, System.currentTimeMillis());
@@ -190,7 +190,7 @@ public class SSTableMetadataTest extends SchemaLoader
             assertEquals(ByteBufferUtil.string(sstable.getSSTableMetadata().maxColumnNames.get(0)), "7col149");
         }
         DecoratedKey key = Util.dk("row2");
-        Mutation rm = new Mutation("Keyspace1", key.key);
+        Mutation rm = new Mutation("Keyspace1", key.getKey());
         for (int i = 101; i<299; i++)
         {
             rm.add("Standard3", cellname(9 + "col" + i), ByteBufferUtil.EMPTY_BYTE_BUFFER, System.currentTimeMillis());
@@ -268,8 +268,8 @@ public class SSTableMetadataTest extends SchemaLoader
         state.writeLocal(CounterId.fromInt(2), 1L, 1L);
         state.writeRemote(CounterId.fromInt(3), 1L, 1L);
         ColumnFamily cells = ArrayBackedSortedColumns.factory.create(cfs.metadata);
-        cells.addColumn(new CounterCell(cellname("col"), state.context, 1L, Long.MIN_VALUE));
-        new Mutation(Util.dk("k").key, cells).apply();
+        cells.addColumn(new BufferCounterCell(cellname("col"), state.context, 1L, Long.MIN_VALUE));
+        new Mutation(Util.dk("k").getKey(), cells).apply();
         cfs.forceBlockingFlush();
         assertTrue(cfs.getSSTables().iterator().next().getSSTableMetadata().hasLegacyCounterShards);
         cfs.truncateBlocking();
@@ -279,8 +279,8 @@ public class SSTableMetadataTest extends SchemaLoader
         state.writeLocal(CounterId.fromInt(2), 1L, 1L);
         state.writeRemote(CounterId.fromInt(3), 1L, 1L);
         cells = ArrayBackedSortedColumns.factory.create(cfs.metadata);
-        cells.addColumn(new CounterCell(cellname("col"), state.context, 1L, Long.MIN_VALUE));
-        new Mutation(Util.dk("k").key, cells).apply();
+        cells.addColumn(new BufferCounterCell(cellname("col"), state.context, 1L, Long.MIN_VALUE));
+        new Mutation(Util.dk("k").getKey(), cells).apply();
         cfs.forceBlockingFlush();
         assertTrue(cfs.getSSTables().iterator().next().getSSTableMetadata().hasLegacyCounterShards);
         cfs.truncateBlocking();
@@ -290,8 +290,8 @@ public class SSTableMetadataTest extends SchemaLoader
         state.writeGlobal(CounterId.fromInt(1), 1L, 1L);
         state.writeLocal(CounterId.fromInt(2), 1L, 1L);
         cells = ArrayBackedSortedColumns.factory.create(cfs.metadata);
-        cells.addColumn(new CounterCell(cellname("col"), state.context, 1L, Long.MIN_VALUE));
-        new Mutation(Util.dk("k").key, cells).apply();
+        cells.addColumn(new BufferCounterCell(cellname("col"), state.context, 1L, Long.MIN_VALUE));
+        new Mutation(Util.dk("k").getKey(), cells).apply();
         cfs.forceBlockingFlush();
         assertTrue(cfs.getSSTables().iterator().next().getSSTableMetadata().hasLegacyCounterShards);
         cfs.truncateBlocking();
@@ -300,8 +300,8 @@ public class SSTableMetadataTest extends SchemaLoader
         state = CounterContext.ContextState.allocate(1, 0, 0);
         state.writeGlobal(CounterId.fromInt(1), 1L, 1L);
         cells = ArrayBackedSortedColumns.factory.create(cfs.metadata);
-        cells.addColumn(new CounterCell(cellname("col"), state.context, 1L, Long.MIN_VALUE));
-        new Mutation(Util.dk("k").key, cells).apply();
+        cells.addColumn(new BufferCounterCell(cellname("col"), state.context, 1L, Long.MIN_VALUE));
+        new Mutation(Util.dk("k").getKey(), cells).apply();
         cfs.forceBlockingFlush();
         assertFalse(cfs.getSSTables().iterator().next().getSSTableMetadata().hasLegacyCounterShards);
         cfs.truncateBlocking();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java b/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
index 8429d37..91f5341 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
@@ -44,6 +44,7 @@ import org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.SchemaLoader;
 import org.apache.cassandra.Util;
 import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.BufferDecoratedKey;
 import org.apache.cassandra.db.ColumnFamily;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.DecoratedKey;
@@ -257,9 +258,9 @@ public class SSTableReaderTest extends SchemaLoader
                 firstKey = key;
             if (lastKey == null)
                 lastKey = key;
-            if (store.metadata.getKeyValidator().compare(lastKey.key, key.key) < 0)
+            if (store.metadata.getKeyValidator().compare(lastKey.getKey(), key.getKey()) < 0)
                 lastKey = key;
-            Mutation rm = new Mutation(ks, key.key);
+            Mutation rm = new Mutation(ks, key.getKey());
             rm.add(cf, cellname("col"),
                    ByteBufferUtil.EMPTY_BYTE_BUFFER, timestamp);
             rm.apply();
@@ -272,7 +273,7 @@ public class SSTableReaderTest extends SchemaLoader
         // test to see if sstable can be opened as expected
         SSTableReader target = SSTableReader.open(desc);
         Assert.assertEquals(target.getIndexSummarySize(), 1);
-        Assert.assertArrayEquals(ByteBufferUtil.getArray(firstKey.key), target.getIndexSummaryKey(0));
+        Assert.assertArrayEquals(ByteBufferUtil.getArray(firstKey.getKey()), target.getIndexSummaryKey(0));
         assert target.first.equals(firstKey);
         assert target.last.equals(lastKey);
     }
@@ -291,7 +292,7 @@ public class SSTableReaderTest extends SchemaLoader
         ColumnFamilyStore indexCfs = store.indexManager.getIndexForColumn(ByteBufferUtil.bytes("birthdate")).getIndexCfs();
         assert indexCfs.partitioner instanceof LocalPartitioner;
         SSTableReader sstable = indexCfs.getSSTables().iterator().next();
-        assert sstable.first.token instanceof LocalToken;
+        assert sstable.first.getToken() instanceof LocalToken;
 
         SegmentedFile.Builder ibuilder = SegmentedFile.getBuilder(DatabaseDescriptor.getIndexAccessMode());
         SegmentedFile.Builder dbuilder = sstable.compression
@@ -300,7 +301,7 @@ public class SSTableReaderTest extends SchemaLoader
         sstable.saveSummary(ibuilder, dbuilder);
 
         SSTableReader reopened = SSTableReader.open(sstable.descriptor);
-        assert reopened.first.token instanceof LocalToken;
+        assert reopened.first.getToken() instanceof LocalToken;
     }
 
     /** see CASSANDRA-5407 */
@@ -441,6 +442,6 @@ public class SSTableReaderTest extends SchemaLoader
 
     private DecoratedKey k(int i)
     {
-        return new DecoratedKey(t(i), ByteBufferUtil.bytes(String.valueOf(i)));
+        return new BufferDecoratedKey(t(i), ByteBufferUtil.bytes(String.valueOf(i)));
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/test/unit/org/apache/cassandra/io/sstable/SSTableScannerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableScannerTest.java b/test/unit/org/apache/cassandra/io/sstable/SSTableScannerTest.java
index 7255705..ff1a305 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableScannerTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableScannerTest.java
@@ -71,7 +71,7 @@ public class SSTableScannerTest extends SchemaLoader
     {
         long timestamp = System.currentTimeMillis();
         DecoratedKey decoratedKey = Util.dk(toKey(key));
-        Mutation rm = new Mutation(KEYSPACE, decoratedKey.key);
+        Mutation rm = new Mutation(KEYSPACE, decoratedKey.getKey());
         rm.add(TABLE, Util.cellname("col"), ByteBufferUtil.EMPTY_BYTE_BUFFER, timestamp, 1000);
         rm.apply();
     }
@@ -80,7 +80,7 @@ public class SSTableScannerTest extends SchemaLoader
     {
         SSTableScanner scanner = sstable.getScanner(new DataRange(boundsFor(scanStart, scanEnd), new IdentityQueryFilter()));
         for (int i = expectedStart; i <= expectedEnd; i++)
-            assertEquals(toKey(i), new String(scanner.next().getKey().key.array()));
+            assertEquals(toKey(i), new String(scanner.next().getKey().getKey().array()));
         assertFalse(scanner.hasNext());
     }
 
@@ -110,7 +110,7 @@ public class SSTableScannerTest extends SchemaLoader
         // full range scan
         SSTableScanner scanner = sstable.getScanner();
         for (int i = 2; i < 10; i++)
-            assertEquals(toKey(i), new String(scanner.next().getKey().key.array()));
+            assertEquals(toKey(i), new String(scanner.next().getKey().getKey().array()));
 
         // a simple read of a chunk in the middle
         assertScanMatches(sstable, 3, 6, 3, 6);
@@ -147,7 +147,7 @@ public class SSTableScannerTest extends SchemaLoader
             for (int expected = rangeStart; expected <= rangeEnd; expected++)
             {
                 assertTrue(String.format("Expected to see key %03d", expected), scanner.hasNext());
-                assertEquals(toKey(expected), new String(scanner.next().getKey().key.array()));
+                assertEquals(toKey(expected), new String(scanner.next().getKey().getKey().array()));
             }
         }
         assertFalse(scanner.hasNext());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/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 19a0b13..d39f968 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableUtils.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableUtils.java
@@ -164,7 +164,7 @@ public class SSTableUtils
             for (String key : keys)
             {
                 ColumnFamily cf = ArrayBackedSortedColumns.factory.create(ksname, cfname);
-                cf.addColumn(new Cell(Util.cellname(key), ByteBufferUtil.bytes(key), 0));
+                cf.addColumn(new BufferCell(Util.cellname(key), ByteBufferUtil.bytes(key), 0));
                 map.put(key, cf);
             }
             return write(map);


[5/7] Push more of memtable data off-heap

Posted by sl...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/db/NativeDeletedCell.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/NativeDeletedCell.java b/src/java/org/apache/cassandra/db/NativeDeletedCell.java
new file mode 100644
index 0000000..8bfc95b
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/NativeDeletedCell.java
@@ -0,0 +1,125 @@
+/*
+ * 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 org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.serializers.MarshalException;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.ObjectSizes;
+import org.apache.cassandra.utils.concurrent.OpOrder;
+import org.apache.cassandra.utils.memory.AbstractAllocator;
+import org.apache.cassandra.utils.memory.MemoryUtil;
+import org.apache.cassandra.utils.memory.MemtableAllocator;
+import org.apache.cassandra.utils.memory.NativeAllocator;
+
+public class NativeDeletedCell extends NativeCell implements DeletedCell
+{
+    private static final long SIZE = ObjectSizes.measure(new NativeDeletedCell());
+
+    private NativeDeletedCell()
+    {}
+
+    public NativeDeletedCell(NativeAllocator allocator, OpOrder.Group writeOp, DeletedCell copyOf)
+    {
+        super(allocator, writeOp, copyOf);
+    }
+
+    @Override
+    public Cell reconcile(Cell cell)
+    {
+        if (cell instanceof DeletedCell)
+            return super.reconcile(cell);
+        return cell.reconcile(this);
+    }
+
+    @Override
+    public boolean isMarkedForDelete(long now)
+    {
+        return true;
+    }
+
+    @Override
+    public long getMarkedForDeleteAt()
+    {
+        return timestamp();
+    }
+
+    @Override
+    public int getLocalDeletionTime()
+    {
+        int v = getInt(valueStartOffset());
+        return MemoryUtil.INVERTED_ORDER ? Integer.reverseBytes(v) : v;
+    }
+
+    @Override
+    public int serializationFlags()
+    {
+        return ColumnSerializer.DELETION_MASK;
+    }
+
+    @Override
+    public void validateFields(CFMetaData metadata) throws MarshalException
+    {
+        validateName(metadata);
+
+        if ((int) (internalSize() - valueStartOffset()) != 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");
+    }
+
+    @Override
+    public void updateDigest(MessageDigest digest)
+    {
+        updateWithName(digest);
+        FBUtilities.updateWithLong(digest, timestamp());
+        FBUtilities.updateWithByte(digest, serializationFlags());
+    }
+
+    @Override
+    public DeletedCell localCopy(CFMetaData metadata, AbstractAllocator allocator)
+    {
+        return new BufferDeletedCell(copy(metadata, allocator), allocator.clone(value()), timestamp());
+    }
+
+    @Override
+    public DeletedCell localCopy(CFMetaData metadata, MemtableAllocator allocator, OpOrder.Group opGroup)
+    {
+        return allocator.clone(this, metadata, opGroup);
+    }
+
+    @Override
+    public boolean equals(Cell cell)
+    {
+        return timestamp() == cell.timestamp() && getLocalDeletionTime() == cell.getLocalDeletionTime() && name().equals(cell.name());
+    }
+
+    @Override
+    public long excessHeapSizeExcludingData()
+    {
+        return SIZE;
+    }
+
+    @Override
+    public long unsharedHeapSize()
+    {
+        return SIZE;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/db/NativeExpiringCell.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/NativeExpiringCell.java b/src/java/org/apache/cassandra/db/NativeExpiringCell.java
new file mode 100644
index 0000000..5ac0e81
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/NativeExpiringCell.java
@@ -0,0 +1,173 @@
+/*
+ * 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 org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.db.composites.CellNameType;
+import org.apache.cassandra.serializers.MarshalException;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.ObjectSizes;
+import org.apache.cassandra.utils.concurrent.OpOrder;
+import org.apache.cassandra.utils.memory.AbstractAllocator;
+import org.apache.cassandra.utils.memory.MemtableAllocator;
+import org.apache.cassandra.utils.memory.NativeAllocator;
+
+public class NativeExpiringCell extends NativeCell implements ExpiringCell
+{
+    private static final long SIZE = ObjectSizes.measure(new NativeExpiringCell());
+
+    private NativeExpiringCell()
+    {}
+
+    public NativeExpiringCell(NativeAllocator allocator, OpOrder.Group writeOp, ExpiringCell copyOf)
+    {
+        super(allocator, writeOp, copyOf);
+    }
+
+    @Override
+    protected int sizeOf(Cell cell)
+    {
+        return super.sizeOf(cell) + 8;
+    }
+
+    @Override
+    protected void construct(Cell from)
+    {
+        ExpiringCell expiring = (ExpiringCell) from;
+
+        setInt(internalSize() - 4, expiring.getTimeToLive());
+        setInt(internalSize() - 8, expiring.getLocalDeletionTime());
+        super.construct(from);
+    }
+
+    @Override
+    protected int postfixSize()
+    {
+        return 8;
+    }
+
+    @Override
+    public int getTimeToLive()
+    {
+        return getInt(internalSize() - 4);
+    }
+
+    @Override
+    public int getLocalDeletionTime()
+    {
+        return getInt(internalSize() - 8);
+    }
+
+    @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 int cellDataSize()
+    {
+        return super.cellDataSize() + TypeSizes.NATIVE.sizeof(getLocalDeletionTime()) + TypeSizes.NATIVE.sizeof(getTimeToLive());
+    }
+
+    @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(getLocalDeletionTime()) + typeSizes.sizeof(getTimeToLive());
+    }
+
+    @Override
+    public void validateFields(CFMetaData metadata) throws MarshalException
+    {
+        super.validateFields(metadata);
+
+        if (getTimeToLive() <= 0)
+            throw new MarshalException("A column TTL should be > 0");
+        if (getLocalDeletionTime() < 0)
+            throw new MarshalException("The local expiration time should not be negative");
+    }
+
+    @Override
+    public void updateDigest(MessageDigest digest)
+    {
+        super.updateDigest(digest);
+        FBUtilities.updateWithInt(digest, getTimeToLive());
+    }
+
+    public boolean equals(Cell cell)
+    {
+        return cell instanceof ExpiringCell && equals((ExpiringCell) this);
+    }
+
+    protected boolean equals(ExpiringCell cell)
+    {
+        // super.equals() returns false if o is not a CounterCell
+        return super.equals(cell)
+                && getLocalDeletionTime() == cell.getLocalDeletionTime()
+                && getTimeToLive() == cell.getTimeToLive();
+    }
+
+    @Override
+    public String getString(CellNameType comparator)
+    {
+        return String.format("%s(%s!%d)", getClass().getSimpleName(), super.getString(comparator), getTimeToLive());
+    }
+
+    @Override
+    public ExpiringCell localCopy(CFMetaData metadata, AbstractAllocator allocator)
+    {
+        return new BufferExpiringCell(name().copy(metadata, allocator), allocator.clone(value()), timestamp(), getTimeToLive(), getLocalDeletionTime());
+    }
+
+    @Override
+    public ExpiringCell localCopy(CFMetaData metadata, MemtableAllocator allocator, OpOrder.Group opGroup)
+    {
+        return allocator.clone(this, metadata, opGroup);
+    }
+
+    @Override
+    public long excessHeapSizeExcludingData()
+    {
+        return SIZE;
+    }
+
+    @Override
+    public long unsharedHeapSize()
+    {
+        return SIZE;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/db/Row.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Row.java b/src/java/org/apache/cassandra/db/Row.java
index 0685116..a826894 100644
--- a/src/java/org/apache/cassandra/db/Row.java
+++ b/src/java/org/apache/cassandra/db/Row.java
@@ -64,7 +64,7 @@ public class Row
     {
         public void serialize(Row row, DataOutputPlus out, int version) throws IOException
         {
-            ByteBufferUtil.writeWithShortLength(row.key.key, out);
+            ByteBufferUtil.writeWithShortLength(row.key.getKey(), out);
             ColumnFamily.serializer.serialize(row.cf, out, version);
         }
 
@@ -81,7 +81,7 @@ public class Row
 
         public long serializedSize(Row row, int version)
         {
-            int keySize = row.key.key.remaining();
+            int keySize = row.key.getKey().remaining();
             return TypeSizes.NATIVE.sizeof((short) keySize) + keySize + ColumnFamily.serializer.serializedSize(row.cf, TypeSizes.NATIVE, version);
         }
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/db/RowIteratorFactory.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/RowIteratorFactory.java b/src/java/org/apache/cassandra/db/RowIteratorFactory.java
index 464b9d1..500291e 100644
--- a/src/java/org/apache/cassandra/db/RowIteratorFactory.java
+++ b/src/java/org/apache/cassandra/db/RowIteratorFactory.java
@@ -96,7 +96,7 @@ public class RowIteratorFactory
             {
                 // First check if this row is in the rowCache. If it is and it covers our filter, we can skip the rest
                 ColumnFamily cached = cfs.getRawCachedRow(key);
-                IDiskAtomFilter filter = range.columnFilter(key.key);
+                IDiskAtomFilter filter = range.columnFilter(key.getKey());
 
                 if (cached == null || !cfs.isFilterFullyCoveredBy(filter, cached, now))
                 {
@@ -150,7 +150,7 @@ public class RowIteratorFactory
             {
                 public OnDiskAtomIterator create()
                 {
-                    return range.columnFilter(entry.getKey().key).getColumnFamilyIterator(entry.getKey(), entry.getValue());
+                    return range.columnFilter(entry.getKey().getKey()).getColumnFamilyIterator(entry.getKey(), entry.getValue());
                 }
             });
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/db/RowPosition.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/RowPosition.java b/src/java/org/apache/cassandra/db/RowPosition.java
index a665d62..3bcd627 100644
--- a/src/java/org/apache/cassandra/db/RowPosition.java
+++ b/src/java/org/apache/cassandra/db/RowPosition.java
@@ -27,7 +27,7 @@ import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
-public abstract class RowPosition implements RingPosition<RowPosition>
+public interface RowPosition extends RingPosition<RowPosition>
 {
     public static enum Kind
     {
@@ -43,20 +43,18 @@ public abstract class RowPosition implements RingPosition<RowPosition>
         }
     }
 
-    public static final RowPositionSerializer serializer = new RowPositionSerializer();
-
-    public static RowPosition forKey(ByteBuffer key, IPartitioner p)
+    public static final class ForKey
     {
-        return key == null || key.remaining() == 0 ? p.getMinimumToken().minKeyBound() : p.decorateKey(key);
+        public static RowPosition get(ByteBuffer key, IPartitioner p)
+        {
+            return key == null || key.remaining() == 0 ? p.getMinimumToken().minKeyBound() : p.decorateKey(key);
+        }
     }
 
-    public abstract Token getToken();
-    public abstract Kind kind();
+    public static final RowPositionSerializer serializer = new RowPositionSerializer();
 
-    public boolean isMinimum()
-    {
-        return isMinimum(StorageService.getPartitioner());
-    }
+    public Kind kind();
+    public boolean isMinimum();
 
     public static class RowPositionSerializer implements ISerializer<RowPosition>
     {
@@ -76,7 +74,7 @@ public abstract class RowPosition implements RingPosition<RowPosition>
             Kind kind = pos.kind();
             out.writeByte(kind.ordinal());
             if (kind == Kind.ROW_KEY)
-                ByteBufferUtil.writeWithShortLength(((DecoratedKey)pos).key, out);
+                ByteBufferUtil.writeWithShortLength(((DecoratedKey)pos).getKey(), out);
             else
                 Token.serializer.serialize(pos.getToken(), out);
         }
@@ -102,7 +100,7 @@ public abstract class RowPosition implements RingPosition<RowPosition>
             int size = 1; // 1 byte for enum
             if (kind == Kind.ROW_KEY)
             {
-                int keySize = ((DecoratedKey)pos).key.remaining();
+                int keySize = ((DecoratedKey)pos).getKey().remaining();
                 size += typeSizes.sizeof((short) keySize) + keySize;
             }
             else

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/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 b19eb1e..2025d5e 100644
--- a/src/java/org/apache/cassandra/db/SystemKeyspace.java
+++ b/src/java/org/apache/cassandra/db/SystemKeyspace.java
@@ -696,7 +696,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 Cell(cf.getComparator().makeCellName(indexName), ByteBufferUtil.EMPTY_BYTE_BUFFER, FBUtilities.timestampMicros()));
+        cf.addColumn(new BufferCell(cf.getComparator().makeCellName(indexName), ByteBufferUtil.EMPTY_BYTE_BUFFER, FBUtilities.timestampMicros()));
         new Mutation(Keyspace.SYSTEM_KS, ByteBufferUtil.bytes(keyspaceName), cf).apply();
     }
 
@@ -774,7 +774,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 Cell(cf.getComparator().makeCellName(newCounterId.bytes()), ip, now));
+        cf.addColumn(new BufferCell(cf.getComparator().makeCellName(newCounterId.bytes()), ip, now));
         new Mutation(Keyspace.SYSTEM_KS, ALL_LOCAL_NODE_ID_KEY, cf).apply();
         forceBlockingFlush(COUNTER_ID_CF);
     }
@@ -833,7 +833,7 @@ public class SystemKeyspace
             Mutation mutation = mutationMap.get(schemaRow.key);
             if (mutation == null)
             {
-                mutation = new Mutation(Keyspace.SYSTEM_KS, schemaRow.key.key);
+                mutation = new Mutation(Keyspace.SYSTEM_KS, schemaRow.key.getKey());
                 mutationMap.put(schemaRow.key, mutation);
             }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java b/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java
index 0691819..0d45e9c 100644
--- a/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java
+++ b/src/java/org/apache/cassandra/db/compaction/AbstractCompactionStrategy.java
@@ -315,7 +315,7 @@ public abstract class AbstractCompactionStrategy
             long keys = sstable.estimatedKeys();
             Set<Range<Token>> ranges = new HashSet<Range<Token>>(overlaps.size());
             for (SSTableReader overlap : overlaps)
-                ranges.add(new Range<Token>(overlap.first.token, overlap.last.token, overlap.partitioner));
+                ranges.add(new Range<Token>(overlap.first.getToken(), overlap.last.getToken(), overlap.partitioner));
             long remainingKeys = keys - sstable.estimatedKeysForRanges(ranges);
             // next, calculate what percentage of columns we have within those keys
             long columns = sstable.getEstimatedColumnCount().mean() * remainingKeys;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/db/compaction/CompactionController.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionController.java b/src/java/org/apache/cassandra/db/compaction/CompactionController.java
index 3fe5c26..2bd9c86 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionController.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionController.java
@@ -156,7 +156,7 @@ public class CompactionController implements AutoCloseable
             // we check index file instead.
             if (sstable.getBloomFilter() instanceof AlwaysPresentFilter && sstable.getPosition(key, SSTableReader.Operator.EQ, false) != null)
                 min = Math.min(min, sstable.getMinTimestamp());
-            else if (sstable.getBloomFilter().isPresent(key.key))
+            else if (sstable.getBloomFilter().isPresent(key.getKey()))
                 min = Math.min(min, sstable.getMinTimestamp());
         }
         return min;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/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 792c962..3c9b004 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
@@ -406,7 +406,7 @@ public class CompactionManager implements CompactionManagerMBean
             SSTableReader sstable = sstableIterator.next();
             for (Range<Token> r : Range.normalize(ranges))
             {
-                Range<Token> sstableRange = new Range<>(sstable.first.token, sstable.last.token, sstable.partitioner);
+                Range<Token> sstableRange = new Range<>(sstable.first.getToken(), sstable.last.getToken(), sstable.partitioner);
                 if (r.contains(sstableRange))
                 {
                     logger.info("SSTable {} fully contained in range {}, mutating repairedAt instead of anticompacting", sstable, r);
@@ -602,7 +602,7 @@ public class CompactionManager implements CompactionManagerMBean
         // see if there are any keys LTE the token for the start of the first range
         // (token range ownership is exclusive on the LHS.)
         Range<Token> firstRange = sortedRanges.get(0);
-        if (sstable.first.token.compareTo(firstRange.left) <= 0)
+        if (sstable.first.getToken().compareTo(firstRange.left) <= 0)
             return true;
 
         // then, iterate over all owned ranges and see if the next key beyond the end of the owned
@@ -631,7 +631,7 @@ public class CompactionManager implements CompactionManagerMBean
             }
 
             Range<Token> nextRange = sortedRanges.get(i + 1);
-            if (!nextRange.contains(firstBeyondRange.token))
+            if (!nextRange.contains(firstBeyondRange.getToken()))
             {
                 // we found a key in between the owned ranges
                 return true;
@@ -651,7 +651,7 @@ public class CompactionManager implements CompactionManagerMBean
     {
         assert !cfs.isIndex();
 
-        if (!hasIndexes && !new Bounds<>(sstable.first.token, sstable.last.token).intersects(ranges))
+        if (!hasIndexes && !new Bounds<>(sstable.first.getToken(), sstable.last.getToken()).intersects(ranges))
         {
             cfs.getDataTracker().markCompactedSSTablesReplaced(Arrays.asList(sstable), Collections.<SSTableReader>emptyList(), OperationType.CLEANUP);
             return;
@@ -796,7 +796,7 @@ public class CompactionManager implements CompactionManagerMBean
             @Override
             public SSTableIdentityIterator cleanup(SSTableIdentityIterator row)
             {
-                if (Range.isInRanges(row.getKey().token, ranges))
+                if (Range.isInRanges(row.getKey().getToken(), ranges))
                     return row;
 
                 cfs.invalidateCachedRow(row.getKey());
@@ -972,7 +972,7 @@ public class CompactionManager implements CompactionManagerMBean
                     {
                         AbstractCompactedRow row = iter.next();
                         // if current range from sstable is repaired, save it into the new repaired sstable
-                        if (Range.isInRanges(row.key.token, ranges))
+                        if (Range.isInRanges(row.key.getToken(), ranges))
                         {
                             repairedSSTableWriter.append(row);
                             repairedKeyCount++;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/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 2fefe0d..4f211f4 100644
--- a/src/java/org/apache/cassandra/db/compaction/LazilyCompactedRow.java
+++ b/src/java/org/apache/cassandra/db/compaction/LazilyCompactedRow.java
@@ -106,7 +106,7 @@ public class LazilyCompactedRow extends AbstractCompactedRow
         ColumnIndex columnsIndex;
         try
         {
-            indexBuilder = new ColumnIndex.Builder(emptyColumnFamily, key.key, out);
+            indexBuilder = new ColumnIndex.Builder(emptyColumnFamily, key.getKey(), out);
             columnsIndex = indexBuilder.buildForCompaction(merger);
 
             // if there aren't any columns or tombstones, return null
@@ -156,7 +156,7 @@ public class LazilyCompactedRow extends AbstractCompactedRow
         }
 
         // initialize indexBuilder for the benefit of its tombstoneTracker, used by our reducing iterator
-        indexBuilder = new ColumnIndex.Builder(emptyColumnFamily, key.key, out);
+        indexBuilder = new ColumnIndex.Builder(emptyColumnFamily, key.getKey(), out);
         while (merger.hasNext())
             merger.next().updateDigest(digest);
         close();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/db/compaction/LeveledManifest.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/LeveledManifest.java b/src/java/org/apache/cassandra/db/compaction/LeveledManifest.java
index e1a3db8..a21924b 100644
--- a/src/java/org/apache/cassandra/db/compaction/LeveledManifest.java
+++ b/src/java/org/apache/cassandra/db/compaction/LeveledManifest.java
@@ -456,13 +456,13 @@ public class LeveledManifest
          */
         Iterator<SSTableReader> iter = candidates.iterator();
         SSTableReader sstable = iter.next();
-        Token first = sstable.first.token;
-        Token last = sstable.last.token;
+        Token first = sstable.first.getToken();
+        Token last = sstable.last.getToken();
         while (iter.hasNext())
         {
             sstable = iter.next();
-            first = first.compareTo(sstable.first.token) <= 0 ? first : sstable.first.token;
-            last = last.compareTo(sstable.last.token) >= 0 ? last : sstable.last.token;
+            first = first.compareTo(sstable.first.getToken()) <= 0 ? first : sstable.first.getToken();
+            last = last.compareTo(sstable.last.getToken()) >= 0 ? last : sstable.last.getToken();
         }
         return overlapping(first, last, others);
     }
@@ -470,7 +470,7 @@ public class LeveledManifest
     @VisibleForTesting
     static Set<SSTableReader> overlapping(SSTableReader sstable, Iterable<SSTableReader> others)
     {
-        return overlapping(sstable.first.token, sstable.last.token, others);
+        return overlapping(sstable.first.getToken(), sstable.last.getToken(), others);
     }
 
     /**
@@ -483,7 +483,7 @@ public class LeveledManifest
         Bounds<Token> promotedBounds = new Bounds<Token>(start, end);
         for (SSTableReader candidate : sstables)
         {
-            Bounds<Token> candidateBounds = new Bounds<Token>(candidate.first.token, candidate.last.token);
+            Bounds<Token> candidateBounds = new Bounds<Token>(candidate.first.getToken(), candidate.last.getToken());
             if (candidateBounds.intersects(promotedBounds))
                 overlapped.add(candidate);
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/db/compaction/Scrubber.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/Scrubber.java b/src/java/org/apache/cassandra/db/compaction/Scrubber.java
index d61f62b..399f96c 100644
--- a/src/java/org/apache/cassandra/db/compaction/Scrubber.java
+++ b/src/java/org/apache/cassandra/db/compaction/Scrubber.java
@@ -163,7 +163,7 @@ public class Scrubber implements Closeable
 
                 dataSize = dataSizeFromIndex;
                 // avoid an NPE if key is null
-                String keyName = key == null ? "(unreadable key)" : ByteBufferUtil.bytesToHex(key.key);
+                String keyName = key == null ? "(unreadable key)" : ByteBufferUtil.bytesToHex(key.getKey());
                 outputHandler.debug(String.format("row %s is %s bytes", keyName, dataSize));
 
                 assert currentIndexKey != null || indexFile.isEOF();
@@ -188,7 +188,7 @@ public class Scrubber implements Closeable
                     else
                         goodRows++;
                     prevKey = key;
-                    if (!key.key.equals(currentIndexKey) || dataStart != dataStartFromIndex)
+                    if (!key.getKey().equals(currentIndexKey) || dataStart != dataStartFromIndex)
                         outputHandler.warn("Index file contained a different key or row size; using key from data file");
                 }
                 catch (Throwable th)
@@ -197,7 +197,7 @@ public class Scrubber implements Closeable
                     outputHandler.warn("Error reading row (stacktrace follows):", th);
 
                     if (currentIndexKey != null
-                        && (key == null || !key.key.equals(currentIndexKey) || dataStart != dataStartFromIndex || dataSize != dataSizeFromIndex))
+                        && (key == null || !key.getKey().equals(currentIndexKey) || dataStart != dataStartFromIndex || dataSize != dataSizeFromIndex))
                     {
                         outputHandler.output(String.format("Retrying from row index; data is %s bytes starting at %s",
                                                   dataSizeFromIndex, dataStartFromIndex));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/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 191750f..74e24f7 100644
--- a/src/java/org/apache/cassandra/db/composites/AbstractCellNameType.java
+++ b/src/java/org/apache/cassandra/db/composites/AbstractCellNameType.java
@@ -23,6 +23,7 @@ import java.nio.ByteBuffer;
 import java.util.*;
 
 import com.google.common.collect.AbstractIterator;
+import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.cql3.CQL3Row;
 import org.apache.cassandra.cql3.ColumnIdentifier;
@@ -294,19 +295,20 @@ public abstract class AbstractCellNameType extends AbstractCType implements Cell
         }
     }
 
-    protected static CQL3Row.Builder makeSparseCQL3RowBuilder(final CellNameType type, final long now)
+    protected static CQL3Row.Builder makeSparseCQL3RowBuilder(final CFMetaData cfMetaData, final CellNameType type, final long now)
     {
         return new CQL3Row.Builder()
         {
             public CQL3Row.RowIterator group(Iterator<Cell> cells)
             {
-                return new SparseRowIterator(type, cells, now);
+                return new SparseRowIterator(cfMetaData, type, cells, now);
             }
         };
     }
 
     private static class SparseRowIterator extends AbstractIterator<CQL3Row> implements CQL3Row.RowIterator
     {
+        private final CFMetaData cfMetaData;
         private final CellNameType type;
         private final Iterator<Cell> cells;
         private final long now;
@@ -316,8 +318,9 @@ public abstract class AbstractCellNameType extends AbstractCType implements Cell
         private CellName previous;
         private CQL3RowOfSparse currentRow;
 
-        public SparseRowIterator(CellNameType type, Iterator<Cell> cells, long now)
+        public SparseRowIterator(CFMetaData cfMetaData, CellNameType type, Iterator<Cell> cells, long now)
         {
+            this.cfMetaData = cfMetaData;
             this.type = type;
             this.cells = cells;
             this.now = now;
@@ -357,7 +360,7 @@ public abstract class AbstractCellNameType extends AbstractCType implements Cell
                 if (currentRow == null || !current.isSameCQL3RowAs(type, previous))
                 {
                     toReturn = currentRow;
-                    currentRow = new CQL3RowOfSparse(current);
+                    currentRow = new CQL3RowOfSparse(cfMetaData, current);
                 }
                 currentRow.add(nextCell);
                 nextCell = null;
@@ -378,12 +381,14 @@ public abstract class AbstractCellNameType extends AbstractCType implements Cell
 
     private static class CQL3RowOfSparse implements CQL3Row
     {
+        private final CFMetaData cfMetaData;
         private final CellName cell;
         private Map<ColumnIdentifier, Cell> columns;
         private Map<ColumnIdentifier, List<Cell>> collections;
 
-        CQL3RowOfSparse(CellName cell)
+        CQL3RowOfSparse(CFMetaData metadata, CellName cell)
         {
+            this.cfMetaData = metadata;
             this.cell = cell;
         }
 
@@ -395,7 +400,7 @@ public abstract class AbstractCellNameType extends AbstractCType implements Cell
         void add(Cell cell)
         {
             CellName cellName = cell.name();
-            ColumnIdentifier columnName =  cellName.cql3ColumnName();
+            ColumnIdentifier columnName =  cellName.cql3ColumnName(cfMetaData);
             if (cellName.isCollectionCell())
             {
                 if (collections == null)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/db/composites/AbstractComposite.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/composites/AbstractComposite.java b/src/java/org/apache/cassandra/db/composites/AbstractComposite.java
index bd94f88..14fa16c 100644
--- a/src/java/org/apache/cassandra/db/composites/AbstractComposite.java
+++ b/src/java/org/apache/cassandra/db/composites/AbstractComposite.java
@@ -20,7 +20,6 @@ package org.apache.cassandra.db.composites;
 import java.nio.ByteBuffer;
 
 import org.apache.cassandra.db.filter.ColumnSlice;
-import org.apache.cassandra.db.marshal.AbstractCompositeType;
 import org.apache.cassandra.db.marshal.CompositeType;
 import org.apache.cassandra.utils.ByteBufferUtil;
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/db/composites/BoundedComposite.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/composites/BoundedComposite.java b/src/java/org/apache/cassandra/db/composites/BoundedComposite.java
index 5ea39a4..7654edc 100644
--- a/src/java/org/apache/cassandra/db/composites/BoundedComposite.java
+++ b/src/java/org/apache/cassandra/db/composites/BoundedComposite.java
@@ -19,9 +19,9 @@ package org.apache.cassandra.db.composites;
 
 import java.nio.ByteBuffer;
 
+import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.utils.memory.AbstractAllocator;
 import org.apache.cassandra.utils.ObjectSizes;
-import org.apache.cassandra.utils.memory.PoolAllocator;
 
 public class BoundedComposite extends AbstractComposite
 {
@@ -94,14 +94,8 @@ public class BoundedComposite extends AbstractComposite
         return EMPTY_SIZE + wrapped.unsharedHeapSize();
     }
 
-    public Composite copy(AbstractAllocator allocator)
+    public Composite copy(CFMetaData cfm, AbstractAllocator allocator)
     {
-        return new BoundedComposite(wrapped.copy(allocator), isStart);
-    }
-
-    @Override
-    public void free(PoolAllocator allocator)
-    {
-        wrapped.free(allocator);
+        return new BoundedComposite(wrapped.copy(cfm, allocator), isStart);
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/db/composites/CellName.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/composites/CellName.java b/src/java/org/apache/cassandra/db/composites/CellName.java
index 202d73a..22b3fbd 100644
--- a/src/java/org/apache/cassandra/db/composites/CellName.java
+++ b/src/java/org/apache/cassandra/db/composites/CellName.java
@@ -19,6 +19,7 @@ package org.apache.cassandra.db.composites;
 
 import java.nio.ByteBuffer;
 
+import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.cql3.ColumnIdentifier;
 import org.apache.cassandra.utils.memory.AbstractAllocator;
 
@@ -52,8 +53,9 @@ public interface CellName extends Composite
      * The name of the CQL3 column this cell represents.
      *
      * Will be null for cells of "dense" tables.
+     * @param metadata
      */
-    public ColumnIdentifier cql3ColumnName();
+    public ColumnIdentifier cql3ColumnName(CFMetaData metadata);
 
     /**
      * The value of the collection element, or null if the cell is not part
@@ -70,7 +72,7 @@ public interface CellName extends Composite
     // If cellnames were sharing some prefix components, this will break it, so
     // we might want to try to do better.
     @Override
-    public CellName copy(AbstractAllocator allocator);
+    public CellName copy(CFMetaData cfm, AbstractAllocator allocator);
 
     public long excessHeapSizeExcludingData();
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/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 6e8744a..acc0527 100644
--- a/src/java/org/apache/cassandra/db/composites/CellNameType.java
+++ b/src/java/org/apache/cassandra/db/composites/CellNameType.java
@@ -22,6 +22,7 @@ import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.Comparator;
 
+import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.cql3.CQL3Row;
 import org.apache.cassandra.cql3.ColumnIdentifier;
@@ -152,7 +153,7 @@ public interface CellNameType extends CType
     /**
      * Creates a new CQL3Row builder for this type. See CQL3Row for details.
      */
-    public CQL3Row.Builder CQL3RowBuilder(long now);
+    public CQL3Row.Builder CQL3RowBuilder(CFMetaData metadata, long now);
 
     // The two following methods are used to pass the declared regular column names (in CFMetaData)
     // to the CellNameType. This is only used for optimization sake, see SparseCellNameType.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/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 dc7d0e2..b941166 100644
--- a/src/java/org/apache/cassandra/db/composites/CellNames.java
+++ b/src/java/org/apache/cassandra/db/composites/CellNames.java
@@ -20,6 +20,7 @@ package org.apache.cassandra.db.composites;
 import java.nio.ByteBuffer;
 import java.util.List;
 
+import org.apache.cassandra.cql3.ColumnIdentifier;
 import org.apache.cassandra.db.Cell;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.marshal.ColumnToCollectionType;
@@ -75,6 +76,11 @@ public abstract class CellNames
         return new SimpleDenseCellName(bb);
     }
 
+    public static CellName simpleSparse(ColumnIdentifier identifier)
+    {
+        return new SimpleSparseCellName(identifier);
+    }
+
     // Mainly for tests and a few cases where we know what we need and didn't wanted to pass the type around
     // Avoid in general, prefer the CellNameType methods.
     public static CellName compositeDense(ByteBuffer... bbs)
@@ -82,6 +88,16 @@ public abstract class CellNames
         return new CompoundDenseCellName(bbs);
     }
 
+    public static CellName compositeSparse(ByteBuffer[] bbs, ColumnIdentifier identifier, boolean isStatic)
+    {
+        return new CompoundSparseCellName(bbs, identifier, isStatic);
+    }
+
+    public static CellName compositeSparseWithCollection(ByteBuffer[] bbs, ByteBuffer collectionElement, ColumnIdentifier identifier, boolean isStatic)
+    {
+        return new CompoundSparseCellName.WithCollection(bbs, identifier, collectionElement, isStatic);
+    }
+
     public static String getColumnsString(CellNameType type, Iterable<Cell> columns)
     {
         StringBuilder builder = new StringBuilder();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/db/composites/Composite.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/composites/Composite.java b/src/java/org/apache/cassandra/db/composites/Composite.java
index c7ed047..98b1c91 100644
--- a/src/java/org/apache/cassandra/db/composites/Composite.java
+++ b/src/java/org/apache/cassandra/db/composites/Composite.java
@@ -20,9 +20,9 @@ package org.apache.cassandra.db.composites;
 import java.nio.ByteBuffer;
 
 import org.apache.cassandra.cache.IMeasurableMemory;
+import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.filter.ColumnSlice;
 import org.apache.cassandra.utils.memory.AbstractAllocator;
-import org.apache.cassandra.utils.memory.PoolAllocator;
 
 /**
  * A composite value.
@@ -75,6 +75,5 @@ public interface Composite extends IMeasurableMemory
     public ByteBuffer toByteBuffer();
 
     public int dataSize();
-    public Composite copy(AbstractAllocator allocator);
-    public void free(PoolAllocator allocator);
+    public Composite copy(CFMetaData cfm, AbstractAllocator allocator);
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/db/composites/Composites.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/composites/Composites.java b/src/java/org/apache/cassandra/db/composites/Composites.java
index 9b37d6e..42ec72d 100644
--- a/src/java/org/apache/cassandra/db/composites/Composites.java
+++ b/src/java/org/apache/cassandra/db/composites/Composites.java
@@ -19,10 +19,10 @@ package org.apache.cassandra.db.composites;
 
 import java.nio.ByteBuffer;
 
+import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.filter.ColumnSlice;
 import org.apache.cassandra.utils.memory.AbstractAllocator;
 import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.memory.PoolAllocator;
 
 public abstract class Composites
 {
@@ -108,15 +108,9 @@ public abstract class Composites
             return true;
         }
 
-        public Composite copy(AbstractAllocator allocator)
+        public Composite copy(CFMetaData cfm, AbstractAllocator allocator)
         {
             return this;
         }
-
-        @Override
-        public void free(PoolAllocator allocator)
-        {
-        }
-
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/db/composites/CompoundComposite.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/composites/CompoundComposite.java b/src/java/org/apache/cassandra/db/composites/CompoundComposite.java
index 5ba4c8a..e16b48a 100644
--- a/src/java/org/apache/cassandra/db/composites/CompoundComposite.java
+++ b/src/java/org/apache/cassandra/db/composites/CompoundComposite.java
@@ -19,9 +19,9 @@ package org.apache.cassandra.db.composites;
 
 import java.nio.ByteBuffer;
 
+import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.utils.ObjectSizes;
 import org.apache.cassandra.utils.memory.AbstractAllocator;
-import org.apache.cassandra.utils.memory.PoolAllocator;
 
 /**
  * A "truly-composite" Composite.
@@ -81,16 +81,8 @@ public class CompoundComposite extends AbstractComposite
         return EMPTY_SIZE + ObjectSizes.sizeOnHeapExcludingData(elements);
     }
 
-    public Composite copy(AbstractAllocator allocator)
+    public Composite copy(CFMetaData cfm, AbstractAllocator allocator)
     {
         return new CompoundComposite(elementsCopy(allocator), size, isStatic);
     }
-
-    @Override
-    public void free(PoolAllocator allocator)
-    {
-        for (ByteBuffer element : elements)
-            allocator.free(element);
-    }
-
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/db/composites/CompoundDenseCellName.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/composites/CompoundDenseCellName.java b/src/java/org/apache/cassandra/db/composites/CompoundDenseCellName.java
index db80043..0a157c2 100644
--- a/src/java/org/apache/cassandra/db/composites/CompoundDenseCellName.java
+++ b/src/java/org/apache/cassandra/db/composites/CompoundDenseCellName.java
@@ -19,6 +19,7 @@ package org.apache.cassandra.db.composites;
 
 import java.nio.ByteBuffer;
 
+import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.cql3.ColumnIdentifier;
 import org.apache.cassandra.utils.memory.AbstractAllocator;
 import org.apache.cassandra.utils.ObjectSizes;
@@ -44,7 +45,7 @@ public class CompoundDenseCellName extends CompoundComposite implements CellName
         return size;
     }
 
-    public ColumnIdentifier cql3ColumnName()
+    public ColumnIdentifier cql3ColumnName(CFMetaData metadata)
     {
         return null;
     }
@@ -77,7 +78,7 @@ public class CompoundDenseCellName extends CompoundComposite implements CellName
         return HEAP_SIZE + ObjectSizes.sizeOnHeapExcludingData(elements);
     }
 
-    public CellName copy(AbstractAllocator allocator)
+    public CellName copy(CFMetaData cfm, AbstractAllocator allocator)
     {
         return new CompoundDenseCellName(elementsCopy(allocator));
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/db/composites/CompoundDenseCellNameType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/composites/CompoundDenseCellNameType.java b/src/java/org/apache/cassandra/db/composites/CompoundDenseCellNameType.java
index 635b997..0fd2f4b 100644
--- a/src/java/org/apache/cassandra/db/composites/CompoundDenseCellNameType.java
+++ b/src/java/org/apache/cassandra/db/composites/CompoundDenseCellNameType.java
@@ -21,6 +21,7 @@ import java.nio.ByteBuffer;
 import java.util.Arrays;
 import java.util.List;
 
+import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.cql3.CQL3Row;
 import org.apache.cassandra.cql3.ColumnIdentifier;
@@ -80,7 +81,7 @@ public class CompoundDenseCellNameType extends AbstractCompoundCellNameType
     public void addCQL3Column(ColumnIdentifier id) {}
     public void removeCQL3Column(ColumnIdentifier id) {}
 
-    public CQL3Row.Builder CQL3RowBuilder(long now)
+    public CQL3Row.Builder CQL3RowBuilder(CFMetaData metadata, long now)
     {
         return makeDenseCQL3RowBuilder(now);
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/db/composites/CompoundSparseCellName.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/composites/CompoundSparseCellName.java b/src/java/org/apache/cassandra/db/composites/CompoundSparseCellName.java
index 753adbf..ad7bb73 100644
--- a/src/java/org/apache/cassandra/db/composites/CompoundSparseCellName.java
+++ b/src/java/org/apache/cassandra/db/composites/CompoundSparseCellName.java
@@ -19,11 +19,11 @@ package org.apache.cassandra.db.composites;
 
 import java.nio.ByteBuffer;
 
+import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.cql3.ColumnIdentifier;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.memory.AbstractAllocator;
 import org.apache.cassandra.utils.ObjectSizes;
-import org.apache.cassandra.utils.memory.PoolAllocator;
 
 public class CompoundSparseCellName extends CompoundComposite implements CellName
 {
@@ -65,7 +65,7 @@ public class CompoundSparseCellName extends CompoundComposite implements CellNam
         return size;
     }
 
-    public ColumnIdentifier cql3ColumnName()
+    public ColumnIdentifier cql3ColumnName(CFMetaData metadata)
     {
         return columnName;
     }
@@ -93,7 +93,7 @@ public class CompoundSparseCellName extends CompoundComposite implements CellNam
         return true;
     }
 
-    public CellName copy(AbstractAllocator allocator)
+    public CellName copy(CFMetaData cfm, AbstractAllocator allocator)
     {
         if (elements.length == 0)
             return this;
@@ -147,7 +147,7 @@ public class CompoundSparseCellName extends CompoundComposite implements CellNam
         }
 
         @Override
-        public CellName copy(AbstractAllocator allocator)
+        public CellName copy(CFMetaData cfm, AbstractAllocator allocator)
         {
             // We don't copy columnName because it's interned in SparseCellNameType
             return new CompoundSparseCellName.WithCollection(elements.length == 0 ? elements : elementsCopy(allocator), size, columnName, allocator.clone(collectionElement), isStatic());
@@ -164,12 +164,5 @@ public class CompoundSparseCellName extends CompoundComposite implements CellNam
         {
             return super.excessHeapSizeExcludingData() + ObjectSizes.sizeOnHeapExcludingData(collectionElement);
         }
-
-        @Override
-        public void free(PoolAllocator allocator)
-        {
-            super.free(allocator);
-            allocator.free(collectionElement);
-        }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/db/composites/CompoundSparseCellNameType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/composites/CompoundSparseCellNameType.java b/src/java/org/apache/cassandra/db/composites/CompoundSparseCellNameType.java
index 88f6892..bcb500d 100644
--- a/src/java/org/apache/cassandra/db/composites/CompoundSparseCellNameType.java
+++ b/src/java/org/apache/cassandra/db/composites/CompoundSparseCellNameType.java
@@ -20,6 +20,7 @@ package org.apache.cassandra.db.composites;
 import java.nio.ByteBuffer;
 import java.util.*;
 
+import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.cql3.CQL3Row;
 import org.apache.cassandra.cql3.ColumnIdentifier;
@@ -29,11 +30,10 @@ import org.apache.cassandra.db.marshal.ColumnToCollectionType;
 import org.apache.cassandra.db.marshal.UTF8Type;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.memory.AbstractAllocator;
-import org.apache.cassandra.utils.memory.PoolAllocator;
 
 public class CompoundSparseCellNameType extends AbstractCompoundCellNameType
 {
-    private static final ColumnIdentifier rowMarkerId = new ColumnIdentifier(ByteBufferUtil.EMPTY_BYTE_BUFFER, UTF8Type.instance);
+    public static final ColumnIdentifier rowMarkerId = new ColumnIdentifier(ByteBufferUtil.EMPTY_BYTE_BUFFER, UTF8Type.instance);
     private static final CellName rowMarkerNoPrefix = new CompoundSparseCellName(rowMarkerId, false);
 
     // For CQL3 columns, this is always UTF8Type. However, for compatibility with super columns, we need to allow it to be non-UTF8.
@@ -87,15 +87,10 @@ public class CompoundSparseCellNameType extends AbstractCompoundCellNameType
             }
 
             @Override
-            public Composite copy(AbstractAllocator allocator)
+            public Composite copy(CFMetaData cfm, AbstractAllocator allocator)
             {
                 return this;
             }
-
-            @Override
-            public void free(PoolAllocator allocator)
-            {
-            }
         };
     }
 
@@ -204,9 +199,9 @@ public class CompoundSparseCellNameType extends AbstractCompoundCellNameType
         internedIds.remove(id.bytes);
     }
 
-    public CQL3Row.Builder CQL3RowBuilder(long now)
+    public CQL3Row.Builder CQL3RowBuilder(CFMetaData metadata, long now)
     {
-        return makeSparseCQL3RowBuilder(this, now);
+        return makeSparseCQL3RowBuilder(metadata, this, now);
     }
 
     public static class WithCollection extends CompoundSparseCellNameType

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/db/composites/SimpleComposite.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/composites/SimpleComposite.java b/src/java/org/apache/cassandra/db/composites/SimpleComposite.java
index 78173c7..3c80d9f 100644
--- a/src/java/org/apache/cassandra/db/composites/SimpleComposite.java
+++ b/src/java/org/apache/cassandra/db/composites/SimpleComposite.java
@@ -19,9 +19,9 @@ package org.apache.cassandra.db.composites;
 
 import java.nio.ByteBuffer;
 
+import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.utils.memory.AbstractAllocator;
 import org.apache.cassandra.utils.ObjectSizes;
-import org.apache.cassandra.utils.memory.PoolAllocator;
 
 /**
  * A "simple" (not-truly-composite) Composite.
@@ -72,14 +72,8 @@ public class SimpleComposite extends AbstractComposite
         return EMPTY_SIZE + ObjectSizes.sizeOnHeapOf(element);
     }
 
-    public Composite copy(AbstractAllocator allocator)
+    public Composite copy(CFMetaData cfm, AbstractAllocator allocator)
     {
         return new SimpleComposite(allocator.clone(element));
     }
-
-    @Override
-    public void free(PoolAllocator allocator)
-    {
-        allocator.free(element);
-    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/db/composites/SimpleDenseCellName.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/composites/SimpleDenseCellName.java b/src/java/org/apache/cassandra/db/composites/SimpleDenseCellName.java
index a4fb9dc..5c0f976 100644
--- a/src/java/org/apache/cassandra/db/composites/SimpleDenseCellName.java
+++ b/src/java/org/apache/cassandra/db/composites/SimpleDenseCellName.java
@@ -19,6 +19,7 @@ package org.apache.cassandra.db.composites;
 
 import java.nio.ByteBuffer;
 
+import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.cql3.ColumnIdentifier;
 import org.apache.cassandra.utils.memory.AbstractAllocator;
 import org.apache.cassandra.utils.ObjectSizes;
@@ -38,7 +39,7 @@ public class SimpleDenseCellName extends SimpleComposite implements CellName
         return 1;
     }
 
-    public ColumnIdentifier cql3ColumnName()
+    public ColumnIdentifier cql3ColumnName(CFMetaData metadata)
     {
         return null;
     }
@@ -74,7 +75,7 @@ public class SimpleDenseCellName extends SimpleComposite implements CellName
     // If cellnames were sharing some prefix components, this will break it, so
     // we might want to try to do better.
     @Override
-    public CellName copy(AbstractAllocator allocator)
+    public CellName copy(CFMetaData cfm, AbstractAllocator allocator)
     {
         return new SimpleDenseCellName(allocator.clone(element));
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/db/composites/SimpleDenseCellNameType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/composites/SimpleDenseCellNameType.java b/src/java/org/apache/cassandra/db/composites/SimpleDenseCellNameType.java
index 1aae580..3db4bc4 100644
--- a/src/java/org/apache/cassandra/db/composites/SimpleDenseCellNameType.java
+++ b/src/java/org/apache/cassandra/db/composites/SimpleDenseCellNameType.java
@@ -19,6 +19,7 @@ package org.apache.cassandra.db.composites;
 
 import java.nio.ByteBuffer;
 
+import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.cql3.CQL3Row;
 import org.apache.cassandra.cql3.ColumnIdentifier;
@@ -72,7 +73,7 @@ public class SimpleDenseCellNameType extends AbstractSimpleCellNameType
     public void addCQL3Column(ColumnIdentifier id) {}
     public void removeCQL3Column(ColumnIdentifier id) {}
 
-    public CQL3Row.Builder CQL3RowBuilder(long now)
+    public CQL3Row.Builder CQL3RowBuilder(CFMetaData metadata, long now)
     {
         return makeDenseCQL3RowBuilder(now);
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/db/composites/SimpleSparseCellName.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/composites/SimpleSparseCellName.java b/src/java/org/apache/cassandra/db/composites/SimpleSparseCellName.java
index 5c9e991..a7fafba 100644
--- a/src/java/org/apache/cassandra/db/composites/SimpleSparseCellName.java
+++ b/src/java/org/apache/cassandra/db/composites/SimpleSparseCellName.java
@@ -19,10 +19,10 @@ package org.apache.cassandra.db.composites;
 
 import java.nio.ByteBuffer;
 
+import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.cql3.ColumnIdentifier;
 import org.apache.cassandra.utils.memory.AbstractAllocator;
 import org.apache.cassandra.utils.ObjectSizes;
-import org.apache.cassandra.utils.memory.PoolAllocator;
 
 public class SimpleSparseCellName extends AbstractComposite implements CellName
 {
@@ -67,7 +67,7 @@ public class SimpleSparseCellName extends AbstractComposite implements CellName
         return 0;
     }
 
-    public ColumnIdentifier cql3ColumnName()
+    public ColumnIdentifier cql3ColumnName(CFMetaData metadata)
     {
         return columnName;
     }
@@ -97,13 +97,8 @@ public class SimpleSparseCellName extends AbstractComposite implements CellName
         return EMPTY_SIZE + columnName.unsharedHeapSize();
     }
 
-    public CellName copy(AbstractAllocator allocator)
+    public CellName copy(CFMetaData cfm, AbstractAllocator allocator)
     {
         return new SimpleSparseCellName(columnName.clone(allocator));
     }
-
-    public void free(PoolAllocator allocator)
-    {
-        allocator.free(columnName.bytes);
-    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/db/composites/SimpleSparseCellNameType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/composites/SimpleSparseCellNameType.java b/src/java/org/apache/cassandra/db/composites/SimpleSparseCellNameType.java
index 8b446fa..5ce0deb 100644
--- a/src/java/org/apache/cassandra/db/composites/SimpleSparseCellNameType.java
+++ b/src/java/org/apache/cassandra/db/composites/SimpleSparseCellNameType.java
@@ -21,6 +21,7 @@ import java.nio.ByteBuffer;
 import java.util.HashMap;
 import java.util.Map;
 
+import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.cql3.CQL3Row;
 import org.apache.cassandra.cql3.ColumnIdentifier;
@@ -92,8 +93,8 @@ public class SimpleSparseCellNameType extends AbstractSimpleCellNameType
         internedNames.remove(id.bytes);
     }
 
-    public CQL3Row.Builder CQL3RowBuilder(long now)
+    public CQL3Row.Builder CQL3RowBuilder(CFMetaData metadata, long now)
     {
-        return makeSparseCQL3RowBuilder(this, now);
+        return makeSparseCQL3RowBuilder(metadata, this, now);
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/db/composites/SimpleSparseInternedCellName.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/composites/SimpleSparseInternedCellName.java b/src/java/org/apache/cassandra/db/composites/SimpleSparseInternedCellName.java
index a0395cf..c5708d8 100644
--- a/src/java/org/apache/cassandra/db/composites/SimpleSparseInternedCellName.java
+++ b/src/java/org/apache/cassandra/db/composites/SimpleSparseInternedCellName.java
@@ -17,9 +17,9 @@
  */
 package org.apache.cassandra.db.composites;
 
+import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.cql3.ColumnIdentifier;
 import org.apache.cassandra.utils.memory.AbstractAllocator;
-import org.apache.cassandra.utils.memory.PoolAllocator;
 
 public class SimpleSparseInternedCellName extends SimpleSparseCellName
 {
@@ -43,16 +43,9 @@ public class SimpleSparseInternedCellName extends SimpleSparseCellName
     }
 
     @Override
-    public CellName copy(AbstractAllocator allocator)
+    public CellName copy(CFMetaData cfm, AbstractAllocator allocator)
     {
         // We're interning those instance in SparceCellNameType so don't need to copy.
         return this;
     }
-
-    @Override
-    public void free(PoolAllocator allocator)
-    {
-        // no-op, never copied
-    }
-
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/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 6e2fa68..ad3b2fe 100644
--- a/src/java/org/apache/cassandra/db/filter/ColumnSlice.java
+++ b/src/java/org/apache/cassandra/db/filter/ColumnSlice.java
@@ -26,6 +26,7 @@ import java.util.NavigableSet;
 
 import com.google.common.collect.AbstractIterator;
 
+import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.cql3.ColumnIdentifier;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.composites.*;
@@ -35,7 +36,6 @@ import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.memory.AbstractAllocator;
-import org.apache.cassandra.utils.memory.PoolAllocator;
 
 public class ColumnSlice
 {
@@ -219,7 +219,7 @@ public class ColumnSlice
 
     private static Cell fakeCell(Composite name)
     {
-        return new Cell(new FakeCellName(name), ByteBufferUtil.EMPTY_BYTE_BUFFER);
+        return new BufferCell(new FakeCellName(name), ByteBufferUtil.EMPTY_BYTE_BUFFER);
     }
 
     /*
@@ -265,7 +265,7 @@ public class ColumnSlice
             throw new UnsupportedOperationException();
         }
 
-        public ColumnIdentifier cql3ColumnName()
+        public ColumnIdentifier cql3ColumnName(CFMetaData metadata)
         {
             throw new UnsupportedOperationException();
         }
@@ -285,7 +285,7 @@ public class ColumnSlice
             throw new UnsupportedOperationException();
         }
 
-        public CellName copy(AbstractAllocator allocator)
+        public CellName copy(CFMetaData cfm, AbstractAllocator allocator)
         {
             throw new UnsupportedOperationException();
         }
@@ -296,12 +296,6 @@ public class ColumnSlice
             throw new UnsupportedOperationException();
         }
 
-        @Override
-        public void free(PoolAllocator allocator)
-        {
-            throw new UnsupportedOperationException();
-        }
-
         public long unsharedHeapSize()
         {
             throw new UnsupportedOperationException();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/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 6bf25b1..2659439 100644
--- a/src/java/org/apache/cassandra/db/filter/ExtendedFilter.java
+++ b/src/java/org/apache/cassandra/db/filter/ExtendedFilter.java
@@ -256,7 +256,7 @@ public abstract class ExtendedFilter
             assert !(cfs.getComparator().isCompound()) : "Sequential scan with filters is not supported (if you just created an index, you "
                                                          + "need to wait for the creation to be propagated to all nodes before querying it)";
 
-            if (!needsExtraQuery(rowKey.key, data))
+            if (!needsExtraQuery(rowKey.getKey(), data))
                 return null;
 
             // Note: for counters we must be careful to not add a column that was already there (to avoid overcount). That is
@@ -278,7 +278,7 @@ public abstract class ExtendedFilter
                 return data;
 
             ColumnFamily pruned = data.cloneMeShallow();
-            IDiskAtomFilter filter = dataRange.columnFilter(rowKey.key);
+            IDiskAtomFilter filter = dataRange.columnFilter(rowKey.getKey());
             OnDiskAtomIterator iter = filter.getColumnFamilyIterator(rowKey, data);
             filter.collectReducedColumns(pruned, QueryFilter.gatherTombstones(pruned, iter), cfs.gcBefore(timestamp), timestamp);
             return pruned;
@@ -311,7 +311,7 @@ public abstract class ExtendedFilter
                         continue;
                     }
 
-                    dataValue = extractDataValue(def, rowKey.key, data, prefix);
+                    dataValue = extractDataValue(def, rowKey.getKey(), data, prefix);
                     validator = def.type;
                 }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/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 4604c16..164c96f 100644
--- a/src/java/org/apache/cassandra/db/index/AbstractSimplePerColumnSecondaryIndex.java
+++ b/src/java/org/apache/cassandra/db/index/AbstractSimplePerColumnSecondaryIndex.java
@@ -22,13 +22,7 @@ import java.util.concurrent.Future;
 
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.db.ArrayBackedSortedColumns;
-import org.apache.cassandra.db.Cell;
-import org.apache.cassandra.db.ColumnFamily;
-import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.DecoratedKey;
-import org.apache.cassandra.db.ExpiringCell;
-import org.apache.cassandra.db.IndexExpression;
+import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.composites.CellName;
 import org.apache.cassandra.db.composites.CellNameType;
 import org.apache.cassandra.db.marshal.AbstractType;
@@ -37,7 +31,7 @@ import org.apache.cassandra.dht.LocalToken;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.concurrent.OpOrder;
-import org.apache.cassandra.utils.memory.PoolAllocator;
+import org.apache.cassandra.utils.memory.MemtableAllocator;
 
 /**
  * Implements a secondary index for a column family using a second column family
@@ -75,7 +69,7 @@ public abstract class AbstractSimplePerColumnSecondaryIndex extends PerColumnSec
     @Override
     public DecoratedKey getIndexKeyFor(ByteBuffer value)
     {
-        return new DecoratedKey(new LocalToken(getIndexKeyComparator(), value), value);
+        return new BufferDecoratedKey(new LocalToken(getIndexKeyComparator(), value), value);
     }
 
     protected abstract CellName makeIndexColumnName(ByteBuffer rowKey, Cell cell);
@@ -115,14 +109,14 @@ public abstract class AbstractSimplePerColumnSecondaryIndex extends PerColumnSec
         if (cell instanceof ExpiringCell)
         {
             ExpiringCell ec = (ExpiringCell) cell;
-            cfi.addColumn(new ExpiringCell(name, ByteBufferUtil.EMPTY_BYTE_BUFFER, ec.timestamp(), ec.getTimeToLive(), ec.getLocalDeletionTime()));
+            cfi.addColumn(new BufferExpiringCell(name, ByteBufferUtil.EMPTY_BYTE_BUFFER, ec.timestamp(), ec.getTimeToLive(), ec.getLocalDeletionTime()));
         }
         else
         {
-            cfi.addColumn(new Cell(name, ByteBufferUtil.EMPTY_BYTE_BUFFER, cell.timestamp()));
+            cfi.addColumn(new BufferCell(name, ByteBufferUtil.EMPTY_BYTE_BUFFER, cell.timestamp()));
         }
         if (logger.isDebugEnabled())
-            logger.debug("applying index row {} in {}", indexCfs.metadata.getKeyValidator().getString(valueKey.key), cfi);
+            logger.debug("applying index row {} in {}", indexCfs.metadata.getKeyValidator().getString(valueKey.getKey()), cfi);
 
         indexCfs.apply(valueKey, cfi, SecondaryIndexManager.nullUpdater, opGroup, null);
     }
@@ -171,11 +165,6 @@ public abstract class AbstractSimplePerColumnSecondaryIndex extends PerColumnSec
         return indexCfs.name;
     }
 
-    public PoolAllocator getAllocator()
-    {
-        return indexCfs.getDataTracker().getView().getCurrentMemtable().getAllocator();
-    }
-
     public void reload()
     {
         indexCfs.metadata.reloadSecondaryIndexMetadata(baseCfs.metadata);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/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 88d8905..4f95a5a 100644
--- a/src/java/org/apache/cassandra/db/index/SecondaryIndex.java
+++ b/src/java/org/apache/cassandra/db/index/SecondaryIndex.java
@@ -32,6 +32,7 @@ import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.db.BufferDecoratedKey;
 import org.apache.cassandra.db.Cell;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.DecoratedKey;
@@ -51,7 +52,7 @@ import org.apache.cassandra.io.sstable.ReducingKeyIterator;
 import org.apache.cassandra.io.sstable.SSTableReader;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.FBUtilities;
-import org.apache.cassandra.utils.memory.PoolAllocator;
+import org.apache.cassandra.utils.memory.MemtableAllocator;
 
 /**
  * Abstract base class for different types of secondary indexes.
@@ -150,11 +151,6 @@ public abstract class SecondaryIndex
     public abstract void forceBlockingFlush();
 
     /**
-     * Get current amount of memory this index is consuming (in bytes)
-     */
-    public abstract PoolAllocator getAllocator();
-
-    /**
      * Allow access to the underlying column family store if there is one
      * @return the underlying column family store or null
      */
@@ -284,7 +280,7 @@ public abstract class SecondaryIndex
     {
         // FIXME: this imply one column definition per index
         ByteBuffer name = columnDefs.iterator().next().name.bytes;
-        return new DecoratedKey(new LocalToken(baseCfs.metadata.getColumnDefinition(name).type, value), value);
+        return new BufferDecoratedKey(new LocalToken(baseCfs.metadata.getColumnDefinition(name).type, value), value);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/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 a97007e..39bc26b 100644
--- a/src/java/org/apache/cassandra/db/index/SecondaryIndexManager.java
+++ b/src/java/org/apache/cassandra/db/index/SecondaryIndexManager.java
@@ -496,7 +496,7 @@ public class SecondaryIndexManager
             }
             else
             {
-                ((PerColumnSecondaryIndex) index).delete(key.key, cell, opGroup);
+                ((PerColumnSecondaryIndex) index).delete(key.getKey(), cell, opGroup);
             }
         }
     }
@@ -665,7 +665,7 @@ public class SecondaryIndexManager
                 {
                     try (OpOrder.Group opGroup = baseCfs.keyspace.writeOrder.start())
                     {
-                        ((PerColumnSecondaryIndex) index).delete(key.key, cell, opGroup);
+                        ((PerColumnSecondaryIndex) index).delete(key.getKey(), cell, opGroup);
                     }
                 }
             }
@@ -674,7 +674,7 @@ public class SecondaryIndexManager
         public void updateRowLevelIndexes()
         {
             for (SecondaryIndex index : rowLevelIndexMap.values())
-                ((PerRowSecondaryIndex) index).index(key.key, null);
+                ((PerRowSecondaryIndex) index).index(key.getKey(), null);
         }
     }
 
@@ -698,7 +698,7 @@ public class SecondaryIndexManager
 
             for (SecondaryIndex index : indexFor(cell.name()))
                 if (index instanceof PerColumnSecondaryIndex)
-                    ((PerColumnSecondaryIndex) index).insert(key.key, cell, opGroup);
+                    ((PerColumnSecondaryIndex) index).insert(key.getKey(), cell, opGroup);
         }
 
         public void update(Cell oldCell, Cell cell)
@@ -711,9 +711,9 @@ public class SecondaryIndexManager
                 if (index instanceof PerColumnSecondaryIndex)
                 {
                     if (!cell.isMarkedForDelete(System.currentTimeMillis()))
-                        ((PerColumnSecondaryIndex) index).update(key.key, oldCell, cell, opGroup);
+                        ((PerColumnSecondaryIndex) index).update(key.getKey(), oldCell, cell, opGroup);
                     else
-                        ((PerColumnSecondaryIndex) index).delete(key.key, oldCell, opGroup);
+                        ((PerColumnSecondaryIndex) index).delete(key.getKey(), oldCell, opGroup);
                 }
             }
         }
@@ -725,13 +725,13 @@ public class SecondaryIndexManager
 
             for (SecondaryIndex index : indexFor(cell.name()))
                 if (index instanceof PerColumnSecondaryIndex)
-                   ((PerColumnSecondaryIndex) index).delete(key.key, cell, opGroup);
+                   ((PerColumnSecondaryIndex) index).delete(key.getKey(), cell, opGroup);
         }
 
         public void updateRowLevelIndexes()
         {
             for (SecondaryIndex index : rowLevelIndexMap.values())
-                ((PerRowSecondaryIndex) index).index(key.key, cf);
+                ((PerRowSecondaryIndex) index).index(key.getKey(), cf);
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/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 a795d7f..d967971 100644
--- a/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnClusteringKey.java
+++ b/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnClusteringKey.java
@@ -86,7 +86,7 @@ public class CompositesIndexOnClusteringKey extends CompositesIndex
         for (int i = 0; i < columnDef.position(); i++)
             builder.add(indexEntry.name().get(i + 1));
 
-        builder.add(indexedValue.key);
+        builder.add(indexedValue.getKey());
 
         for (int i = columnDef.position() + 1; i < ckCount; i++)
             builder.add(indexEntry.name().get(i));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/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 c8fc56c..2aff39d 100644
--- a/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnCollectionKey.java
+++ b/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnCollectionKey.java
@@ -99,7 +99,7 @@ public class CompositesIndexOnCollectionKey extends CompositesIndex
 
     public boolean isStale(IndexedEntry entry, ColumnFamily data, long now)
     {
-        CellName name = data.getComparator().create(entry.indexedEntryPrefix, columnDef, entry.indexValue.key);
+        CellName name = data.getComparator().create(entry.indexedEntryPrefix, columnDef, entry.indexValue.getKey());
         Cell liveCell = data.getColumn(name);
         return (liveCell == null || liveCell.isMarkedForDelete(now));
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/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 9536e2e..67293aa 100644
--- a/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnCollectionValue.java
+++ b/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnCollectionValue.java
@@ -103,6 +103,6 @@ public class CompositesIndexOnCollectionValue extends CompositesIndex
             return true;
 
         ByteBuffer liveValue = liveCell.value();
-        return ((CollectionType)columnDef.type).valueComparator().compare(entry.indexValue.key, liveValue) != 0;
+        return ((CollectionType)columnDef.type).valueComparator().compare(entry.indexValue.getKey(), liveValue) != 0;
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/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 fc2f9db..0cfd240 100644
--- a/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnRegular.java
+++ b/src/java/org/apache/cassandra/db/index/composites/CompositesIndexOnRegular.java
@@ -95,6 +95,6 @@ public class CompositesIndexOnRegular extends CompositesIndex
             return true;
 
         ByteBuffer liveValue = liveCell.value();
-        return columnDef.type.compare(entry.indexValue.key, liveValue) != 0;
+        return columnDef.type.compare(entry.indexValue.getKey(), liveValue) != 0;
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/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 9de2bd0..22d4361 100644
--- a/src/java/org/apache/cassandra/db/index/composites/CompositesSearcher.java
+++ b/src/java/org/apache/cassandra/db/index/composites/CompositesSearcher.java
@@ -111,8 +111,8 @@ public class CompositesSearcher extends SecondaryIndexSearcher
          * indexed row.
          */
         final AbstractBounds<RowPosition> range = filter.dataRange.keyRange();
-        ByteBuffer startKey = range.left instanceof DecoratedKey ? ((DecoratedKey)range.left).key : ByteBufferUtil.EMPTY_BYTE_BUFFER;
-        ByteBuffer endKey = range.right instanceof DecoratedKey ? ((DecoratedKey)range.right).key : ByteBufferUtil.EMPTY_BYTE_BUFFER;
+        ByteBuffer startKey = range.left instanceof DecoratedKey ? ((DecoratedKey)range.left).getKey() : ByteBufferUtil.EMPTY_BYTE_BUFFER;
+        ByteBuffer endKey = range.right instanceof DecoratedKey ? ((DecoratedKey)range.right).getKey() : ByteBufferUtil.EMPTY_BYTE_BUFFER;
 
         final CellNameType baseComparator = baseCfs.getComparator();
         final CellNameType indexComparator = index.getIndexCfs().getComparator();
@@ -243,14 +243,14 @@ public class CompositesSearcher extends SecondaryIndexSearcher
                             }
                             else
                             {
-                                logger.debug("Skipping entry {} before assigned scan range", dk.token);
+                                logger.debug("Skipping entry {} before assigned scan range", dk.getToken());
                                 continue;
                             }
                         }
 
                         // 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))
+                        if (!filter.columnFilter(dk.getKey()).maySelectPrefix(baseComparator, start))
                             continue;
 
                         // If we've record the previous prefix, it means we're dealing with an index on the collection value. In

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/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 27b61e4..d166c08 100644
--- a/src/java/org/apache/cassandra/db/index/keys/KeysSearcher.java
+++ b/src/java/org/apache/cassandra/db/index/keys/KeysSearcher.java
@@ -85,8 +85,8 @@ public class KeysSearcher extends SecondaryIndexSearcher
          */
         final AbstractBounds<RowPosition> range = filter.dataRange.keyRange();
         CellNameType type = index.getIndexCfs().getComparator();
-        final Composite startKey = range.left instanceof DecoratedKey ? type.make(((DecoratedKey)range.left).key) : Composites.EMPTY;
-        final Composite endKey = range.right instanceof DecoratedKey ? type.make(((DecoratedKey)range.right).key) : Composites.EMPTY;
+        final Composite startKey = range.left instanceof DecoratedKey ? type.make(((DecoratedKey)range.left).getKey()) : Composites.EMPTY;
+        final Composite endKey = range.right instanceof DecoratedKey ? type.make(((DecoratedKey)range.right).getKey()) : Composites.EMPTY;
 
         final CellName primaryColumn = baseCfs.getComparator().cellFromByteBuffer(primary.column);
 
@@ -168,7 +168,7 @@ public class KeysSearcher extends SecondaryIndexSearcher
                         }
                         if (!range.contains(dk))
                         {
-                            logger.trace("Skipping entry {} outside of assigned scan range", dk.token);
+                            logger.trace("Skipping entry {} outside of assigned scan range", dk.getToken());
                             continue;
                         }
 
@@ -188,11 +188,11 @@ public class KeysSearcher extends SecondaryIndexSearcher
                                 data.addAll(cf);
                         }
 
-                        if (((KeysIndex)index).isIndexEntryStale(indexKey.key, data, filter.timestamp))
+                        if (((KeysIndex)index).isIndexEntryStale(indexKey.getKey(), data, filter.timestamp))
                         {
                             // delete the index entry w/ its own timestamp
-                            Cell dummyCell = new Cell(primaryColumn, indexKey.key, cell.timestamp());
-                            ((PerColumnSecondaryIndex)index).delete(dk.key, dummyCell, writeOp);
+                            Cell dummyCell = new BufferCell(primaryColumn, indexKey.getKey(), cell.timestamp());
+                            ((PerColumnSecondaryIndex)index).delete(dk.getKey(), dummyCell, writeOp);
                             continue;
                         }
                         return new Row(dk, data);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/db/marshal/LocalByPartionerType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/LocalByPartionerType.java b/src/java/org/apache/cassandra/db/marshal/LocalByPartionerType.java
index a686b90..60bce9d 100644
--- a/src/java/org/apache/cassandra/db/marshal/LocalByPartionerType.java
+++ b/src/java/org/apache/cassandra/db/marshal/LocalByPartionerType.java
@@ -63,7 +63,7 @@ public class LocalByPartionerType<T extends Token> extends AbstractType<ByteBuff
     public int compare(ByteBuffer o1, ByteBuffer o2)
     {
         // o1 and o2 can be empty so we need to use RowPosition, not DecoratedKey
-        return RowPosition.forKey(o1, partitioner).compareTo(RowPosition.forKey(o2, partitioner));
+        return RowPosition.ForKey.get(o1, partitioner).compareTo(RowPosition.ForKey.get(o2, partitioner));
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/dht/AbstractBounds.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/dht/AbstractBounds.java b/src/java/org/apache/cassandra/dht/AbstractBounds.java
index 32e0818..b69f5ee 100644
--- a/src/java/org/apache/cassandra/dht/AbstractBounds.java
+++ b/src/java/org/apache/cassandra/dht/AbstractBounds.java
@@ -18,7 +18,6 @@
 package org.apache.cassandra.dht;
 
 import java.io.DataInput;
-import java.io.DataOutput;
 import java.io.IOException;
 import java.io.Serializable;
 import java.util.*;
@@ -100,7 +99,7 @@ public abstract class AbstractBounds<T extends RingPosition> implements Serializ
     {
         if (value instanceof DecoratedKey)
         {
-            return keyValidator.getString(((DecoratedKey)value).key);
+            return keyValidator.getString(((DecoratedKey)value).getKey());
         }
         else
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/dht/AbstractByteOrderedPartitioner.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/dht/AbstractByteOrderedPartitioner.java b/src/java/org/apache/cassandra/dht/AbstractByteOrderedPartitioner.java
index 0adab50..94be94d 100644
--- a/src/java/org/apache/cassandra/dht/AbstractByteOrderedPartitioner.java
+++ b/src/java/org/apache/cassandra/dht/AbstractByteOrderedPartitioner.java
@@ -22,6 +22,7 @@ import java.nio.ByteBuffer;
 import java.util.*;
 
 import org.apache.cassandra.config.*;
+import org.apache.cassandra.db.BufferDecoratedKey;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.commons.lang3.ArrayUtils;
 
@@ -42,7 +43,7 @@ public abstract class AbstractByteOrderedPartitioner extends AbstractPartitioner
 
     public DecoratedKey decorateKey(ByteBuffer key)
     {
-        return new DecoratedKey(getToken(key), key);
+        return new BufferDecoratedKey(getToken(key), key);
     }
 
     public BytesToken midpoint(Token ltoken, Token rtoken)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/dht/LocalPartitioner.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/dht/LocalPartitioner.java b/src/java/org/apache/cassandra/dht/LocalPartitioner.java
index 2e75bd3..c32e792 100644
--- a/src/java/org/apache/cassandra/dht/LocalPartitioner.java
+++ b/src/java/org/apache/cassandra/dht/LocalPartitioner.java
@@ -22,6 +22,7 @@ import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.cassandra.db.BufferDecoratedKey;
 import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.utils.ByteBufferUtil;
@@ -40,7 +41,7 @@ public class LocalPartitioner extends AbstractPartitioner<LocalToken>
 
     public DecoratedKey decorateKey(ByteBuffer key)
     {
-        return new DecoratedKey(getToken(key), key);
+        return new BufferDecoratedKey(getToken(key), key);
     }
 
     public Token midpoint(Token left, Token right)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/dht/Murmur3Partitioner.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/dht/Murmur3Partitioner.java b/src/java/org/apache/cassandra/dht/Murmur3Partitioner.java
index de22a34..5a3c4bb 100644
--- a/src/java/org/apache/cassandra/dht/Murmur3Partitioner.java
+++ b/src/java/org/apache/cassandra/dht/Murmur3Partitioner.java
@@ -25,6 +25,7 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.cassandra.db.BufferDecoratedKey;
 import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.marshal.LongType;
@@ -46,7 +47,7 @@ public class Murmur3Partitioner extends AbstractPartitioner<LongToken>
 
     public DecoratedKey decorateKey(ByteBuffer key)
     {
-        return new DecoratedKey(getToken(key), key);
+        return new BufferDecoratedKey(getToken(key), key);
     }
 
     public Token midpoint(Token lToken, Token rToken)


[6/7] Push more of memtable data off-heap

Posted by sl...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/db/BufferDecoratedKey.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/BufferDecoratedKey.java b/src/java/org/apache/cassandra/db/BufferDecoratedKey.java
new file mode 100644
index 0000000..d375162
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/BufferDecoratedKey.java
@@ -0,0 +1,39 @@
+/*
+ * 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.dht.Token;
+
+public class BufferDecoratedKey extends DecoratedKey
+{
+    private final ByteBuffer key;
+
+    public BufferDecoratedKey(Token token, ByteBuffer key)
+    {
+        super(token);
+        assert key != null;
+        this.key = key;
+    }
+
+    public ByteBuffer getKey()
+    {
+        return key;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/db/BufferDeletedCell.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/BufferDeletedCell.java b/src/java/org/apache/cassandra/db/BufferDeletedCell.java
new file mode 100644
index 0000000..a6518de
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/BufferDeletedCell.java
@@ -0,0 +1,123 @@
+/*
+ * 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.security.MessageDigest;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.db.composites.CellName;
+import org.apache.cassandra.serializers.MarshalException;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.concurrent.OpOrder;
+import org.apache.cassandra.utils.memory.AbstractAllocator;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.memory.MemtableAllocator;
+
+public class BufferDeletedCell extends BufferCell implements DeletedCell
+{
+    public BufferDeletedCell(CellName name, int localDeletionTime, long timestamp)
+    {
+        this(name, ByteBufferUtil.bytes(localDeletionTime), timestamp);
+    }
+
+    public BufferDeletedCell(CellName name, ByteBuffer value, long timestamp)
+    {
+        super(name, value, timestamp);
+    }
+
+    @Override
+    public Cell withUpdatedName(CellName newName)
+    {
+        return new BufferDeletedCell(newName, value, timestamp);
+    }
+
+    @Override
+    public Cell withUpdatedTimestamp(long newTimestamp)
+    {
+        return new BufferDeletedCell(name, value, newTimestamp);
+    }
+
+    @Override
+    public boolean isMarkedForDelete(long now)
+    {
+        return true;
+    }
+
+    @Override
+    public long getMarkedForDeleteAt()
+    {
+        return timestamp;
+    }
+
+    @Override
+    public int getLocalDeletionTime()
+    {
+       return value().getInt(value.position());
+    }
+
+    @Override
+    public Cell reconcile(Cell cell)
+    {
+        if (cell instanceof DeletedCell)
+            return super.reconcile(cell);
+        return cell.reconcile(this);
+    }
+
+    @Override
+    public DeletedCell localCopy(CFMetaData metadata, AbstractAllocator allocator)
+    {
+        return new BufferDeletedCell(name.copy(metadata, allocator), allocator.clone(value), timestamp);
+    }
+
+    @Override
+    public DeletedCell localCopy(CFMetaData metadata, MemtableAllocator allocator, OpOrder.Group opGroup)
+    {
+        return allocator.clone(this, metadata, opGroup);
+    }
+
+    @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");
+    }
+
+    public boolean equals(Cell cell)
+    {
+        return timestamp() == cell.timestamp() && getLocalDeletionTime() == cell.getLocalDeletionTime() && name().equals(cell.name());
+    }
+
+    @Override
+    public void updateDigest(MessageDigest digest)
+    {
+        digest.update(name().toByteBuffer().duplicate());
+
+        FBUtilities.updateWithLong(digest, timestamp());
+        FBUtilities.updateWithByte(digest, serializationFlags());
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/db/BufferExpiringCell.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/BufferExpiringCell.java b/src/java/org/apache/cassandra/db/BufferExpiringCell.java
new file mode 100644
index 0000000..95ed45a
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/BufferExpiringCell.java
@@ -0,0 +1,170 @@
+/*
+ * 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.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.serializers.MarshalException;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.concurrent.OpOrder;
+import org.apache.cassandra.utils.memory.AbstractAllocator;
+import org.apache.cassandra.utils.memory.MemtableAllocator;
+
+public class BufferExpiringCell extends BufferCell implements ExpiringCell
+{
+    private final int localExpirationTime;
+    private final int timeToLive;
+
+    public BufferExpiringCell(CellName name, ByteBuffer value, long timestamp, int timeToLive)
+    {
+        this(name, value, timestamp, timeToLive, (int) (System.currentTimeMillis() / 1000) + timeToLive);
+    }
+
+    public BufferExpiringCell(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;
+    }
+
+    public int getTimeToLive()
+    {
+        return timeToLive;
+    }
+
+    @Override
+    public Cell withUpdatedName(CellName newName)
+    {
+        return new BufferExpiringCell(newName, value(), timestamp(), timeToLive, localExpirationTime);
+    }
+
+    @Override
+    public Cell withUpdatedTimestamp(long newTimestamp)
+    {
+        return new BufferExpiringCell(name(), value(), newTimestamp, timeToLive, localExpirationTime);
+    }
+
+    @Override
+    public int cellDataSize()
+    {
+        return super.cellDataSize() + 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)
+    {
+        super.updateDigest(digest);
+        FBUtilities.updateWithInt(digest, timeToLive);
+    }
+
+    @Override
+    public int getLocalDeletionTime()
+    {
+        return localExpirationTime;
+    }
+
+    @Override
+    public ExpiringCell localCopy(CFMetaData metadata, AbstractAllocator allocator)
+    {
+        return new BufferExpiringCell(name.copy(metadata, allocator), allocator.clone(value), timestamp, timeToLive, localExpirationTime);
+    }
+
+    @Override
+    public ExpiringCell localCopy(CFMetaData metadata, MemtableAllocator allocator, OpOrder.Group opGroup)
+    {
+        return allocator.clone(this, metadata, opGroup);
+    }
+
+    @Override
+    public String getString(CellNameType comparator)
+    {
+        return String.format("%s!%d", super.getString(comparator), timeToLive);
+    }
+
+    @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(Cell cell)
+    {
+        return cell instanceof ExpiringCell && equals((ExpiringCell) cell);
+    }
+
+    public boolean equals(ExpiringCell cell)
+    {
+        // super.equals() returns false if o is not a CounterCell
+        return super.equals(cell)
+               && getLocalDeletionTime() == cell.getLocalDeletionTime()
+               && getTimeToLive() == cell.getTimeToLive();
+    }
+
+    /** @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 BufferExpiringCell(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 BufferDeletedCell(name, localExpirationTime - timeToLive, timestamp);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/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 f853d17..dfe49ee 100644
--- a/src/java/org/apache/cassandra/db/CFRowAdder.java
+++ b/src/java/org/apache/cassandra/db/CFRowAdder.java
@@ -52,7 +52,7 @@ public class CFRowAdder
 
         // If a CQL3 table, add the row marker
         if (cf.metadata().isCQL3Table() && !prefix.isStatic())
-            cf.addColumn(new Cell(cf.getComparator().rowMarker(prefix), ByteBufferUtil.EMPTY_BYTE_BUFFER, timestamp));
+            cf.addColumn(new BufferCell(cf.getComparator().rowMarker(prefix), ByteBufferUtil.EMPTY_BYTE_BUFFER, timestamp));
     }
 
     public CFRowAdder add(String cql3ColumnName, Object value)
@@ -96,14 +96,14 @@ public class CFRowAdder
     {
         if (value == null)
         {
-            cf.addColumn(new DeletedCell(name, ldt, timestamp));
+            cf.addColumn(new BufferDeletedCell(name, ldt, timestamp));
         }
         else
         {
             AbstractType valueType = def.type.isCollection()
                                    ? ((CollectionType) def.type).valueComparator()
                                    : def.type;
-            cf.addColumn(new Cell(name, value instanceof ByteBuffer ? (ByteBuffer)value : valueType.decompose(value), timestamp));
+            cf.addColumn(new BufferCell(name, value instanceof ByteBuffer ? (ByteBuffer)value : valueType.decompose(value), timestamp));
         }
         return this;
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/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
index 8db9770..c19b5dd 100644
--- a/src/java/org/apache/cassandra/db/Cell.java
+++ b/src/java/org/apache/cassandra/db/Cell.java
@@ -17,261 +17,58 @@
  */
 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.composites.CellNames;
-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.ObjectSizes;
+import org.apache.cassandra.utils.concurrent.OpOrder;
 import org.apache.cassandra.utils.memory.AbstractAllocator;
-import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.memory.MemtableAllocator;
 
 /**
  * Cell is immutable, which prevents all kinds of confusion in a multithreaded environment.
  */
-public class Cell implements OnDiskAtom
+public interface Cell extends OnDiskAtom
 {
     public static final int MAX_NAME_LENGTH = FBUtilities.MAX_UNSIGNED_SHORT;
 
-    private static final long EMPTY_SIZE = ObjectSizes.measure(new Cell(CellNames.simpleDense(ByteBuffer.allocate(1))));
-
-    public static Iterator<OnDiskAtom> onDiskIterator(final DataInput in,
-                                                      final ColumnSerializer.Flag flag,
-                                                      final int expireBefore,
-                                                      final Descriptor.Version version,
-                                                      final CellNameType type)
-    {
-        return new AbstractIterator<OnDiskAtom>()
-        {
-            protected OnDiskAtom computeNext()
-            {
-                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 withUpdatedName(CellName newName);
 
-    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 withUpdatedTimestamp(long newTimestamp);
 
-    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;
-    }
+    @Override
+    public CellName name();
 
-    public long timestamp()
-    {
-        return timestamp;
-    }
+    public ByteBuffer value();
 
-    public boolean isMarkedForDelete(long now)
-    {
-        return false;
-    }
+    public boolean isMarkedForDelete(long now);
 
-    public boolean isLive(long now)
-    {
-        return !isMarkedForDelete(now);
-    }
+    public boolean isLive(long now);
 
     // Don't call unless the column is actually marked for delete.
-    public long getMarkedForDeleteAt()
-    {
-        return Long.MAX_VALUE;
-    }
+    public long getMarkedForDeleteAt();
 
-    public int dataSize()
-    {
-        return name.dataSize() + value.remaining() + TypeSizes.NATIVE.sizeof(timestamp);
-    }
+    public int cellDataSize();
 
     // returns the size of the Cell and all references on the heap, excluding any costs associated with byte arrays
     // that would be allocated by a localCopy, as these will be accounted for by the allocator
-    public long excessHeapSizeExcludingData()
-    {
-        return EMPTY_SIZE + name.excessHeapSizeExcludingData() + ObjectSizes.sizeOnHeapExcludingData(value);
-    }
-
-    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)
-    {
-        // 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;
-
-        return timestamp == cell.timestamp && name.equals(cell.name) && 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 long excessHeapSizeExcludingData();
 
-    public Cell localCopy(AbstractAllocator allocator)
-    {
-        return new Cell(name.copy(allocator), allocator.clone(value), timestamp);
-    }
+    public int serializedSize(CellNameType type, TypeSizes typeSizes);
+    public int serializationFlags();
 
-    public String getString(CellNameType comparator)
-    {
-        return String.format("%s:%b:%d@%d",
-                             comparator.getString(name),
-                             isMarkedForDelete(System.currentTimeMillis()),
-                             value.remaining(),
-                             timestamp);
-    }
+    public Cell diff(Cell cell);
 
-    protected void validateName(CFMetaData metadata) throws MarshalException
-    {
-        metadata.comparator.validate(name());
-    }
+    public Cell reconcile(Cell cell);
 
-    public void validateFields(CFMetaData metadata) throws MarshalException
-    {
-        validateName(metadata);
+    public Cell localCopy(CFMetaData metadata, AbstractAllocator allocator);
 
-        AbstractType<?> valueValidator = metadata.getValueValidator(name());
-        if (valueValidator != null)
-            valueValidator.validate(value());
-    }
+    public Cell localCopy(CFMetaData metaData, MemtableAllocator allocator, OpOrder.Group opGroup);
 
-    public static Cell create(CellName name, ByteBuffer value, long timestamp, int ttl, CFMetaData metadata)
-    {
-        if (ttl <= 0)
-            ttl = metadata.getDefaultTimeToLive();
+    public String getString(CellNameType comparator);
 
-        return ttl > 0
-               ? new ExpiringCell(name, value, timestamp, ttl)
-               : new Cell(name, value, timestamp);
-    }
+    void validateName(CFMetaData metadata) throws MarshalException;
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/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 36a9ebf..f88c1a7 100644
--- a/src/java/org/apache/cassandra/db/CollationController.java
+++ b/src/java/org/apache/cassandra/db/CollationController.java
@@ -86,7 +86,7 @@ public class CollationController
                     {
                         OnDiskAtom atom = iter.next();
                         if (copyOnHeap)
-                            atom = ((Cell) atom).localCopy(HeapAllocator.instance);
+                            atom = ((Cell) atom).localCopy(cfs.metadata, HeapAllocator.instance);
                         container.addAtom(atom);
                     }
                 }
@@ -147,7 +147,7 @@ public class CollationController
                 && cfs.getCompactionStrategy() instanceof SizeTieredCompactionStrategy)
             {
                 Tracing.trace("Defragmenting requested data");
-                Mutation mutation = new Mutation(cfs.keyspace.getName(), filter.key.key, returnCF.cloneMe());
+                Mutation mutation = new Mutation(cfs.keyspace.getName(), filter.key.getKey(), returnCF.cloneMe());
                 // skipping commitlog and index updates is fine since we're just de-fragmenting existing data
                 Keyspace.open(mutation.getKeyspaceName()).apply(mutation, false, false);
             }
@@ -204,7 +204,7 @@ public class CollationController
                         ColumnFamily newCf = cf.cloneMeShallow(ArrayBackedSortedColumns.factory, false);
                         for (Cell cell : cf)
                         {
-                            newCf.addColumn(cell.localCopy(HeapAllocator.instance));
+                            newCf.addColumn(cell.localCopy(cfs.metadata, HeapAllocator.instance));
                         }
                         cf = newCf;
                         iter = filter.getColumnFamilyIterator(cf);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/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 4f85610..a261d73 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamily.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamily.java
@@ -128,23 +128,23 @@ public abstract class ColumnFamily implements Iterable<Cell>, IRowCacheEntry
     public void addColumn(CellName name, ByteBuffer value, long timestamp, int timeToLive)
     {
         assert !metadata().isCounter();
-        Cell cell = Cell.create(name, value, timestamp, timeToLive, metadata());
+        Cell cell = AbstractCell.create(name, value, timestamp, timeToLive, metadata());
         addColumn(cell);
     }
 
     public void addCounter(CellName name, long value)
     {
-        addColumn(new CounterUpdateCell(name, value, System.currentTimeMillis()));
+        addColumn(new BufferCounterUpdateCell(name, value, System.currentTimeMillis()));
     }
 
     public void addTombstone(CellName name, ByteBuffer localDeletionTime, long timestamp)
     {
-        addColumn(new DeletedCell(name, localDeletionTime, timestamp));
+        addColumn(new BufferDeletedCell(name, localDeletionTime, timestamp));
     }
 
     public void addTombstone(CellName name, int localDeletionTime, long timestamp)
     {
-        addColumn(new DeletedCell(name, localDeletionTime, timestamp));
+        addColumn(new BufferDeletedCell(name, localDeletionTime, timestamp));
     }
 
     public void addAtom(OnDiskAtom atom)
@@ -325,7 +325,7 @@ public abstract class ColumnFamily implements Iterable<Cell>, IRowCacheEntry
     {
         long size = 0;
         for (Cell cell : this)
-            size += cell.dataSize();
+            size += cell.cellDataSize();
         return size;
     }
 
@@ -424,8 +424,8 @@ public abstract class ColumnFamily implements Iterable<Cell>, IRowCacheEntry
             int deletionTime = cell.getLocalDeletionTime();
             if (deletionTime < Integer.MAX_VALUE)
                 tombstones.update(deletionTime);
-            minColumnNamesSeen = ColumnNameHelper.minComponents(minColumnNamesSeen, cell.name, metadata.comparator);
-            maxColumnNamesSeen = ColumnNameHelper.maxComponents(maxColumnNamesSeen, cell.name, metadata.comparator);
+            minColumnNamesSeen = ColumnNameHelper.minComponents(minColumnNamesSeen, cell.name(), metadata.comparator);
+            maxColumnNamesSeen = ColumnNameHelper.maxComponents(maxColumnNamesSeen, cell.name(), metadata.comparator);
             if (cell instanceof CounterCell)
                 hasLegacyCounterShards = hasLegacyCounterShards || ((CounterCell) cell).hasLegacyShards();
         }
@@ -474,7 +474,7 @@ public abstract class ColumnFamily implements Iterable<Cell>, IRowCacheEntry
     {
         ImmutableMap.Builder<CellName, ByteBuffer> builder = ImmutableMap.builder();
         for (Cell cell : this)
-            builder.put(cell.name, cell.value);
+            builder.put(cell.name(), cell.value());
         return builder.build();
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/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 6559b40..3b1c67e 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@ -75,6 +75,7 @@ import org.apache.cassandra.streaming.StreamLockfile;
 import org.apache.cassandra.tracing.Tracing;
 import org.apache.cassandra.utils.*;
 import org.apache.cassandra.utils.concurrent.OpOrder;
+import org.apache.cassandra.utils.memory.MemtableAllocator;
 
 public class ColumnFamilyStore implements ColumnFamilyStoreMBean
 {
@@ -843,12 +844,13 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
 
         for (SecondaryIndex index : indexManager.getIndexes())
         {
-            if (index.getAllocator() != null)
+            if (index.getIndexCfs() != null)
             {
-                onHeapRatio += index.getAllocator().onHeap().ownershipRatio();
-                offHeapRatio += index.getAllocator().offHeap().ownershipRatio();
-                onHeapTotal += index.getAllocator().onHeap().owns();
-                offHeapTotal += index.getAllocator().offHeap().owns();
+                MemtableAllocator allocator = index.getIndexCfs().getDataTracker().getView().getCurrentMemtable().getAllocator();
+                onHeapRatio += allocator.onHeap().ownershipRatio();
+                offHeapRatio += allocator.offHeap().ownershipRatio();
+                onHeapTotal += allocator.onHeap().owns();
+                offHeapTotal += allocator.offHeap().owns();
             }
         }
 
@@ -1095,10 +1097,11 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
 
                 for (SecondaryIndex index : cfs.indexManager.getIndexes())
                 {
-                    if (index.getAllocator() != null)
+                    if (index.getIndexCfs() != null)
                     {
-                        onHeap += index.getAllocator().onHeap().ownershipRatio();
-                        offHeap += index.getAllocator().offHeap().ownershipRatio();
+                        MemtableAllocator allocator = index.getIndexCfs().getDataTracker().getView().getCurrentMemtable().getAllocator();
+                        onHeap += allocator.onHeap().ownershipRatio();
+                        offHeap += allocator.offHeap().ownershipRatio();
                     }
                 }
 
@@ -1213,7 +1216,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
     // 2. if it has been re-added since then, this particular column was inserted before the last drop
     private static boolean isDroppedColumn(Cell c, CFMetaData meta)
     {
-        Long droppedAt = meta.getDroppedColumns().get(c.name().cql3ColumnName());
+        Long droppedAt = meta.getDroppedColumns().get(c.name().cql3ColumnName(meta));
         return droppedAt != null && c.timestamp() <= droppedAt;
     }
 
@@ -1869,7 +1872,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         ColumnFamily columns;
         try (OpOrder.Group op = readOrdering.start())
         {
-            columns = controller.getTopLevelColumns(Memtable.memoryPool.needToCopyOnHeap());
+            columns = controller.getTopLevelColumns(Memtable.MEMORY_POOL.needToCopyOnHeap());
         }
         metric.updateSSTableIterated(controller.getSstablesIterated());
         return columns;
@@ -1882,7 +1885,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         for (RowCacheKey key : CacheService.instance.rowCache.getKeySet())
         {
             DecoratedKey dk = partitioner.decorateKey(ByteBuffer.wrap(key.key));
-            if (key.cfId == metadata.cfId && !Range.isInRanges(dk.token, ranges))
+            if (key.cfId == metadata.cfId && !Range.isInRanges(dk.getToken(), ranges))
                 invalidateCachedRow(dk);
         }
 
@@ -1891,7 +1894,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
             for (CounterCacheKey key : CacheService.instance.counterCache.getKeySet())
             {
                 DecoratedKey dk = partitioner.decorateKey(ByteBuffer.wrap(key.partitionKey));
-                if (key.cfId == metadata.cfId && !Range.isInRanges(dk.token, ranges))
+                if (key.cfId == metadata.cfId && !Range.isInRanges(dk.getToken(), ranges))
                     CacheService.instance.counterCache.remove(key);
             }
         }
@@ -1939,7 +1942,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
                     return computeNext();
 
                 if (logger.isTraceEnabled())
-                    logger.trace("scanned {}", metadata.getKeyValidator().getString(key.key));
+                    logger.trace("scanned {}", metadata.getKeyValidator().getString(key.getKey()));
 
                 return current;
             }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/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 8c22d71..8e7026c 100644
--- a/src/java/org/apache/cassandra/db/ColumnSerializer.java
+++ b/src/java/org/apache/cassandra/db/ColumnSerializer.java
@@ -116,7 +116,7 @@ public class ColumnSerializer implements ISerializer<Cell>
             long timestampOfLastDelete = in.readLong();
             long ts = in.readLong();
             ByteBuffer value = ByteBufferUtil.readWithLength(in);
-            return CounterCell.create(name, value, ts, timestampOfLastDelete, flag);
+            return BufferCounterCell.create(name, value, ts, timestampOfLastDelete, flag);
         }
         else if ((mask & EXPIRATION_MASK) != 0)
         {
@@ -124,17 +124,17 @@ public class ColumnSerializer implements ISerializer<Cell>
             int expiration = in.readInt();
             long ts = in.readLong();
             ByteBuffer value = ByteBufferUtil.readWithLength(in);
-            return ExpiringCell.create(name, value, ts, ttl, expiration, expireBefore, flag);
+            return BufferExpiringCell.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 CounterUpdateCell(name, value, ts)
+                   ? new BufferCounterUpdateCell(name, value, ts)
                    : ((mask & DELETION_MASK) == 0
-                      ? new Cell(name, value, ts)
-                      : new DeletedCell(name, value, ts));
+                      ? new BufferCell(name, value, ts)
+                      : new BufferDeletedCell(name, value, ts));
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/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
index fc4ac3f..cda1200 100644
--- a/src/java/org/apache/cassandra/db/CounterCell.java
+++ b/src/java/org/apache/cassandra/db/CounterCell.java
@@ -17,223 +17,28 @@
  */
 package org.apache.cassandra.db;
 
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.security.MessageDigest;
-import org.apache.cassandra.serializers.MarshalException;
-import org.apache.cassandra.utils.memory.AbstractAllocator;
-
 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.context.CounterContext;
-import org.apache.cassandra.io.util.DataOutputBuffer;
+import org.apache.cassandra.utils.concurrent.OpOrder;
+import org.apache.cassandra.utils.memory.AbstractAllocator;
+import org.apache.cassandra.utils.memory.MemtableAllocator;
 
 /**
  * A column that represents a partitioned counter.
  */
-public class CounterCell extends Cell
+public interface CounterCell extends Cell
 {
-    protected static final CounterContext contextManager = CounterContext.instance();
-
-    private final long 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)
-    {
-        if (flag == ColumnSerializer.Flag.FROM_REMOTE || (flag == ColumnSerializer.Flag.LOCAL && contextManager.shouldClearLocal(value)))
-            value = contextManager.clearAllLocal(value);
-        return new CounterCell(name, value, timestamp, timestampOfLastDelete);
-    }
-
-    // For use by tests of compatibility with pre-2.1 counter only.
-    public static CounterCell createLocal(CellName name, long value, long timestamp, long timestampOfLastDelete)
-    {
-        return new CounterCell(name, contextManager.createLocal(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 8 bytes for timestampOfLastDelete to Cell.
-        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 CounterCell : "Wrong class type: " + cell.getClass();
-
-        if (timestampOfLastDelete() < ((CounterCell) cell).timestampOfLastDelete())
-            return cell;
-        CounterContext.Relationship rel = contextManager.diff(cell.value(), value());
-        if (rel == CounterContext.Relationship.GREATER_THAN || rel == CounterContext.Relationship.DISJOINT)
-            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)
-    {
-        // 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());
-        }
-
-        assert cell instanceof CounterCell : "Wrong class type: " + cell.getClass();
-
-        // live < live last delete
-        if (timestamp() < ((CounterCell) cell).timestampOfLastDelete())
-            return cell;
-        // live last delete > live
-        if (timestampOfLastDelete() > cell.timestamp())
-            return this;
-
-        // live + live. return one of the cells if its context is a superset of the other's, or merge them otherwise
-        ByteBuffer context = contextManager.merge(value(), cell.value());
-        if (context == value() && timestamp() >= cell.timestamp() && timestampOfLastDelete() >= ((CounterCell) cell).timestampOfLastDelete())
-            return this;
-        else if (context == cell.value() && cell.timestamp() >= timestamp() && ((CounterCell) cell).timestampOfLastDelete() >= timestampOfLastDelete())
-            return cell;
-        else // merge clocks and timsestamps.
-            return new CounterCell(name(),
-                                   context,
-                                   Math.max(timestamp(), cell.timestamp()),
-                                   Math.max(timestampOfLastDelete(), ((CounterCell) cell).timestampOfLastDelete()));
-    }
-
-    public boolean hasLegacyShards()
-    {
-        return contextManager.hasLegacyShards(value);
-    }
-
-    @Override
-    public boolean equals(Object o)
-    {
-        // super.equals() returns false if o is not a CounterCell
-        return super.equals(o) && timestampOfLastDelete == ((CounterCell)o).timestampOfLastDelete;
-    }
+    static final CounterContext contextManager = CounterContext.instance();
 
-    @Override
-    public int hashCode()
-    {
-        return 31 * super.hashCode() + (int)(timestampOfLastDelete ^ (timestampOfLastDelete >>> 32));
-    }
+    public long timestampOfLastDelete();
 
-    @Override
-    public Cell localCopy(AbstractAllocator allocator)
-    {
-        return new CounterCell(name.copy(allocator), allocator.clone(value), timestamp, timestampOfLastDelete);
-    }
+    public long total();
 
-    @Override
-    public String getString(CellNameType comparator)
-    {
-        return String.format("%s:false:%s@%d!%d",
-                             comparator.getString(name),
-                             contextManager.toString(value),
-                             timestamp,
-                             timestampOfLastDelete);
-    }
+    public boolean hasLegacyShards();
 
-    @Override
-    public int serializationFlags()
-    {
-        return ColumnSerializer.COUNTER_MASK;
-    }
+    public Cell markLocalToBeCleared();
 
-    @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());
-    }
+    CounterCell localCopy(CFMetaData metadata, AbstractAllocator allocator);
 
-    public Cell markLocalToBeCleared()
-    {
-        ByteBuffer marked = contextManager.markLocalToBeCleared(value);
-        return marked == value ? this : new CounterCell(name, marked, timestamp, timestampOfLastDelete);
-    }
+    CounterCell localCopy(CFMetaData metaData, MemtableAllocator allocator, OpOrder.Group opGroup);
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/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 c19b436..32571cc 100644
--- a/src/java/org/apache/cassandra/db/CounterMutation.java
+++ b/src/java/org/apache/cassandra/db/CounterMutation.java
@@ -200,9 +200,9 @@ public class CounterMutation implements IMutation
             long clock = currentValue.clock + 1L;
             long count = currentValue.count + update.delta();
 
-            resultCF.addColumn(new CounterCell(update.name(),
-                                               CounterContext.instance().createGlobal(CounterId.getLocalId(), clock, count),
-                                               update.timestamp()));
+            resultCF.addColumn(new BufferCounterCell(update.name(),
+                                                     CounterContext.instance().createGlobal(CounterId.getLocalId(), clock, count),
+                                                     update.timestamp()));
         }
 
         return resultCF;
@@ -253,7 +253,7 @@ public class CounterMutation implements IMutation
         SortedSet<CellName> names = new TreeSet<>(cfs.metadata.comparator);
         for (int i = 0; i < currentValues.length; i++)
             if (currentValues[i] == null)
-                names.add(counterUpdateCells.get(i).name);
+                names.add(counterUpdateCells.get(i).name());
 
         ReadCommand cmd = new SliceByNamesReadCommand(getKeyspaceName(), key(), cfs.metadata.cfName, Long.MIN_VALUE, new NamesQueryFilter(names));
         Row row = cmd.getRow(cfs.keyspace);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/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
index 27d5270..58ac365 100644
--- a/src/java/org/apache/cassandra/db/CounterUpdateCell.java
+++ b/src/java/org/apache/cassandra/db/CounterUpdateCell.java
@@ -17,13 +17,6 @@
  */
 package org.apache.cassandra.db;
 
-import java.nio.ByteBuffer;
-
-import org.apache.cassandra.db.composites.CellName;
-import org.apache.cassandra.utils.memory.AbstractAllocator;
-import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.db.composites.CellNameType;
-
 /**
  * A counter update while it hasn't been applied yet by the leader replica.
  *
@@ -31,61 +24,7 @@ import org.apache.cassandra.db.composites.CellNameType;
  * 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 interface 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)
-    {
-        // The only time this could happen is if a batchAdd ships two
-        // increment for the same cell. Hence we simply sums the delta.
-
-        // 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
-        assert cell instanceof CounterUpdateCell : "Wrong class type.";
-        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 Cell localCopy(AbstractAllocator allocator)
-    {
-        throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public String getString(CellNameType comparator)
-    {
-        return String.format("%s:%s@%d", comparator.getString(name), ByteBufferUtil.toLong(value), timestamp);
-    }
+    public long delta();
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/db/DataRange.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/DataRange.java b/src/java/org/apache/cassandra/db/DataRange.java
index 91ff512..31a9370 100644
--- a/src/java/org/apache/cassandra/db/DataRange.java
+++ b/src/java/org/apache/cassandra/db/DataRange.java
@@ -163,7 +163,7 @@ public class DataRange
 
         private boolean equals(RowPosition pos, ByteBuffer rowKey)
         {
-            return pos instanceof DecoratedKey && ((DecoratedKey)pos).key.equals(rowKey);
+            return pos instanceof DecoratedKey && ((DecoratedKey)pos).getKey().equals(rowKey);
         }
 
         @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/db/DecoratedKey.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/DecoratedKey.java b/src/java/org/apache/cassandra/db/DecoratedKey.java
index 8f7a22b..fb3d7ab 100644
--- a/src/java/org/apache/cassandra/db/DecoratedKey.java
+++ b/src/java/org/apache/cassandra/db/DecoratedKey.java
@@ -22,6 +22,7 @@ import java.util.Comparator;
 
 import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 /**
@@ -33,7 +34,7 @@ import org.apache.cassandra.utils.ByteBufferUtil;
  * if this matters, you can subclass RP to use a stronger hash, or use a non-lossy tokenization scheme (as in the
  * OrderPreservingPartitioner classes).
  */
-public class DecoratedKey extends RowPosition
+public abstract class DecoratedKey implements RowPosition
 {
     public static final Comparator<DecoratedKey> comparator = new Comparator<DecoratedKey>()
     {
@@ -43,20 +44,18 @@ public class DecoratedKey extends RowPosition
         }
     };
 
-    public final Token token;
-    public final ByteBuffer key;
+    private final Token token;
 
-    public DecoratedKey(Token token, ByteBuffer key)
+    public DecoratedKey(Token token)
     {
-        assert token != null && key != null;
+        assert token != null;
         this.token = token;
-        this.key = key;
     }
 
     @Override
     public int hashCode()
     {
-        return key.hashCode(); // hash of key is enough
+        return getKey().hashCode(); // hash of key is enough
     }
 
     @Override
@@ -64,12 +63,11 @@ public class DecoratedKey extends RowPosition
     {
         if (this == obj)
             return true;
-        if (obj == null || this.getClass() != obj.getClass())
+        if (obj == null || !(obj instanceof DecoratedKey))
             return false;
 
         DecoratedKey other = (DecoratedKey)obj;
-
-        return ByteBufferUtil.compareUnsigned(key, other.key) == 0; // we compare faster than BB.equals for array backed BB
+        return ByteBufferUtil.compareUnsigned(getKey(), other.getKey()) == 0; // we compare faster than BB.equals for array backed BB
     }
 
     public int compareTo(RowPosition pos)
@@ -82,8 +80,8 @@ public class DecoratedKey extends RowPosition
             return -pos.compareTo(this);
 
         DecoratedKey otherKey = (DecoratedKey) pos;
-        int cmp = token.compareTo(otherKey.getToken());
-        return cmp == 0 ? ByteBufferUtil.compareUnsigned(key, otherKey.key) : cmp;
+        int cmp = getToken().compareTo(otherKey.getToken());
+        return cmp == 0 ? ByteBufferUtil.compareUnsigned(getKey(), otherKey.getKey()) : cmp;
     }
 
     public static int compareTo(IPartitioner partitioner, ByteBuffer key, RowPosition position)
@@ -94,7 +92,7 @@ public class DecoratedKey extends RowPosition
 
         DecoratedKey otherKey = (DecoratedKey) position;
         int cmp = partitioner.getToken(key).compareTo(otherKey.getToken());
-        return cmp == 0 ? ByteBufferUtil.compareUnsigned(key, otherKey.key) : cmp;
+        return cmp == 0 ? ByteBufferUtil.compareUnsigned(key, otherKey.getKey()) : cmp;
     }
 
     public boolean isMinimum(IPartitioner partitioner)
@@ -103,6 +101,11 @@ public class DecoratedKey extends RowPosition
         return false;
     }
 
+    public boolean isMinimum()
+    {
+        return isMinimum(StorageService.getPartitioner());
+    }
+
     public RowPosition.Kind kind()
     {
         return RowPosition.Kind.ROW_KEY;
@@ -111,12 +114,14 @@ public class DecoratedKey extends RowPosition
     @Override
     public String toString()
     {
-        String keystring = key == null ? "null" : ByteBufferUtil.bytesToHex(key);
-        return "DecoratedKey(" + token + ", " + keystring + ")";
+        String keystring = getKey() == null ? "null" : ByteBufferUtil.bytesToHex(getKey());
+        return "DecoratedKey(" + getToken() + ", " + keystring + ")";
     }
 
     public Token getToken()
     {
         return token;
     }
+
+    public abstract ByteBuffer getKey();
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/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 351ee4b..6f9a270 100644
--- a/src/java/org/apache/cassandra/db/DefsTables.java
+++ b/src/java/org/apache/cassandra/db/DefsTables.java
@@ -248,7 +248,7 @@ public class DefsTables
             if (newState.hasColumns())
                 updateKeyspace(KSMetaData.fromSchema(new Row(key, newState), Collections.<CFMetaData>emptyList(), new UTMetaData()));
             else
-                keyspacesToDrop.add(AsciiType.instance.getString(key.key));
+                keyspacesToDrop.add(AsciiType.instance.getString(key.getKey()));
         }
 
         return keyspacesToDrop;
@@ -297,7 +297,7 @@ public class DefsTables
             }
             else // has modifications in the nested ColumnFamilies, need to perform nested diff to determine what was really changed
             {
-                String ksName = AsciiType.instance.getString(keyspace.key);
+                String ksName = AsciiType.instance.getString(keyspace.getKey());
 
                 Map<String, CFMetaData> oldCfDefs = new HashMap<String, CFMetaData>();
                 for (CFMetaData cfm : Schema.instance.getKSMetaData(ksName).cfMetaData().values())

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/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
index 00788f8..998c409 100644
--- a/src/java/org/apache/cassandra/db/DeletedCell.java
+++ b/src/java/org/apache/cassandra/db/DeletedCell.java
@@ -17,104 +17,14 @@
  */
 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.concurrent.OpOrder;
 import org.apache.cassandra.utils.memory.AbstractAllocator;
-import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.memory.MemtableAllocator;
 
-public class DeletedCell extends Cell
+public interface 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)
-    {
-        if (cell instanceof DeletedCell)
-            return super.reconcile(cell);
-        return cell.reconcile(this);
-    }
-
-    @Override
-    public Cell localCopy(AbstractAllocator allocator)
-    {
-        return new DeletedCell(name.copy(allocator), allocator.clone(value), timestamp);
-    }
-
-    @Override
-    public int serializationFlags()
-    {
-        return ColumnSerializer.DELETION_MASK;
-    }
+    DeletedCell localCopy(CFMetaData metadata, AbstractAllocator allocator);
 
-    @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");
-    }
+    DeletedCell localCopy(CFMetaData metaData, MemtableAllocator allocator, OpOrder.Group opGroup);
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/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
index 57fac5b..5fc0f94 100644
--- a/src/java/org/apache/cassandra/db/ExpiringCell.java
+++ b/src/java/org/apache/cassandra/db/ExpiringCell.java
@@ -17,16 +17,10 @@
  */
 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.concurrent.OpOrder;
 import org.apache.cassandra.utils.memory.AbstractAllocator;
+import org.apache.cassandra.utils.memory.MemtableAllocator;
 
 /**
  * Alternative to Cell that have an expiring time.
@@ -38,154 +32,13 @@ import org.apache.cassandra.utils.memory.AbstractAllocator;
  * 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 interface 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(AbstractAllocator allocator)
-    {
-        return new ExpiringCell(name.copy(allocator), allocator.clone(value), timestamp, timeToLive, localExpirationTime);
-    }
-
-    @Override
-    public String getString(CellNameType comparator)
-    {
-        return String.format("%s!%d", super.getString(comparator), timeToLive);
-    }
-
-    @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");
-    }
+    public int getTimeToLive();
 
-    @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;
-    }
+    ExpiringCell localCopy(CFMetaData metadata, AbstractAllocator allocator);
 
-    @Override
-    public int hashCode()
-    {
-        int result = super.hashCode();
-        result = 31 * result + localExpirationTime;
-        result = 31 * result + timeToLive;
-        return result;
-    }
+    ExpiringCell localCopy(CFMetaData metaData, MemtableAllocator allocator, OpOrder.Group opGroup);
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/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 4415e63..8c892d6 100644
--- a/src/java/org/apache/cassandra/db/HintedHandOffManager.java
+++ b/src/java/org/apache/cassandra/db/HintedHandOffManager.java
@@ -514,7 +514,7 @@ public class HintedHandOffManager implements HintedHandOffManagerMBean
         List<Row> rows = hintStore.getRangeSlice(range, null, filter, Integer.MAX_VALUE, System.currentTimeMillis());
         for (Row row : rows)
         {
-            UUID hostId = UUIDGen.getUUID(row.key.key);
+            UUID hostId = UUIDGen.getUUID(row.key.getKey());
             InetAddress target = StorageService.instance.getTokenMetadata().getEndpointForHostId(hostId);
             // token may have since been removed (in which case we have just read back a tombstone)
             if (target != null)
@@ -573,7 +573,7 @@ public class HintedHandOffManager implements HintedHandOffManagerMBean
         for (Row row : getHintsSlice(1))
         {
             if (row.cf != null) //ignore removed rows
-                result.addFirst(tokenFactory.toString(row.key.token));
+                result.addFirst(tokenFactory.toString(row.key.getToken()));
         }
         return result;
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/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 31e68c1..17d1364 100644
--- a/src/java/org/apache/cassandra/db/Keyspace.java
+++ b/src/java/org/apache/cassandra/db/Keyspace.java
@@ -410,13 +410,13 @@ public class Keyspace
     public static void indexRow(DecoratedKey key, ColumnFamilyStore cfs, Set<String> idxNames)
     {
         if (logger.isDebugEnabled())
-            logger.debug("Indexing row {} ", cfs.metadata.getKeyValidator().getString(key.key));
+            logger.debug("Indexing row {} ", cfs.metadata.getKeyValidator().getString(key.getKey()));
 
         try (OpOrder.Group opGroup = cfs.keyspace.writeOrder.start())
         {
             Set<SecondaryIndex> indexes = cfs.indexManager.getIndexesByNames(idxNames);
 
-            Iterator<ColumnFamily> pager = QueryPagers.pageRowLocally(cfs, key.key, DEFAULT_PAGE_SIZE);
+            Iterator<ColumnFamily> pager = QueryPagers.pageRowLocally(cfs, key.getKey(), DEFAULT_PAGE_SIZE);
             while (pager.hasNext())
             {
                 ColumnFamily cf = pager.next();
@@ -426,7 +426,7 @@ public class Keyspace
                     if (cfs.indexManager.indexes(cell.name(), indexes))
                         cf2.addColumn(cell);
                 }
-                cfs.indexManager.indexRow(key.key, cf2, opGroup);
+                cfs.indexManager.indexRow(key.getKey(), cf2, opGroup);
             }
         }
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/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 4cf6654..6f4c1c7 100644
--- a/src/java/org/apache/cassandra/db/Memtable.java
+++ b/src/java/org/apache/cassandra/db/Memtable.java
@@ -46,19 +46,16 @@ import org.apache.cassandra.service.ActiveRepairService;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.ObjectSizes;
 import org.apache.cassandra.utils.concurrent.OpOrder;
-import org.apache.cassandra.utils.memory.ContextAllocator;
-import org.apache.cassandra.utils.memory.HeapAllocator;
-import org.apache.cassandra.utils.memory.Pool;
-import org.apache.cassandra.utils.memory.PoolAllocator;
+import org.apache.cassandra.utils.memory.*;
 
 public class Memtable
 {
     private static final Logger logger = LoggerFactory.getLogger(Memtable.class);
 
-    static final Pool memoryPool = DatabaseDescriptor.getMemtableAllocatorPool();
+    static final MemtablePool MEMORY_POOL = DatabaseDescriptor.getMemtableAllocatorPool();
     private static final int ROW_OVERHEAD_HEAP_SIZE;
 
-    private final PoolAllocator allocator;
+    private final MemtableAllocator allocator;
     private final AtomicLong liveDataSize = new AtomicLong(0);
     private final AtomicLong currentOperations = new AtomicLong(0);
 
@@ -85,12 +82,12 @@ public class Memtable
     public Memtable(ColumnFamilyStore cfs)
     {
         this.cfs = cfs;
-        this.allocator = memoryPool.newAllocator();
+        this.allocator = MEMORY_POOL.newAllocator();
         this.initialComparator = cfs.metadata.comparator;
         this.cfs.scheduleFlush();
     }
 
-    public PoolAllocator getAllocator()
+    public MemtableAllocator getAllocator()
     {
         return allocator;
     }
@@ -177,7 +174,7 @@ public class Memtable
         if (previous == null)
         {
             AtomicBTreeColumns empty = cf.cloneMeShallow(AtomicBTreeColumns.factory, false);
-            final DecoratedKey cloneKey = new DecoratedKey(key.token, allocator.clone(key.key, opGroup));
+            final DecoratedKey cloneKey = allocator.clone(key, opGroup);
             // We'll add the columns later. This avoids wasting works if we get beaten in the putIfAbsent
             previous = rows.putIfAbsent(cloneKey, empty);
             if (previous == null)
@@ -185,27 +182,17 @@ public class Memtable
                 previous = empty;
                 // allocate the row overhead after the fact; this saves over allocating and having to free after, but
                 // means we can overshoot our declared limit.
-                int overhead = (int) (cfs.partitioner.getHeapSizeOf(key.token) + ROW_OVERHEAD_HEAP_SIZE);
-                allocator.allocate(overhead, opGroup);
+                int overhead = (int) (cfs.partitioner.getHeapSizeOf(key.getToken()) + ROW_OVERHEAD_HEAP_SIZE);
+                allocator.onHeap().allocate(overhead, opGroup);
             }
             else
             {
-                allocator.free(cloneKey.key);
+                allocator.reclaimer().reclaimImmediately(cloneKey);
             }
         }
 
-        ContextAllocator contextAllocator = allocator.wrap(opGroup);
-        AtomicBTreeColumns.Delta delta = previous.addAllWithSizeDelta(cf, contextAllocator, indexer, new AtomicBTreeColumns.Delta());
-        liveDataSize.addAndGet(delta.dataSize());
+        liveDataSize.addAndGet(previous.addAllWithSizeDelta(cf, allocator, opGroup, indexer));
         currentOperations.addAndGet(cf.getColumnCount() + (cf.isMarkedForDelete() ? 1 : 0) + cf.deletionInfo().rangeCount());
-
-        // allocate or free the delta in column overhead after the fact
-        for (Cell cell : delta.reclaimed())
-        {
-            cell.name.free(allocator);
-            allocator.free(cell.value);
-        }
-        allocator.allocate((int) delta.excessHeapSize(), opGroup);
     }
 
     // for debugging
@@ -256,10 +243,10 @@ public class Memtable
                 Map.Entry<? extends RowPosition, ? extends ColumnFamily> entry = iter.next();
                 // Actual stored key should be true DecoratedKey
                 assert entry.getKey() instanceof DecoratedKey;
-                if (memoryPool.needToCopyOnHeap())
+                if (MEMORY_POOL.needToCopyOnHeap())
                 {
                     DecoratedKey key = (DecoratedKey) entry.getKey();
-                    key = new DecoratedKey(key.token, HeapAllocator.instance.clone(key.key));
+                    key = new BufferDecoratedKey(key.getToken(), HeapAllocator.instance.clone(key.getKey()));
                     ColumnFamily cells = ArrayBackedSortedColumns.localCopy(entry.getValue(), HeapAllocator.instance);
                     entry = new AbstractMap.SimpleImmutableEntry<>(key, cells);
                 }
@@ -307,7 +294,7 @@ public class Memtable
             {
                 //  make sure we don't write non-sensical keys
                 assert key instanceof DecoratedKey;
-                keySize += ((DecoratedKey)key).key.remaining();
+                keySize += ((DecoratedKey)key).getKey().remaining();
             }
             estimatedSize = (long) ((keySize // index entries
                                     + keySize // keys in data file
@@ -410,16 +397,20 @@ public class Memtable
     static
     {
         // calculate row overhead
+        final OpOrder.Group group = new OpOrder().start();
         int rowOverhead;
+        MemtableAllocator allocator = MEMORY_POOL.newAllocator();
         ConcurrentNavigableMap<RowPosition, Object> rows = new ConcurrentSkipListMap<>();
         final int count = 100000;
         final Object val = new Object();
         for (int i = 0 ; i < count ; i++)
-            rows.put(new DecoratedKey(new LongToken((long) i), ByteBufferUtil.EMPTY_BYTE_BUFFER), val);
+            rows.put(allocator.clone(new BufferDecoratedKey(new LongToken((long) i), ByteBufferUtil.EMPTY_BYTE_BUFFER), group), val);
         double avgSize = ObjectSizes.measureDeep(rows) / (double) count;
         rowOverhead = (int) ((avgSize - Math.floor(avgSize)) < 0.05 ? Math.floor(avgSize) : Math.ceil(avgSize));
         rowOverhead -= ObjectSizes.measureDeep(new LongToken((long) 0));
-        rowOverhead += AtomicBTreeColumns.HEAP_SIZE;
+        rowOverhead += AtomicBTreeColumns.EMPTY_SIZE;
+        allocator.setDiscarding();
+        allocator.setDiscarded();
         ROW_OVERHEAD_HEAP_SIZE = rowOverhead;
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/db/Mutation.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Mutation.java b/src/java/org/apache/cassandra/db/Mutation.java
index 6fae9b0..b64c675 100644
--- a/src/java/org/apache/cassandra/db/Mutation.java
+++ b/src/java/org/apache/cassandra/db/Mutation.java
@@ -64,7 +64,7 @@ public class Mutation implements IMutation
 
     public Mutation(String keyspaceName, Row row)
     {
-        this(keyspaceName, row.key.key, row.cf);
+        this(keyspaceName, row.key.getKey(), row.cf);
     }
 
     protected Mutation(String keyspaceName, ByteBuffer key, Map<UUID, ColumnFamily> modifications)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/db/NativeCell.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/NativeCell.java b/src/java/org/apache/cassandra/db/NativeCell.java
new file mode 100644
index 0000000..1c8ebd9
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/NativeCell.java
@@ -0,0 +1,88 @@
+/*
+ * 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 org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.db.composites.CellName;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.ObjectSizes;
+import org.apache.cassandra.utils.concurrent.OpOrder;
+import org.apache.cassandra.utils.memory.AbstractAllocator;
+import org.apache.cassandra.utils.memory.MemtableAllocator;
+import org.apache.cassandra.utils.memory.NativeAllocator;
+
+public class NativeCell extends AbstractNativeCell
+{
+    private static final long SIZE = ObjectSizes.measure(new NativeCell());
+
+    NativeCell()
+    {}
+
+    public NativeCell(NativeAllocator allocator, OpOrder.Group writeOp, Cell copyOf)
+    {
+        super(allocator, writeOp, copyOf);
+    }
+
+    @Override
+    public CellName name()
+    {
+        return this;
+    }
+
+    @Override
+    public long timestamp()
+    {
+        return getLong(TIMESTAMP_OFFSET);
+    }
+
+    @Override
+    public Cell localCopy(CFMetaData metadata, AbstractAllocator allocator)
+    {
+        return new BufferCell(copy(metadata, allocator), allocator.clone(value()), timestamp());
+    }
+
+    @Override
+    public Cell localCopy(CFMetaData metadata, MemtableAllocator allocator, OpOrder.Group opGroup)
+    {
+        return allocator.clone(this, metadata, opGroup);
+    }
+
+    @Override
+    public void updateDigest(MessageDigest digest)
+    {
+        updateWithName(digest);  // name
+        updateWithValue(digest); // value
+
+        FBUtilities.updateWithLong(digest, timestamp());
+        FBUtilities.updateWithByte(digest, serializationFlags());
+    }
+
+    @Override
+    public long excessHeapSizeExcludingData()
+    {
+        return SIZE;
+    }
+
+    @Override
+    public long unsharedHeapSize()
+    {
+        return SIZE;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/db/NativeCounterCell.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/NativeCounterCell.java b/src/java/org/apache/cassandra/db/NativeCounterCell.java
new file mode 100644
index 0000000..abcf598
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/NativeCounterCell.java
@@ -0,0 +1,190 @@
+/*
+ * 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 org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.db.composites.CellNameType;
+import org.apache.cassandra.serializers.MarshalException;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.ObjectSizes;
+import org.apache.cassandra.utils.concurrent.OpOrder;
+import org.apache.cassandra.utils.memory.AbstractAllocator;
+import org.apache.cassandra.utils.memory.MemtableAllocator;
+import org.apache.cassandra.utils.memory.NativeAllocator;
+
+public class NativeCounterCell extends NativeCell implements CounterCell
+{
+    private static final long SIZE = ObjectSizes.measure(new NativeCounterCell());
+
+    private NativeCounterCell()
+    {}
+
+    public NativeCounterCell(NativeAllocator allocator, OpOrder.Group writeOp, CounterCell copyOf)
+    {
+        super(allocator, writeOp, copyOf);
+    }
+
+    @Override
+    protected void construct(Cell from)
+    {
+        super.construct(from);
+        setLong(internalSize() - 8, ((CounterCell) from).timestampOfLastDelete());
+    }
+
+    @Override
+    protected int postfixSize()
+    {
+        return 8;
+    }
+
+    @Override
+    protected int sizeOf(Cell cell)
+    {
+        return 8 + super.sizeOf(cell);
+    }
+
+    @Override
+    public long timestampOfLastDelete()
+    {
+        return getLong(internalSize() - 8);
+    }
+
+    @Override
+    public long total()
+    {
+        return contextManager.total(value());
+    }
+
+    @Override
+    public boolean hasLegacyShards()
+    {
+        return contextManager.hasLegacyShards(value());
+    }
+
+    @Override
+    public Cell markLocalToBeCleared()
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public Cell diff(Cell cell)
+    {
+        return diff(this, cell);
+    }
+
+    @Override
+    public Cell reconcile(Cell cell)
+    {
+        return reconcile(this, cell);
+    }
+
+    @Override
+    public int serializationFlags()
+    {
+        return ColumnSerializer.COUNTER_MASK;
+    }
+
+    @Override
+    public int cellDataSize()
+    {
+        // A counter column adds 8 bytes for timestampOfLastDelete to Cell.
+        return super.cellDataSize() + TypeSizes.NATIVE.sizeof(timestampOfLastDelete());
+    }
+
+    @Override
+    public int serializedSize(CellNameType type, TypeSizes typeSizes)
+    {
+        return super.serializedSize(type, typeSizes) + typeSizes.sizeof(timestampOfLastDelete());
+    }
+
+    @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());
+    }
+
+    /*
+     * 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)
+    {
+        updateWithName(digest);
+
+        // We don't take the deltas into account in a digest
+        contextManager.updateDigest(digest, value());
+
+        FBUtilities.updateWithLong(digest, timestamp());
+        FBUtilities.updateWithByte(digest, serializationFlags());
+        FBUtilities.updateWithLong(digest, timestampOfLastDelete());
+    }
+
+    @Override
+    public String getString(CellNameType comparator)
+    {
+        return String.format("%s(%s:false:%s@%d!%d)",
+                             getClass().getSimpleName(),
+                             comparator.getString(name()),
+                             contextManager.toString(value()),
+                             timestamp(),
+                             timestampOfLastDelete());
+    }
+
+    @Override
+    public CounterCell localCopy(CFMetaData metadata, AbstractAllocator allocator)
+    {
+        return new BufferCounterCell(copy(metadata, allocator), allocator.clone(value()), timestamp(), timestampOfLastDelete());
+    }
+
+    @Override
+    public CounterCell localCopy(CFMetaData metadata, MemtableAllocator allocator, OpOrder.Group opGroup)
+    {
+        return allocator.clone(this, metadata, opGroup);
+    }
+
+    @Override
+    public long excessHeapSizeExcludingData()
+    {
+        return SIZE;
+    }
+
+    @Override
+    public long unsharedHeapSize()
+    {
+        return SIZE;
+    }
+
+    public boolean equals(Cell cell)
+    {
+        return cell instanceof CounterCell && equals((CounterCell) this);
+    }
+
+    public boolean equals(CounterCell cell)
+    {
+        return super.equals(cell) && timestampOfLastDelete() == cell.timestampOfLastDelete();
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/db/NativeDecoratedKey.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/NativeDecoratedKey.java b/src/java/org/apache/cassandra/db/NativeDecoratedKey.java
new file mode 100644
index 0000000..52aa50c
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/NativeDecoratedKey.java
@@ -0,0 +1,45 @@
+/*
+ * 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.dht.Token;
+import org.apache.cassandra.utils.concurrent.OpOrder;
+import org.apache.cassandra.utils.memory.MemoryUtil;
+import org.apache.cassandra.utils.memory.NativeAllocator;
+
+public class NativeDecoratedKey extends DecoratedKey
+{
+    private final long peer;
+
+    public NativeDecoratedKey(Token token, NativeAllocator allocator, OpOrder.Group writeOp, ByteBuffer key)
+    {
+        super(token);
+        assert key != null;
+        int size = key.remaining();
+        this.peer = allocator.allocate(4 + size, writeOp);
+        MemoryUtil.setInt(peer, size);
+        MemoryUtil.setBytes(peer + 4, key);
+    }
+
+    public ByteBuffer getKey()
+    {
+        return MemoryUtil.getByteBuffer(peer + 4, MemoryUtil.getInt(peer));
+    }
+}


[3/7] Push more of memtable data off-heap

Posted by sl...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/utils/memory/NativeAllocator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/memory/NativeAllocator.java b/src/java/org/apache/cassandra/utils/memory/NativeAllocator.java
new file mode 100644
index 0000000..1b5dcf2
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/memory/NativeAllocator.java
@@ -0,0 +1,258 @@
+/*
+ * 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.utils.memory;
+
+import java.lang.reflect.Field;
+
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.db.Cell;
+import org.apache.cassandra.db.CounterCell;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.DeletedCell;
+import org.apache.cassandra.db.ExpiringCell;
+import org.apache.cassandra.db.NativeCell;
+import org.apache.cassandra.db.NativeCounterCell;
+import org.apache.cassandra.db.NativeDecoratedKey;
+import org.apache.cassandra.db.NativeDeletedCell;
+import org.apache.cassandra.db.NativeExpiringCell;
+import org.apache.cassandra.utils.concurrent.OpOrder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import sun.misc.Unsafe;
+
+public class NativeAllocator extends MemtableAllocator
+{
+    private static final Logger logger = LoggerFactory.getLogger(NativeAllocator.class);
+
+    private final static int REGION_SIZE = 1024 * 1024;
+    private final static int MAX_CLONED_SIZE = 128 * 1024; // bigger than this don't go in the region
+
+    // globally stash any Regions we allocate but are beaten to using, and use these up before allocating any more
+    private static final ConcurrentLinkedQueue<Region> RACE_ALLOCATED = new ConcurrentLinkedQueue<>();
+
+    private final AtomicReference<Region> currentRegion = new AtomicReference<>();
+    private final AtomicInteger regionCount = new AtomicInteger(0);
+    private final ConcurrentLinkedQueue<Region> regions = new ConcurrentLinkedQueue<>();
+    private AtomicLong unslabbed = new AtomicLong(0);
+
+    protected NativeAllocator(NativePool pool)
+    {
+        super(pool.onHeap.newAllocator(), pool.offHeap.newAllocator());
+    }
+
+    @Override
+    public Cell clone(Cell cell, CFMetaData cfm, OpOrder.Group writeOp)
+    {
+        return new NativeCell(this, writeOp, cell);
+    }
+
+    @Override
+    public CounterCell clone(CounterCell cell, CFMetaData cfm, OpOrder.Group writeOp)
+    {
+        return new NativeCounterCell(this, writeOp, cell);
+    }
+
+    @Override
+    public DeletedCell clone(DeletedCell cell, CFMetaData cfm, OpOrder.Group writeOp)
+    {
+        return new NativeDeletedCell(this, writeOp, cell);
+    }
+
+    @Override
+    public ExpiringCell clone(ExpiringCell cell, CFMetaData cfm, OpOrder.Group writeOp)
+    {
+        return new NativeExpiringCell(this, writeOp, cell);
+    }
+
+    public DecoratedKey clone(DecoratedKey key, OpOrder.Group writeOp)
+    {
+        return new NativeDecoratedKey(key.getToken(), this, writeOp, key.getKey());
+    }
+
+    @Override
+    public MemtableAllocator.DataReclaimer reclaimer()
+    {
+        return NO_OP;
+    }
+
+    public long allocate(int size, OpOrder.Group opGroup)
+    {
+        assert size >= 0;
+        offHeap().allocate(size, opGroup);
+        // satisfy large allocations directly from JVM since they don't cause fragmentation
+        // as badly, and fill up our regions quickly
+        if (size > MAX_CLONED_SIZE)
+        {
+            unslabbed.addAndGet(size);
+            Region region = new Region(unsafe.allocateMemory(size), size);
+            regions.add(region);
+
+            long peer;
+            if ((peer = region.allocate(size)) == -1)
+                throw new AssertionError();
+
+            return peer;
+        }
+
+        while (true)
+        {
+            Region region = getRegion();
+
+            long peer;
+            if ((peer = region.allocate(size)) > 0)
+                return peer;
+
+            // not enough space!
+            currentRegion.compareAndSet(region, null);
+        }
+    }
+
+    public void setDiscarded()
+    {
+        for (Region region : regions)
+            unsafe.freeMemory(region.peer);
+        super.setDiscarded();
+    }
+
+    /**
+     * Get the current region, or, if there is no current region, allocate a new one
+     */
+    private Region getRegion()
+    {
+        while (true)
+        {
+            // Try to get the region
+            Region region = currentRegion.get();
+            if (region != null)
+                return region;
+
+            // No current region, so we want to allocate one. We race
+            // against other allocators to CAS in a Region, and if we fail we stash the region for re-use
+            region = RACE_ALLOCATED.poll();
+            if (region == null)
+                region = new Region(unsafe.allocateMemory(REGION_SIZE), REGION_SIZE);
+            if (currentRegion.compareAndSet(null, region))
+            {
+                regions.add(region);
+                regionCount.incrementAndGet();
+                logger.trace("{} regions now allocated in {}", regionCount, this);
+                return region;
+            }
+
+            // someone else won race - that's fine, we'll try to grab theirs
+            // in the next iteration of the loop.
+            RACE_ALLOCATED.add(region);
+        }
+    }
+
+    /**
+     * A region of memory out of which allocations are sliced.
+     *
+     * This serves two purposes:
+     *  - to provide a step between initialization and allocation, so that racing to CAS a
+     *    new region in is harmless
+     *  - encapsulates the allocation offset
+     */
+    private static class Region
+    {
+        /**
+         * Actual underlying data
+         */
+        private final long peer;
+
+        private final long capacity;
+
+        /**
+         * Offset for the next allocation, or the sentinel value -1
+         * which implies that the region is still uninitialized.
+         */
+        private AtomicInteger nextFreeOffset = new AtomicInteger(0);
+
+        /**
+         * Total number of allocations satisfied from this buffer
+         */
+        private AtomicInteger allocCount = new AtomicInteger();
+
+        /**
+         * Create an uninitialized region. Note that memory is not allocated yet, so
+         * this is cheap.
+         *
+         * @param peer peer
+         */
+        private Region(long peer, long capacity)
+        {
+            this.peer = peer;
+            this.capacity = capacity;
+        }
+
+        /**
+         * Try to allocate <code>size</code> bytes from the region.
+         *
+         * @return the successful allocation, or null to indicate not-enough-space
+         */
+        long allocate(int size)
+        {
+            while (true)
+            {
+                int oldOffset = nextFreeOffset.get();
+
+                if (oldOffset + size > capacity) // capacity == remaining
+                    return -1;
+
+                // Try to atomically claim this region
+                if (nextFreeOffset.compareAndSet(oldOffset, oldOffset + size))
+                {
+                    // we got the alloc
+                    allocCount.incrementAndGet();
+                    return peer + oldOffset;
+                }
+                // we raced and lost alloc, try again
+            }
+        }
+
+        @Override
+        public String toString()
+        {
+            return "Region@" + System.identityHashCode(this) +
+                    " allocs=" + allocCount.get() + "waste=" +
+                    (capacity - nextFreeOffset.get());
+        }
+    }
+
+
+    static final Unsafe unsafe;
+
+    static
+    {
+        try
+        {
+            Field field = sun.misc.Unsafe.class.getDeclaredField("theUnsafe");
+            field.setAccessible(true);
+            unsafe = (sun.misc.Unsafe) field.get(null);
+        }
+        catch (Exception e)
+        {
+            throw new AssertionError(e);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/utils/memory/NativePool.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/memory/NativePool.java b/src/java/org/apache/cassandra/utils/memory/NativePool.java
new file mode 100644
index 0000000..012867a
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/memory/NativePool.java
@@ -0,0 +1,39 @@
+/*
+ * 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.utils.memory;
+
+public class NativePool extends MemtablePool
+{
+    public NativePool(long maxOnHeapMemory, long maxOffHeapMemory, float cleanThreshold, Runnable cleaner)
+    {
+        super(maxOnHeapMemory, maxOffHeapMemory, cleanThreshold, cleaner);
+    }
+
+    @Override
+    public boolean needToCopyOnHeap()
+    {
+        return true;
+    }
+
+    @Override
+    public NativeAllocator newAllocator()
+    {
+        return new NativeAllocator(this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/utils/memory/Pool.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/memory/Pool.java b/src/java/org/apache/cassandra/utils/memory/Pool.java
deleted file mode 100644
index aa5e05c..0000000
--- a/src/java/org/apache/cassandra/utils/memory/Pool.java
+++ /dev/null
@@ -1,199 +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.utils.memory;
-
-import java.util.concurrent.atomic.AtomicLongFieldUpdater;
-
-import org.apache.cassandra.utils.concurrent.WaitQueue;
-
-
-/**
- * Represents an amount of memory used for a given purpose, that can be allocated to specific tasks through
- * child PoolAllocator objects.
- */
-public abstract class Pool
-{
-    final PoolCleanerThread<?> cleaner;
-
-    // the total memory used by this pool
-    public final SubPool onHeap;
-    public final SubPool offHeap;
-
-    final WaitQueue hasRoom = new WaitQueue();
-
-    Pool(long maxOnHeapMemory, long maxOffHeapMemory, float cleanThreshold, Runnable cleaner)
-    {
-        this.onHeap = getSubPool(maxOnHeapMemory, cleanThreshold);
-        this.offHeap = getSubPool(maxOffHeapMemory, cleanThreshold);
-        this.cleaner = getCleaner(cleaner);
-        if (this.cleaner != null)
-            this.cleaner.start();
-    }
-
-    SubPool getSubPool(long limit, float cleanThreshold)
-    {
-        return new SubPool(limit, cleanThreshold);
-    }
-
-    PoolCleanerThread<?> getCleaner(Runnable cleaner)
-    {
-        return cleaner == null ? null : new PoolCleanerThread<>(this, cleaner);
-    }
-
-    public abstract boolean needToCopyOnHeap();
-    public abstract PoolAllocator newAllocator();
-
-    /**
-     * Note the difference between acquire() and allocate(); allocate() makes more resources available to all owners,
-     * and acquire() makes shared resources unavailable but still recorded. An Owner must always acquire resources,
-     * but only needs to allocate if there are none already available. This distinction is not always meaningful.
-     */
-    public class SubPool
-    {
-
-        // total memory/resource permitted to allocate
-        public final long limit;
-
-        // ratio of used to spare (both excluding 'reclaiming') at which to trigger a clean
-        public final float cleanThreshold;
-
-        // total bytes allocated and reclaiming
-        volatile long allocated;
-        volatile long reclaiming;
-
-        // a cache of the calculation determining at what allocation threshold we should next clean
-        volatile long nextClean;
-
-        public SubPool(long limit, float cleanThreshold)
-        {
-            this.limit = limit;
-            this.cleanThreshold = cleanThreshold;
-        }
-
-        /** Methods for tracking and triggering a clean **/
-
-        boolean needsCleaning()
-        {
-            // use strictly-greater-than so we don't clean when limit is 0
-            return used() > nextClean && updateNextClean();
-        }
-
-        void maybeClean()
-        {
-            if (needsCleaning() && cleaner != null)
-                cleaner.trigger();
-        }
-
-        private boolean updateNextClean()
-        {
-            while (true)
-            {
-                long current = nextClean;
-                long reclaiming = this.reclaiming;
-                long next =  reclaiming + (long) (this.limit * cleanThreshold);
-                if (current == next || nextCleanUpdater.compareAndSet(this, current, next))
-                    return used() > next;
-            }
-        }
-
-        /** Methods to allocate space **/
-
-        boolean tryAllocate(long size)
-        {
-            while (true)
-            {
-                long cur;
-                if ((cur = allocated) + size > limit)
-                    return false;
-                if (allocatedUpdater.compareAndSet(this, cur, cur + size))
-                    return true;
-            }
-        }
-
-        /**
-         * apply the size adjustment to allocated, bypassing any limits or constraints. If this reduces the
-         * allocated total, we will signal waiters
-         */
-        void adjustAllocated(long size)
-        {
-            if (size == 0)
-                return;
-            while (true)
-            {
-                long cur = allocated;
-                if (allocatedUpdater.compareAndSet(this, cur, cur + size))
-                    return;
-            }
-        }
-
-        // 'acquires' an amount of memory, and maybe also marks it allocated. This method is meant to be overridden
-        // by implementations with a separate concept of acquired/allocated. As this method stands, an acquire
-        // without an allocate is a no-op (acquisition is achieved through allocation), however a release (where size < 0)
-        // is always processed and accounted for in allocated.
-        void adjustAcquired(long size, boolean alsoAllocated)
-        {
-            if (size > 0 || alsoAllocated)
-            {
-                if (alsoAllocated)
-                    adjustAllocated(size);
-                maybeClean();
-            }
-            else if (size < 0)
-            {
-                adjustAllocated(size);
-                hasRoom.signalAll();
-            }
-        }
-
-        // space reclaimed should be released prior to calling this, to avoid triggering unnecessary cleans
-        void adjustReclaiming(long reclaiming)
-        {
-            if (reclaiming == 0)
-                return;
-            reclaimingUpdater.addAndGet(this, reclaiming);
-            if (reclaiming < 0 && updateNextClean() && cleaner != null)
-                cleaner.trigger();
-        }
-
-        public long allocated()
-        {
-            return allocated;
-        }
-
-        public long used()
-        {
-            return allocated;
-        }
-
-        public PoolAllocator.SubAllocator newAllocator()
-        {
-            return new PoolAllocator.SubAllocator(this);
-        }
-
-        public WaitQueue hasRoom()
-        {
-            return hasRoom;
-        }
-    }
-
-    private static final AtomicLongFieldUpdater<SubPool> reclaimingUpdater = AtomicLongFieldUpdater.newUpdater(SubPool.class, "reclaiming");
-    private static final AtomicLongFieldUpdater<SubPool> allocatedUpdater = AtomicLongFieldUpdater.newUpdater(SubPool.class, "allocated");
-    private static final AtomicLongFieldUpdater<SubPool> nextCleanUpdater = AtomicLongFieldUpdater.newUpdater(SubPool.class, "nextClean");
-
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/utils/memory/PoolAllocator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/memory/PoolAllocator.java b/src/java/org/apache/cassandra/utils/memory/PoolAllocator.java
deleted file mode 100644
index aa374fe..0000000
--- a/src/java/org/apache/cassandra/utils/memory/PoolAllocator.java
+++ /dev/null
@@ -1,225 +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.utils.memory;
-
-import java.nio.ByteBuffer;
-
-import org.apache.cassandra.utils.concurrent.OpOrder;
-import java.util.concurrent.atomic.AtomicLongFieldUpdater;
-
-import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.concurrent.WaitQueue;
-
-public abstract class PoolAllocator extends AbstractAllocator
-{
-
-    private final SubAllocator onHeap;
-    private final SubAllocator offHeap;
-    volatile LifeCycle state = LifeCycle.LIVE;
-
-    static enum LifeCycle
-    {
-        LIVE, DISCARDING, DISCARDED;
-        LifeCycle transition(LifeCycle targetState)
-        {
-            switch (targetState)
-            {
-                case DISCARDING:
-                    assert this == LifeCycle.LIVE;
-                    return LifeCycle.DISCARDING;
-                case DISCARDED:
-                    assert this == LifeCycle.DISCARDING;
-                    return LifeCycle.DISCARDED;
-            }
-            throw new IllegalStateException();
-        }
-    }
-
-    PoolAllocator(SubAllocator onHeap, SubAllocator offHeap)
-    {
-        this.onHeap = onHeap;
-        this.offHeap = offHeap;
-    }
-
-    public SubAllocator onHeap()
-    {
-        return onHeap;
-    }
-
-    public SubAllocator offHeap()
-    {
-        return offHeap;
-    }
-
-    /**
-     * Mark this allocator reclaiming; this will permit any outstanding allocations to temporarily
-     * overshoot the maximum memory limit so that flushing can begin immediately
-     */
-    public void setDiscarding()
-    {
-        state = state.transition(LifeCycle.DISCARDING);
-        // mark the memory owned by this allocator as reclaiming
-        onHeap.markAllReclaiming();
-        offHeap.markAllReclaiming();
-    }
-
-    /**
-     * Indicate the memory and resources owned by this allocator are no longer referenced,
-     * and can be reclaimed/reused.
-     */
-    public void setDiscarded()
-    {
-        state = state.transition(LifeCycle.DISCARDED);
-        // release any memory owned by this allocator; automatically signals waiters
-        onHeap.releaseAll();
-        offHeap.releaseAll();
-    }
-
-    public boolean isLive()
-    {
-        return state == LifeCycle.LIVE;
-    }
-
-    public abstract ByteBuffer allocate(int size, OpOrder.Group opGroup);
-    public abstract void free(ByteBuffer name);
-
-    /**
-     * Allocate a slice of the given length.
-     */
-    public ByteBuffer clone(ByteBuffer buffer, OpOrder.Group opGroup)
-    {
-        assert buffer != null;
-        if (buffer.remaining() == 0)
-            return ByteBufferUtil.EMPTY_BYTE_BUFFER;
-        ByteBuffer cloned = allocate(buffer.remaining(), opGroup);
-
-        cloned.mark();
-        cloned.put(buffer.duplicate());
-        cloned.reset();
-        return cloned;
-    }
-
-    public ContextAllocator wrap(OpOrder.Group opGroup)
-    {
-        return new ContextAllocator(opGroup, this);
-    }
-
-    /** Mark the BB as unused, permitting it to be reclaimed */
-    public static final class SubAllocator
-    {
-        // the tracker we are owning memory from
-        private final Pool.SubPool parent;
-
-        // the amount of memory/resource owned by this object
-        private volatile long owns;
-        // the amount of memory we are reporting to collect; this may be inaccurate, but is close
-        // and is used only to ensure that once we have reclaimed we mark the tracker with the same amount
-        private volatile long reclaiming;
-
-        SubAllocator(Pool.SubPool parent)
-        {
-            this.parent = parent;
-        }
-
-        // should only be called once we know we will never allocate to the object again.
-        // currently no corroboration/enforcement of this is performed.
-        void releaseAll()
-        {
-            parent.adjustAcquired(-ownsUpdater.getAndSet(this, 0), false);
-            parent.adjustReclaiming(-reclaimingUpdater.getAndSet(this, 0));
-        }
-
-        // allocate memory in the tracker, and mark ourselves as owning it
-        public void allocate(long size, OpOrder.Group opGroup)
-        {
-            while (true)
-            {
-                if (parent.tryAllocate(size))
-                {
-                    acquired(size);
-                    return;
-                }
-                WaitQueue.Signal signal = opGroup.isBlockingSignal(parent.hasRoom().register());
-                boolean allocated = parent.tryAllocate(size);
-                if (allocated || opGroup.isBlocking())
-                {
-                    signal.cancel();
-                    if (allocated) // if we allocated, take ownership
-                        acquired(size);
-                    else // otherwise we're blocking so we're permitted to overshoot our constraints, to just allocate without blocking
-                        allocated(size);
-                    return;
-                }
-                else
-                    signal.awaitUninterruptibly();
-            }
-        }
-
-        // retroactively mark an amount allocated amd acquired in the tracker, and owned by us
-        void allocated(long size)
-        {
-            parent.adjustAcquired(size, true);
-            ownsUpdater.addAndGet(this, size);
-        }
-
-        // retroactively mark an amount acquired in the tracker, and owned by us
-        void acquired(long size)
-        {
-            parent.adjustAcquired(size, false);
-            ownsUpdater.addAndGet(this, size);
-        }
-
-        void release(long size)
-        {
-            parent.adjustAcquired(-size, false);
-            ownsUpdater.addAndGet(this, -size);
-        }
-
-        // mark everything we currently own as reclaiming, both here and in our parent
-        void markAllReclaiming()
-        {
-            while (true)
-            {
-                long cur = owns;
-                long prev = reclaiming;
-                if (reclaimingUpdater.compareAndSet(this, prev, cur))
-                {
-                    parent.adjustReclaiming(cur - prev);
-                    return;
-                }
-            }
-        }
-
-        public long owns()
-        {
-            return owns;
-        }
-
-        public float ownershipRatio()
-        {
-            float r = owns / (float) parent.limit;
-            if (Float.isNaN(r))
-                return 0;
-            return r;
-        }
-
-        private static final AtomicLongFieldUpdater<SubAllocator> ownsUpdater = AtomicLongFieldUpdater.newUpdater(SubAllocator.class, "owns");
-        private static final AtomicLongFieldUpdater<SubAllocator> reclaimingUpdater = AtomicLongFieldUpdater.newUpdater(SubAllocator.class, "reclaiming");
-    }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/utils/memory/PoolCleanerThread.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/memory/PoolCleanerThread.java b/src/java/org/apache/cassandra/utils/memory/PoolCleanerThread.java
deleted file mode 100644
index 68b0c20..0000000
--- a/src/java/org/apache/cassandra/utils/memory/PoolCleanerThread.java
+++ /dev/null
@@ -1,73 +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.utils.memory;
-
-import org.apache.cassandra.utils.concurrent.WaitQueue;
-
-/**
- * A thread that reclaims memory from a Pool on demand.  The actual reclaiming work is delegated to the
- * cleaner Runnable, e.g., FlushLargestColumnFamily
- */
-class PoolCleanerThread<P extends Pool> extends Thread
-{
-    /** The pool we're cleaning */
-    final P pool;
-
-    /** should ensure that at least some memory has been marked reclaiming after completion */
-    final Runnable cleaner;
-
-    /** signalled whenever needsCleaning() may return true */
-    final WaitQueue wait = new WaitQueue();
-
-    PoolCleanerThread(P pool, Runnable cleaner)
-    {
-        super(pool.getClass().getSimpleName() + "Cleaner");
-        this.pool = pool;
-        this.cleaner = cleaner;
-    }
-
-    boolean needsCleaning()
-    {
-        return pool.onHeap.needsCleaning() || pool.offHeap.needsCleaning();
-    }
-
-    // should ONLY be called when we really think it already needs cleaning
-    void trigger()
-    {
-        wait.signal();
-    }
-
-    @Override
-    public void run()
-    {
-        while (true)
-        {
-            while (!needsCleaning())
-            {
-                final WaitQueue.Signal signal = wait.register();
-                if (!needsCleaning())
-                    signal.awaitUninterruptibly();
-                else
-                    signal.cancel();
-            }
-
-            cleaner.run();
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/utils/memory/SlabAllocator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/memory/SlabAllocator.java b/src/java/org/apache/cassandra/utils/memory/SlabAllocator.java
index a90357c..19334ee 100644
--- a/src/java/org/apache/cassandra/utils/memory/SlabAllocator.java
+++ b/src/java/org/apache/cassandra/utils/memory/SlabAllocator.java
@@ -44,7 +44,7 @@ import sun.nio.ch.DirectBuffer;
  * interleaved throughout the heap, and the old generation gets progressively
  * more fragmented until a stop-the-world compacting collection occurs.
  */
-public class SlabAllocator extends PoolAllocator
+public class SlabAllocator extends MemtableBufferAllocator
 {
     private static final Logger logger = LoggerFactory.getLogger(SlabAllocator.class);
 
@@ -54,7 +54,7 @@ public class SlabAllocator extends PoolAllocator
     // globally stash any Regions we allocate but are beaten to using, and use these up before allocating any more
     private static final ConcurrentLinkedQueue<Region> RACE_ALLOCATED = new ConcurrentLinkedQueue<>();
 
-    private final AtomicReference<Region> currentRegion = new AtomicReference<Region>();
+    private final AtomicReference<Region> currentRegion = new AtomicReference<>();
     private final AtomicInteger regionCount = new AtomicInteger(0);
 
     // this queue is used to keep references to off-heap allocated regions so that we can free them when we are discarded
@@ -106,9 +106,9 @@ public class SlabAllocator extends PoolAllocator
         }
     }
 
-    public void free(ByteBuffer name)
+    public DataReclaimer reclaimer()
     {
-        // have to assume we cannot free the memory here, and just reclaim it all when we flush
+        return NO_OP;
     }
 
     public void setDiscarded()
@@ -150,6 +150,11 @@ public class SlabAllocator extends PoolAllocator
         }
     }
 
+    protected AbstractAllocator allocator(OpOrder.Group writeOp)
+    {
+        return new ContextAllocator(writeOp, this);
+    }
+
     /**
      * A region of memory out of which allocations are sliced.
      *

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/src/java/org/apache/cassandra/utils/memory/SlabPool.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/memory/SlabPool.java b/src/java/org/apache/cassandra/utils/memory/SlabPool.java
index 7276e57..c5c44e1 100644
--- a/src/java/org/apache/cassandra/utils/memory/SlabPool.java
+++ b/src/java/org/apache/cassandra/utils/memory/SlabPool.java
@@ -18,18 +18,17 @@
  */
 package org.apache.cassandra.utils.memory;
 
-
-public class SlabPool extends Pool
+public class SlabPool extends MemtablePool
 {
-
     final boolean allocateOnHeap;
+
     public SlabPool(long maxOnHeapMemory, long maxOffHeapMemory, float cleanupThreshold, Runnable cleaner)
     {
         super(maxOnHeapMemory, maxOffHeapMemory, cleanupThreshold, cleaner);
         this.allocateOnHeap = maxOffHeapMemory == 0;
     }
 
-    public SlabAllocator newAllocator()
+    public MemtableAllocator newAllocator()
     {
         return new SlabAllocator(onHeap.newAllocator(), offHeap.newAllocator(), allocateOnHeap);
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/test/conf/cassandra.yaml
----------------------------------------------------------------------
diff --git a/test/conf/cassandra.yaml b/test/conf/cassandra.yaml
index 37b0b96..b766a64 100644
--- a/test/conf/cassandra.yaml
+++ b/test/conf/cassandra.yaml
@@ -3,7 +3,7 @@
 # Consider the effects on 'o.a.c.i.s.LegacySSTableTest' before changing schemas in this file.
 #
 cluster_name: Test Cluster
-memtable_allocation_type: offheap_buffers
+memtable_allocation_type: offheap_objects
 in_memory_compaction_limit_in_mb: 1
 commitlog_sync: batch
 commitlog_sync_batch_window_in_ms: 1.0

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/test/long/org/apache/cassandra/db/LongFlushMemtableTest.java
----------------------------------------------------------------------
diff --git a/test/long/org/apache/cassandra/db/LongFlushMemtableTest.java b/test/long/org/apache/cassandra/db/LongFlushMemtableTest.java
index ba59404..4bb8fdd 100644
--- a/test/long/org/apache/cassandra/db/LongFlushMemtableTest.java
+++ b/test/long/org/apache/cassandra/db/LongFlushMemtableTest.java
@@ -54,7 +54,7 @@ public class LongFlushMemtableTest extends SchemaLoader
                 ColumnFamily cf = ArrayBackedSortedColumns.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 Cell(Util.cellname("c"), value));
+                cf.addColumn(new BufferCell(Util.cellname("c"), value));
                 rm.add(cf);
                 rm.applyUnsafe();
             }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/test/long/org/apache/cassandra/db/LongKeyspaceTest.java
----------------------------------------------------------------------
diff --git a/test/long/org/apache/cassandra/db/LongKeyspaceTest.java b/test/long/org/apache/cassandra/db/LongKeyspaceTest.java
index 327ff47..7a5b837 100644
--- a/test/long/org/apache/cassandra/db/LongKeyspaceTest.java
+++ b/test/long/org/apache/cassandra/db/LongKeyspaceTest.java
@@ -37,7 +37,7 @@ public class LongKeyspaceTest extends SchemaLoader
 
         for (int i = 1; i < 5000; i += 100)
         {
-            Mutation rm = new Mutation("Keyspace1", Util.dk("key" + i).key);
+            Mutation rm = new Mutation("Keyspace1", Util.dk("key" + i).getKey());
             ColumnFamily cf = ArrayBackedSortedColumns.factory.create("Keyspace1", "Standard1");
             for (int j = 0; j < i; j++)
                 cf.addColumn(column("c" + j, "v" + j, 1L));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/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 35c2b5e..94bc09f 100644
--- a/test/long/org/apache/cassandra/db/compaction/LongCompactionsTest.java
+++ b/test/long/org/apache/cassandra/db/compaction/LongCompactionsTest.java
@@ -127,7 +127,7 @@ public class LongCompactionsTest extends SchemaLoader
         for (int j = 0; j < SSTABLES; j++) {
             for (int i = 0; i < ROWS_PER_SSTABLE; i++) {
                 DecoratedKey key = Util.dk(String.valueOf(i % 2));
-                Mutation rm = new Mutation(KEYSPACE1, key.key);
+                Mutation rm = new Mutation(KEYSPACE1, key.getKey());
                 long timestamp = j * ROWS_PER_SSTABLE + i;
                 rm.add("Standard1", Util.cellname(String.valueOf(i / 2)),
                        ByteBufferUtil.EMPTY_BYTE_BUFFER,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/test/long/org/apache/cassandra/db/compaction/LongLeveledCompactionStrategyTest.java
----------------------------------------------------------------------
diff --git a/test/long/org/apache/cassandra/db/compaction/LongLeveledCompactionStrategyTest.java b/test/long/org/apache/cassandra/db/compaction/LongLeveledCompactionStrategyTest.java
index 90e7123..b071001 100644
--- a/test/long/org/apache/cassandra/db/compaction/LongLeveledCompactionStrategyTest.java
+++ b/test/long/org/apache/cassandra/db/compaction/LongLeveledCompactionStrategyTest.java
@@ -54,7 +54,7 @@ public class LongLeveledCompactionStrategyTest extends SchemaLoader
         for (int r = 0; r < rows; r++)
         {
             DecoratedKey key = Util.dk(String.valueOf(r));
-            Mutation rm = new Mutation(ksname, key.key);
+            Mutation rm = new Mutation(ksname, key.getKey());
             for (int c = 0; c < columns; c++)
             {
                 rm.add(cfname, Util.cellname("column" + c), value, 0);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/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 fe80009..d2fe949 100644
--- a/test/unit/org/apache/cassandra/Util.java
+++ b/test/unit/org/apache/cassandra/Util.java
@@ -77,7 +77,7 @@ public class Util
 
     public static RowPosition rp(String key, IPartitioner partitioner)
     {
-        return RowPosition.forKey(ByteBufferUtil.bytes(key), partitioner);
+        return RowPosition.ForKey.get(ByteBufferUtil.bytes(key), partitioner);
     }
 
     public static CellName cellname(ByteBuffer... bbs)
@@ -108,17 +108,17 @@ public class Util
 
     public static Cell column(String name, String value, long timestamp)
     {
-        return new Cell(cellname(name), ByteBufferUtil.bytes(value), timestamp);
+        return new BufferCell(cellname(name), ByteBufferUtil.bytes(value), timestamp);
     }
 
     public static Cell expiringColumn(String name, String value, long timestamp, int ttl)
     {
-        return new ExpiringCell(cellname(name), ByteBufferUtil.bytes(value), timestamp, ttl);
+        return new BufferExpiringCell(cellname(name), ByteBufferUtil.bytes(value), timestamp, ttl);
     }
 
     public static Cell counterColumn(String name, long value, long timestamp)
     {
-        return new CounterUpdateCell(cellname(name), value, timestamp);
+        return new BufferCounterUpdateCell(cellname(name), value, timestamp);
     }
 
     public static Token token(String key)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/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 2e1876f..94738ac 100644
--- a/test/unit/org/apache/cassandra/config/DefsTest.java
+++ b/test/unit/org/apache/cassandra/config/DefsTest.java
@@ -20,7 +20,6 @@ package org.apache.cassandra.config;
 
 import java.io.File;
 import java.nio.ByteBuffer;
-import java.util.*;
 
 import org.apache.cassandra.OrderedJUnit4ClassRunner;
 import org.apache.cassandra.SchemaLoader;
@@ -170,7 +169,7 @@ public class DefsTest extends SchemaLoader
         // now read and write to it.
         CellName col0 = cellname("col0");
         DecoratedKey dk = Util.dk("key0");
-        Mutation rm = new Mutation(ks, dk.key);
+        Mutation rm = new Mutation(ks, dk.getKey());
         rm.add(cf, col0, ByteBufferUtil.bytes("value0"), 1L);
         rm.apply();
         ColumnFamilyStore store = Keyspace.open(ks).getColumnFamilyStore(cf);
@@ -194,7 +193,7 @@ public class DefsTest extends SchemaLoader
         assert cfm != null;
 
         // write some data, force a flush, then verify that files exist on disk.
-        Mutation rm = new Mutation(ks.name, dk.key);
+        Mutation rm = new Mutation(ks.name, dk.getKey());
         for (int i = 0; i < 100; i++)
             rm.add(cfm.cfName, cellname("col" + i), ByteBufferUtil.bytes("anyvalue"), 1L);
         rm.apply();
@@ -208,7 +207,7 @@ public class DefsTest extends SchemaLoader
         assert !Schema.instance.getKSMetaData(ks.name).cfMetaData().containsKey(cfm.cfName);
 
         // any write should fail.
-        rm = new Mutation(ks.name, dk.key);
+        rm = new Mutation(ks.name, dk.getKey());
         boolean success = true;
         try
         {
@@ -244,7 +243,7 @@ public class DefsTest extends SchemaLoader
 
         // test reads and writes.
         CellName col0 = cellname("col0");
-        Mutation rm = new Mutation(newCf.ksName, dk.key);
+        Mutation rm = new Mutation(newCf.ksName, dk.getKey());
         rm.add(newCf.cfName, col0, ByteBufferUtil.bytes("value0"), 1L);
         rm.apply();
         ColumnFamilyStore store = Keyspace.open(newCf.ksName).getColumnFamilyStore(newCf.cfName);
@@ -268,7 +267,7 @@ public class DefsTest extends SchemaLoader
         assert cfm != null;
 
         // write some data, force a flush, then verify that files exist on disk.
-        Mutation rm = new Mutation(ks.name, dk.key);
+        Mutation rm = new Mutation(ks.name, dk.getKey());
         for (int i = 0; i < 100; i++)
             rm.add(cfm.cfName, cellname("col" + i), ByteBufferUtil.bytes("anyvalue"), 1L);
         rm.apply();
@@ -282,7 +281,7 @@ public class DefsTest extends SchemaLoader
         assert Schema.instance.getKSMetaData(ks.name) == null;
 
         // write should fail.
-        rm = new Mutation(ks.name, dk.key);
+        rm = new Mutation(ks.name, dk.getKey());
         boolean success = true;
         try
         {
@@ -319,7 +318,7 @@ public class DefsTest extends SchemaLoader
         assert cfm != null;
 
         // write some data
-        Mutation rm = new Mutation(ks.name, dk.key);
+        Mutation rm = new Mutation(ks.name, dk.getKey());
         for (int i = 0; i < 100; i++)
             rm.add(cfm.cfName, cellname("col" + i), ByteBufferUtil.bytes("anyvalue"), 1L);
         rm.apply();
@@ -353,7 +352,7 @@ public class DefsTest extends SchemaLoader
         // now read and write to it.
         CellName col0 = cellname("col0");
         DecoratedKey dk = Util.dk("key0");
-        Mutation rm = new Mutation(newKs.name, dk.key);
+        Mutation rm = new Mutation(newKs.name, dk.getKey());
         rm.add(newCf.cfName, col0, ByteBufferUtil.bytes("value0"), 1L);
         rm.apply();
         ColumnFamilyStore store = Keyspace.open(newKs.name).getColumnFamilyStore(newCf.cfName);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/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 a1c98f3..83a58e4 100644
--- a/test/unit/org/apache/cassandra/db/ArrayBackedSortedColumnsTest.java
+++ b/test/unit/org/apache/cassandra/db/ArrayBackedSortedColumnsTest.java
@@ -54,7 +54,7 @@ public class ArrayBackedSortedColumnsTest extends SchemaLoader
         int[] values = new int[]{ 1, 2, 2, 3 };
 
         for (int i = 0; i < values.length; ++i)
-            map.addColumn(new Cell(type.makeCellName(values[reversed ? values.length - 1 - i : i])));
+            map.addColumn(new BufferCell(type.makeCellName(values[reversed ? values.length - 1 - i : i])));
 
         Iterator<Cell> iter = map.iterator();
         assertEquals("1st column", 1, iter.next().name().toByteBuffer().getInt(0));
@@ -76,7 +76,7 @@ public class ArrayBackedSortedColumnsTest extends SchemaLoader
 
         int[] values = new int[]{ 1, 2, 1, 3, 4, 4, 5, 5, 1, 2, 6, 6, 6, 1, 2, 3 };
         for (int i = 0; i < values.length; ++i)
-            cells.addColumn(new Cell(type.makeCellName(values[reversed ? values.length - 1 - i : i])));
+            cells.addColumn(new BufferCell(type.makeCellName(values[reversed ? values.length - 1 - i : i])));
 
         assertEquals(6, cells.getColumnCount());
 
@@ -91,7 +91,7 @@ public class ArrayBackedSortedColumnsTest extends SchemaLoader
         // Add more values
         values = new int[]{ 11, 15, 12, 12, 12, 16, 10, 8, 8, 7, 4, 4, 5 };
         for (int i = 0; i < values.length; ++i)
-            cells.addColumn(new Cell(type.makeCellName(values[reversed ? values.length - 1 - i : i])));
+            cells.addColumn(new BufferCell(type.makeCellName(values[reversed ? values.length - 1 - i : i])));
 
         assertEquals(13, cells.getColumnCount());
 
@@ -125,7 +125,7 @@ public class ArrayBackedSortedColumnsTest extends SchemaLoader
 
         int[] values = new int[]{ -1, 20, 44, 55, 27, 27, 17, 1, 9, 89, 33, 44, 0, 9 };
         for (int i = 0; i < values.length; ++i)
-            cells.addColumn(new Cell(type.makeCellName(values[reversed ? values.length - 1 - i : i])));
+            cells.addColumn(new BufferCell(type.makeCellName(values[reversed ? values.length - 1 - i : i])));
 
         for (int i : values)
             assertEquals(i, cells.getColumn(type.makeCellName(i)).name().toByteBuffer().getInt(0));
@@ -148,10 +148,10 @@ public class ArrayBackedSortedColumnsTest extends SchemaLoader
         int[] values2 = new int[]{ 2, 4, 5, 6 };
 
         for (int i = 0; i < values1.length; ++i)
-            map.addColumn(new Cell(type.makeCellName(values1[reversed ? values1.length - 1 - i : i])));
+            map.addColumn(new BufferCell(type.makeCellName(values1[reversed ? values1.length - 1 - i : i])));
 
         for (int i = 0; i < values2.length; ++i)
-            map2.addColumn(new Cell(type.makeCellName(values2[reversed ? values2.length - 1 - i : i])));
+            map2.addColumn(new BufferCell(type.makeCellName(values2[reversed ? values2.length - 1 - i : i])));
 
         map2.addAll(map);
 
@@ -179,12 +179,12 @@ public class ArrayBackedSortedColumnsTest extends SchemaLoader
 
         List<Cell> sorted = new ArrayList<>();
         for (int v : values)
-            sorted.add(new Cell(type.makeCellName(v)));
+            sorted.add(new BufferCell(type.makeCellName(v)));
         List<Cell> reverseSorted = new ArrayList<>(sorted);
         Collections.reverse(reverseSorted);
 
         for (int i = 0; i < values.length; ++i)
-            map.addColumn(new Cell(type.makeCellName(values[reversed ? values.length - 1 - i : i])));
+            map.addColumn(new BufferCell(type.makeCellName(values[reversed ? values.length - 1 - i : i])));
 
         assertSame(sorted, map.getSortedColumns());
         assertSame(reverseSorted, map.getReverseSortedColumns());
@@ -205,7 +205,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 Cell(type.makeCellName(values[reversed ? values.length - 1 - i : i])));
+            map.addColumn(new BufferCell(type.makeCellName(values[reversed ? values.length - 1 - i : i])));
 
         assertSame(new int[]{ 3, 2, 1 }, map.reverseIterator(new ColumnSlice[]{ new ColumnSlice(type.make(3), Composites.EMPTY) }));
         assertSame(new int[]{ 3, 2, 1 }, map.reverseIterator(new ColumnSlice[]{ new ColumnSlice(type.make(4), Composites.EMPTY) }));
@@ -251,7 +251,7 @@ public class ArrayBackedSortedColumnsTest extends SchemaLoader
         int[] values = new int[]{ 1, 2, 2, 3 };
 
         for (int i = 0; i < values.length; ++i)
-            map.addColumn(new Cell(type.makeCellName(values[reversed ? values.length - 1 - i : i])));
+            map.addColumn(new BufferCell(type.makeCellName(values[reversed ? values.length - 1 - i : i])));
 
         Iterator<Cell> iter = map.getReverseSortedColumns().iterator();
         assertTrue(iter.hasNext());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/test/unit/org/apache/cassandra/db/CollationControllerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/CollationControllerTest.java b/test/unit/org/apache/cassandra/db/CollationControllerTest.java
index fc92aae..22c60b8 100644
--- a/test/unit/org/apache/cassandra/db/CollationControllerTest.java
+++ b/test/unit/org/apache/cassandra/db/CollationControllerTest.java
@@ -40,26 +40,26 @@ public class CollationControllerTest extends SchemaLoader
         DecoratedKey dk = Util.dk("key1");
         
         // add data
-        rm = new Mutation(keyspace.getName(), dk.key);
+        rm = new Mutation(keyspace.getName(), dk.getKey());
         rm.add(cfs.name, Util.cellname("Column1"), ByteBufferUtil.bytes("asdf"), 0);
         rm.apply();
         cfs.forceBlockingFlush();
         
         // remove
-        rm = new Mutation(keyspace.getName(), dk.key);
+        rm = new Mutation(keyspace.getName(), dk.getKey());
         rm.delete(cfs.name, 10);
         rm.apply();
         
         // add another mutation because sstable maxtimestamp isn't set
         // correctly during flush if the most recent mutation is a row delete
-        rm = new Mutation(keyspace.getName(), Util.dk("key2").key);
+        rm = new Mutation(keyspace.getName(), Util.dk("key2").getKey());
         rm.add(cfs.name, Util.cellname("Column1"), ByteBufferUtil.bytes("zxcv"), 20);
         rm.apply();
         
         cfs.forceBlockingFlush();
 
         // add yet one more mutation
-        rm = new Mutation(keyspace.getName(), dk.key);
+        rm = new Mutation(keyspace.getName(), dk.getKey());
         rm.add(cfs.name, Util.cellname("Column1"), ByteBufferUtil.bytes("foobar"), 30);
         rm.apply();
         cfs.forceBlockingFlush();
@@ -92,13 +92,13 @@ public class CollationControllerTest extends SchemaLoader
         CellName cellName = Util.cellname("Column1");
 
         // add data
-        rm = new Mutation(keyspace.getName(), dk.key);
+        rm = new Mutation(keyspace.getName(), dk.getKey());
         rm.add(cfs.name, cellName, ByteBufferUtil.bytes("asdf"), 0);
         rm.apply();
         cfs.forceBlockingFlush();
 
         // remove
-        rm = new Mutation(keyspace.getName(), dk.key);
+        rm = new Mutation(keyspace.getName(), dk.getKey());
         rm.delete(cfs.name, cellName, 0);
         rm.apply();
         cfs.forceBlockingFlush();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/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 d180b82..f812849 100644
--- a/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
+++ b/test/unit/org/apache/cassandra/db/ColumnFamilyStoreTest.java
@@ -202,7 +202,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
                                              Util.namesFilter(cfs, "asdf"),
                                              10);
         assertEquals(1, result.size());
-        assert result.get(0).key.key.equals(ByteBufferUtil.bytes("key2"));
+        assert result.get(0).key.getKey().equals(ByteBufferUtil.bytes("key2"));
     }
 
     @Test
@@ -243,10 +243,10 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         assert rows != null;
         assert rows.size() == 2 : StringUtils.join(rows, ",");
 
-        String key = new String(rows.get(0).key.key.array(),rows.get(0).key.key.position(),rows.get(0).key.key.remaining());
+        String key = new String(rows.get(0).key.getKey().array(), rows.get(0).key.getKey().position(), rows.get(0).key.getKey().remaining());
         assert "k1".equals( key ) : key;
 
-        key = new String(rows.get(1).key.key.array(),rows.get(1).key.key.position(),rows.get(1).key.key.remaining());
+        key = new String(rows.get(1).key.getKey().array(), rows.get(1).key.getKey().position(), rows.get(1).key.getKey().remaining());
         assert "k3".equals(key) : key;
 
         assert ByteBufferUtil.bytes(1L).equals( rows.get(0).cf.getColumn(birthdate).value());
@@ -258,14 +258,14 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         rows = cfs.search(range, clause, filter, 100);
 
         assert rows.size() == 1 : StringUtils.join(rows, ",");
-        key = new String(rows.get(0).key.key.array(),rows.get(0).key.key.position(),rows.get(0).key.key.remaining());
+        key = new String(rows.get(0).key.getKey().array(), rows.get(0).key.getKey().position(), rows.get(0).key.getKey().remaining());
         assert "k3".equals( key );
 
         // same query again, but with resultset not including the subordinate expression
         rows = cfs.search(range, clause, Util.namesFilter(cfs, "birthdate"), 100);
 
         assert rows.size() == 1 : StringUtils.join(rows, ",");
-        key = new String(rows.get(0).key.key.array(),rows.get(0).key.key.position(),rows.get(0).key.key.remaining());
+        key = new String(rows.get(0).key.getKey().array(), rows.get(0).key.getKey().position(), rows.get(0).key.getKey().remaining());
         assert "k3".equals( key );
 
         assert rows.get(0).cf.getColumnCount() == 1 : rows.get(0).cf;
@@ -275,7 +275,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         rows = cfs.search(range, clause, emptyFilter, 100);
 
         assert rows.size() == 1 : StringUtils.join(rows, ",");
-        key = new String(rows.get(0).key.key.array(),rows.get(0).key.key.position(),rows.get(0).key.key.remaining());
+        key = new String(rows.get(0).key.getKey().array(), rows.get(0).key.getKey().position(), rows.get(0).key.getKey().remaining());
         assert "k3".equals( key );
 
         assertFalse(rows.get(0).cf.hasColumns());
@@ -334,7 +334,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         Range<RowPosition> range = Util.range("", "");
         List<Row> rows = cfs.search(range, clause, filter, 100);
         assert rows.size() == 1 : StringUtils.join(rows, ",");
-        String key = ByteBufferUtil.string(rows.get(0).key.key);
+        String key = ByteBufferUtil.string(rows.get(0).key.getKey());
         assert "k1".equals( key );
 
         // delete the column directly
@@ -358,7 +358,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         rm.apply();
         rows = cfs.search(range, clause, filter, 100);
         assert rows.size() == 1 : StringUtils.join(rows, ",");
-        key = ByteBufferUtil.string(rows.get(0).key.key);
+        key = ByteBufferUtil.string(rows.get(0).key.getKey());
         assert "k1".equals( key );
 
         // verify that row and delete w/ older timestamp does nothing
@@ -367,7 +367,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         rm.apply();
         rows = cfs.search(range, clause, filter, 100);
         assert rows.size() == 1 : StringUtils.join(rows, ",");
-        key = ByteBufferUtil.string(rows.get(0).key.key);
+        key = ByteBufferUtil.string(rows.get(0).key.getKey());
         assert "k1".equals( key );
 
         // similarly, column delete w/ older timestamp should do nothing
@@ -376,7 +376,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         rm.apply();
         rows = cfs.search(range, clause, filter, 100);
         assert rows.size() == 1 : StringUtils.join(rows, ",");
-        key = ByteBufferUtil.string(rows.get(0).key.key);
+        key = ByteBufferUtil.string(rows.get(0).key.getKey());
         assert "k1".equals( key );
 
         // delete the entire row (w/ newer timestamp this time)
@@ -408,7 +408,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         rm.apply();
         rows = cfs.search(range, clause, filter, 100);
         assert rows.size() == 1 : StringUtils.join(rows, ",");
-        key = ByteBufferUtil.string(rows.get(0).key.key);
+        key = ByteBufferUtil.string(rows.get(0).key.getKey());
         assert "k1".equals( key );
     }
 
@@ -438,7 +438,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         expr = new IndexExpression(ByteBufferUtil.bytes("birthdate"), IndexExpression.Operator.EQ, ByteBufferUtil.bytes(2L));
         clause = Arrays.asList(expr);
         rows = keyspace.getColumnFamilyStore("Indexed1").search(range, clause, filter, 100);
-        String key = ByteBufferUtil.string(rows.get(0).key.key);
+        String key = ByteBufferUtil.string(rows.get(0).key.getKey());
         assert "k1".equals( key );
 
         // update the birthdate value with an OLDER timestamp, and test that the index ignores this
@@ -447,7 +447,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         rm.apply();
 
         rows = keyspace.getColumnFamilyStore("Indexed1").search(range, clause, filter, 100);
-        key = ByteBufferUtil.string(rows.get(0).key.key);
+        key = ByteBufferUtil.string(rows.get(0).key.getKey());
         assert "k1".equals( key );
 
     }
@@ -714,7 +714,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         IDiskAtomFilter filter = new IdentityQueryFilter();
         List<Row> rows = keyspace.getColumnFamilyStore("Indexed2").search(Util.range("", ""), clause, filter, 100);
         assert rows.size() == 1 : StringUtils.join(rows, ",");
-        assertEquals("k1", ByteBufferUtil.string(rows.get(0).key.key));
+        assertEquals("k1", ByteBufferUtil.string(rows.get(0).key.getKey()));
     }
 
     @Test
@@ -771,7 +771,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
                                              Util.namesFilter(cfs, "asdf"),
                                              10);
         assertEquals(2, result.size());
-        assert result.get(0).key.key.equals(ByteBufferUtil.bytes("key1"));
+        assert result.get(0).key.getKey().equals(ByteBufferUtil.bytes("key1"));
     }
 
     @Test
@@ -786,16 +786,16 @@ public class ColumnFamilyStoreTest extends SchemaLoader
 
         // create an isolated sstable.
         putColsSuper(cfs, key, scfName,
-                new Cell(cellname(1L), ByteBufferUtil.bytes("val1"), 1),
-                new Cell(cellname(2L), ByteBufferUtil.bytes("val2"), 1),
-                new Cell(cellname(3L), ByteBufferUtil.bytes("val3"), 1));
+                new BufferCell(cellname(1L), ByteBufferUtil.bytes("val1"), 1),
+                new BufferCell(cellname(2L), ByteBufferUtil.bytes("val2"), 1),
+                new BufferCell(cellname(3L), ByteBufferUtil.bytes("val3"), 1));
         cfs.forceBlockingFlush();
 
         // insert, don't flush.
         putColsSuper(cfs, key, scfName,
-                new Cell(cellname(4L), ByteBufferUtil.bytes("val4"), 1),
-                new Cell(cellname(5L), ByteBufferUtil.bytes("val5"), 1),
-                new Cell(cellname(6L), ByteBufferUtil.bytes("val6"), 1));
+                new BufferCell(cellname(4L), ByteBufferUtil.bytes("val4"), 1),
+                new BufferCell(cellname(5L), ByteBufferUtil.bytes("val5"), 1),
+                new BufferCell(cellname(6L), ByteBufferUtil.bytes("val6"), 1));
 
         // verify insert.
         final SlicePredicate sp = new SlicePredicate();
@@ -807,7 +807,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         assertRowAndColCount(1, 6, false, cfs.getRangeSlice(Util.range("f", "g"), null, ThriftValidation.asIFilter(sp, cfs.metadata, scfName), 100));
 
         // delete
-        Mutation rm = new Mutation(keyspace.getName(), key.key);
+        Mutation rm = new Mutation(keyspace.getName(), key.getKey());
         rm.deleteRange(cfName, SuperColumns.startOf(scfName), SuperColumns.endOf(scfName), 2);
         rm.apply();
 
@@ -822,17 +822,17 @@ public class ColumnFamilyStoreTest extends SchemaLoader
 
         // late insert.
         putColsSuper(cfs, key, scfName,
-                new Cell(cellname(4L), ByteBufferUtil.bytes("val4"), 1L),
-                new Cell(cellname(7L), ByteBufferUtil.bytes("val7"), 1L));
+                new BufferCell(cellname(4L), ByteBufferUtil.bytes("val4"), 1L),
+                new BufferCell(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 Cell(cellname(3L), ByteBufferUtil.bytes("val3"), 3),
-                new Cell(cellname(8L), ByteBufferUtil.bytes("val8"), 3),
-                new Cell(cellname(9L), ByteBufferUtil.bytes("val9"), 3));
+                new BufferCell(cellname(3L), ByteBufferUtil.bytes("val3"), 3),
+                new BufferCell(cellname(8L), ByteBufferUtil.bytes("val8"), 3),
+                new BufferCell(cellname(9L), ByteBufferUtil.bytes("val9"), 3));
         assertRowAndColCount(1, 3, false, cfs.getRangeSlice(Util.range("f", "g"), null, ThriftValidation.asIFilter(sp, cfs.metadata, scfName), 100));
     }
 
@@ -861,7 +861,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         ColumnFamily cf = ArrayBackedSortedColumns.factory.create(cfs.keyspace.getName(), cfs.name);
         for (Cell col : cols)
             cf.addColumn(col.withUpdatedName(CellNames.compositeDense(scfName, col.name().toByteBuffer())));
-        Mutation rm = new Mutation(cfs.keyspace.getName(), key.key, cf);
+        Mutation rm = new Mutation(cfs.keyspace.getName(), key.getKey(), cf);
         rm.apply();
     }
 
@@ -870,7 +870,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         ColumnFamily cf = ArrayBackedSortedColumns.factory.create(cfs.keyspace.getName(), cfs.name);
         for (Cell col : cols)
             cf.addColumn(col);
-        Mutation rm = new Mutation(cfs.keyspace.getName(), key.key, cf);
+        Mutation rm = new Mutation(cfs.keyspace.getName(), key.getKey(), cf);
         rm.apply();
     }
 
@@ -902,7 +902,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         assertRowAndColCount(1, 4, false, cfs.getRangeSlice(Util.range("f", "g"), null, ThriftValidation.asIFilter(sp, cfs.metadata, null), 100));
 
         // delete (from sstable and memtable)
-        Mutation rm = new Mutation(keyspace.getName(), key.key);
+        Mutation rm = new Mutation(keyspace.getName(), key.getKey());
         rm.delete(cfs.name, 2);
         rm.apply();
 
@@ -973,8 +973,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 Cell(cellname("a"), ByteBufferUtil.bytes("A"), 1),
-                                             new Cell(cellname("b"), ByteBufferUtil.bytes("B"), 1));
+        putColsSuper(cfs, key, superColName, new BufferCell(cellname("a"), ByteBufferUtil.bytes("A"), 1),
+                                             new BufferCell(cellname("b"), ByteBufferUtil.bytes("B"), 1));
 
         // Get the entire supercolumn like normal
         ColumnFamily cfGet = cfs.getColumnFamily(QueryFilter.getIdentityFilter(key, cfName, System.currentTimeMillis()));
@@ -985,7 +985,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         SortedSet<CellName> sliceColNames = new TreeSet<CellName>(cfs.metadata.comparator);
         sliceColNames.add(CellNames.compositeDense(superColName, ByteBufferUtil.bytes("a")));
         sliceColNames.add(CellNames.compositeDense(superColName, ByteBufferUtil.bytes("b")));
-        SliceByNamesReadCommand cmd = new SliceByNamesReadCommand(keyspaceName, key.key, cfName, System.currentTimeMillis(), new NamesQueryFilter(sliceColNames));
+        SliceByNamesReadCommand cmd = new SliceByNamesReadCommand(keyspaceName, key.getKey(), cfName, System.currentTimeMillis(), new NamesQueryFilter(sliceColNames));
         ColumnFamily cfSliced = cmd.getRow(keyspace).cf;
 
         // Make sure the slice returns the same as the straight get
@@ -1005,7 +1005,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         cfs.clearUnsafe();
 
         // Create a cell a 'high timestamp'
-        putColsStandard(cfs, key, new Cell(cname, ByteBufferUtil.bytes("a"), 2));
+        putColsStandard(cfs, key, new BufferCell(cname, ByteBufferUtil.bytes("a"), 2));
         cfs.forceBlockingFlush();
 
         // Nuke the metadata and reload that sstable
@@ -1018,10 +1018,10 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         cfs.loadNewSSTables();
 
         // Add another cell with a lower timestamp
-        putColsStandard(cfs, key, new Cell(cname, ByteBufferUtil.bytes("b"), 1));
+        putColsStandard(cfs, key, new BufferCell(cname, ByteBufferUtil.bytes("b"), 1));
 
         // 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())));
+        SliceByNamesReadCommand cmd = new SliceByNamesReadCommand(keyspaceName, key.getKey(), cfName, System.currentTimeMillis(), new NamesQueryFilter(FBUtilities.singleton(cname, cfs.getComparator())));
         ColumnFamily cf = cmd.getRow(keyspace).cf;
         Cell cell = cf.getColumn(cname);
         assert cell.value().equals(ByteBufferUtil.bytes("a")) : "expecting a, got " + ByteBufferUtil.string(cell.value());
@@ -1250,7 +1250,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
             for (Row row : rows)
             {
                 sb.append("{");
-                sb.append(ByteBufferUtil.string(row.key.key));
+                sb.append(ByteBufferUtil.string(row.key.getKey()));
                 sb.append(":");
                 if (row.cf != null && !row.cf.isEmpty())
                 {
@@ -1399,7 +1399,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         Cell[] cols = new Cell[letters.length];
         for (int i = 0; i < cols.length; i++)
         {
-            cols[i] = new Cell(cellname("col" + letters[i].toUpperCase()),
+            cols[i] = new BufferCell(cellname("col" + letters[i].toUpperCase()),
                     ByteBuffer.wrap(new byte[1]), 1);
         }
 
@@ -1448,7 +1448,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         Cell[] cols = new Cell[letters.length];
         for (int i = 0; i < cols.length; i++)
         {
-            cols[i] = new Cell(cellname("col" + letters[i].toUpperCase()),
+            cols[i] = new BufferCell(cellname("col" + letters[i].toUpperCase()),
                     ByteBuffer.wrap(new byte[1366]), 1);
         }
 
@@ -1497,7 +1497,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         Cell[] cols = new Cell[letters.length];
         for (int i = 0; i < cols.length; i++)
         {
-            cols[i] = new Cell(cellname("col" + letters[i].toUpperCase()),
+            cols[i] = new BufferCell(cellname("col" + letters[i].toUpperCase()),
                     ByteBuffer.wrap(new byte[1]), 1);
         }
 
@@ -1547,7 +1547,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         Cell[] cols = new Cell[letters.length];
         for (int i = 0; i < cols.length; i++)
         {
-            cols[i] = new Cell(cellname("col" + letters[i].toUpperCase()),
+            cols[i] = new BufferCell(cellname("col" + letters[i].toUpperCase()),
                     ByteBuffer.wrap(new byte[1366]), 1);
         }
 
@@ -1595,7 +1595,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         Cell[] cols = new Cell[letters.length];
         for (int i = 0; i < cols.length; i++)
         {
-            cols[i] = new Cell(cellname("col" + letters[i].toUpperCase()),
+            cols[i] = new BufferCell(cellname("col" + letters[i].toUpperCase()),
                     // use 1366 so that three cols make an index segment
                     ByteBuffer.wrap(new byte[1366]), 1);
         }
@@ -1856,7 +1856,7 @@ public class ColumnFamilyStoreTest extends SchemaLoader
         Cell[] cols = new Cell[12];
         for (int i = 0; i < cols.length; i++)
         {
-            cols[i] = new Cell(cellname("col" + letters[i]), ByteBuffer.wrap(new byte[valueSize]), 1);
+            cols[i] = new BufferCell(cellname("col" + letters[i]), ByteBuffer.wrap(new byte[valueSize]), 1);
         }
 
         for (int i = 0; i < 12; i++)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/test/unit/org/apache/cassandra/db/CounterCacheTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/CounterCacheTest.java b/test/unit/org/apache/cassandra/db/CounterCacheTest.java
index a015a43..cb2d97a 100644
--- a/test/unit/org/apache/cassandra/db/CounterCacheTest.java
+++ b/test/unit/org/apache/cassandra/db/CounterCacheTest.java
@@ -75,8 +75,8 @@ public class CounterCacheTest extends SchemaLoader
         CacheService.instance.invalidateCounterCache();
 
         ColumnFamily cells = ArrayBackedSortedColumns.factory.create(cfs.metadata);
-        cells.addColumn(new CounterUpdateCell(cellname(1), 1L, FBUtilities.timestampMicros()));
-        cells.addColumn(new CounterUpdateCell(cellname(2), 2L, FBUtilities.timestampMicros()));
+        cells.addColumn(new BufferCounterUpdateCell(cellname(1), 1L, FBUtilities.timestampMicros()));
+        cells.addColumn(new BufferCounterUpdateCell(cellname(2), 2L, FBUtilities.timestampMicros()));
         new CounterMutation(new Mutation(KS, bytes(1), cells), ConsistencyLevel.ONE).apply();
         new CounterMutation(new Mutation(KS, bytes(2), cells), ConsistencyLevel.ONE).apply();
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/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
index 86b856c..efc365d 100644
--- a/test/unit/org/apache/cassandra/db/CounterCellTest.java
+++ b/test/unit/org/apache/cassandra/db/CounterCellTest.java
@@ -64,7 +64,7 @@ public class CounterCellTest extends SchemaLoader
     public void testCreate()
     {
         long delta = 3L;
-        CounterCell cell = new CounterCell(Util.cellname("x"),
+        CounterCell cell = new BufferCounterCell(Util.cellname("x"),
                                            CounterContext.instance().createLocal(delta),
                                            1L,
                                            Long.MIN_VALUE);
@@ -87,33 +87,33 @@ public class CounterCellTest extends SchemaLoader
         ByteBuffer context;
 
         // tombstone + tombstone
-        left  = new DeletedCell(cellname("x"), 1, 1L);
-        right = new DeletedCell(cellname("x"), 2, 2L);
+        left  = new BufferDeletedCell(cellname("x"), 1, 1L);
+        right = new BufferDeletedCell(cellname("x"), 2, 2L);
 
         assert left.reconcile(right).getMarkedForDeleteAt() == right.getMarkedForDeleteAt();
         assert right.reconcile(left).getMarkedForDeleteAt() == right.getMarkedForDeleteAt();
 
         // tombstone > live
-        left  = new DeletedCell(cellname("x"), 1, 2L);
-        right = CounterCell.createLocal(cellname("x"), 0L, 1L, Long.MIN_VALUE);
+        left  = new BufferDeletedCell(cellname("x"), 1, 2L);
+        right = BufferCounterCell.createLocal(cellname("x"), 0L, 1L, Long.MIN_VALUE);
 
         assert left.reconcile(right) == left;
 
         // tombstone < live last delete
-        left  = new DeletedCell(cellname("x"), 1, 1L);
-        right = CounterCell.createLocal(cellname("x"), 0L, 4L, 2L);
+        left  = new BufferDeletedCell(cellname("x"), 1, 1L);
+        right = BufferCounterCell.createLocal(cellname("x"), 0L, 4L, 2L);
 
         assert left.reconcile(right) == right;
 
         // tombstone == live last delete
-        left  = new DeletedCell(cellname("x"), 1, 2L);
-        right = CounterCell.createLocal(cellname("x"), 0L, 4L, 2L);
+        left  = new BufferDeletedCell(cellname("x"), 1, 2L);
+        right = BufferCounterCell.createLocal(cellname("x"), 0L, 4L, 2L);
 
         assert left.reconcile(right) == right;
 
         // tombstone > live last delete
-        left  = new DeletedCell(cellname("x"), 1, 4L);
-        right = CounterCell.createLocal(cellname("x"), 0L, 9L, 1L);
+        left  = new BufferDeletedCell(cellname("x"), 1, 4L);
+        right = BufferCounterCell.createLocal(cellname("x"), 0L, 9L, 1L);
 
         reconciled = left.reconcile(right);
         assert reconciled.name() == right.name();
@@ -122,26 +122,26 @@ public class CounterCellTest extends SchemaLoader
         assert ((CounterCell)reconciled).timestampOfLastDelete() == left.getMarkedForDeleteAt();
 
         // live < tombstone
-        left  = CounterCell.createLocal(cellname("x"), 0L, 1L, Long.MIN_VALUE);
-        right = new DeletedCell(cellname("x"), 1, 2L);
+        left  = BufferCounterCell.createLocal(cellname("x"), 0L, 1L, Long.MIN_VALUE);
+        right = new BufferDeletedCell(cellname("x"), 1, 2L);
 
         assert left.reconcile(right) == right;
 
         // live last delete > tombstone
-        left  = CounterCell.createLocal(cellname("x"), 0L, 4L, 2L);
-        right = new DeletedCell(cellname("x"), 1, 1L);
+        left  = BufferCounterCell.createLocal(cellname("x"), 0L, 4L, 2L);
+        right = new BufferDeletedCell(cellname("x"), 1, 1L);
 
         assert left.reconcile(right) == left;
 
         // live last delete == tombstone
-        left  = CounterCell.createLocal(cellname("x"), 0L, 4L, 2L);
-        right = new DeletedCell(cellname("x"), 1, 2L);
+        left  = BufferCounterCell.createLocal(cellname("x"), 0L, 4L, 2L);
+        right = new BufferDeletedCell(cellname("x"), 1, 2L);
 
         assert left.reconcile(right) == left;
 
         // live last delete < tombstone
-        left  = CounterCell.createLocal(cellname("x"), 0L, 9L, 1L);
-        right = new DeletedCell(cellname("x"), 1, 4L);
+        left  = BufferCounterCell.createLocal(cellname("x"), 0L, 9L, 1L);
+        right = new BufferDeletedCell(cellname("x"), 1, 4L);
 
         reconciled = left.reconcile(right);
         assert reconciled.name() == left.name();
@@ -150,20 +150,20 @@ public class CounterCellTest extends SchemaLoader
         assert ((CounterCell)reconciled).timestampOfLastDelete() == right.getMarkedForDeleteAt();
 
         // live < live last delete
-        left  = new CounterCell(cellname("x"), cc.createRemote(CounterId.fromInt(1), 2L, 3L), 1L, Long.MIN_VALUE);
-        right = new CounterCell(cellname("x"), cc.createRemote(CounterId.fromInt(1), 1L, 1L), 4L, 3L);
+        left  = new BufferCounterCell(cellname("x"), cc.createRemote(CounterId.fromInt(1), 2L, 3L), 1L, Long.MIN_VALUE);
+        right = new BufferCounterCell(cellname("x"), cc.createRemote(CounterId.fromInt(1), 1L, 1L), 4L, 3L);
 
         assert left.reconcile(right) == right;
 
         // live last delete > live
-        left  = new CounterCell(cellname("x"), cc.createRemote(CounterId.fromInt(1), 2L, 3L), 6L, 5L);
-        right = new CounterCell(cellname("x"), cc.createRemote(CounterId.fromInt(1), 1L, 1L), 4L, 3L);
+        left  = new BufferCounterCell(cellname("x"), cc.createRemote(CounterId.fromInt(1), 2L, 3L), 6L, 5L);
+        right = new BufferCounterCell(cellname("x"), cc.createRemote(CounterId.fromInt(1), 1L, 1L), 4L, 3L);
 
         assert left.reconcile(right) == left;
 
         // live + live
-        left = new CounterCell(cellname("x"), cc.createRemote(CounterId.fromInt(1), 1L, 1L), 4L, Long.MIN_VALUE);
-        right = new CounterCell(cellname("x"), cc.createRemote(CounterId.fromInt(1), 2L, 3L), 1L, Long.MIN_VALUE);
+        left = new BufferCounterCell(cellname("x"), cc.createRemote(CounterId.fromInt(1), 1L, 1L), 4L, Long.MIN_VALUE);
+        right = new BufferCounterCell(cellname("x"), cc.createRemote(CounterId.fromInt(1), 2L, 3L), 1L, Long.MIN_VALUE);
 
         reconciled = left.reconcile(right);
         assert reconciled.name().equals(left.name());
@@ -171,7 +171,7 @@ public class CounterCellTest extends SchemaLoader
         assert reconciled.timestamp() == 4L;
 
         left = reconciled;
-        right = new CounterCell(cellname("x"), cc.createRemote(CounterId.fromInt(2), 1L, 5L), 2L, Long.MIN_VALUE);
+        right = new BufferCounterCell(cellname("x"), cc.createRemote(CounterId.fromInt(2), 1L, 5L), 2L, Long.MIN_VALUE);
 
         reconciled = left.reconcile(right);
         assert reconciled.name().equals(left.name());
@@ -179,7 +179,7 @@ public class CounterCellTest extends SchemaLoader
         assert reconciled.timestamp() == 4L;
 
         left = reconciled;
-        right = new CounterCell(cellname("x"), cc.createRemote(CounterId.fromInt(2), 2L, 2L), 6L, Long.MIN_VALUE);
+        right = new BufferCounterCell(cellname("x"), cc.createRemote(CounterId.fromInt(2), 2L, 2L), 6L, Long.MIN_VALUE);
 
         reconciled = left.reconcile(right);
         assert reconciled.name().equals(left.name());
@@ -211,15 +211,15 @@ public class CounterCellTest extends SchemaLoader
         CounterCell rightCell;
 
         // timestamp
-        leftCell = CounterCell.createLocal(cellname("x"), 0, 1L, Long.MIN_VALUE);
-        rightCell = CounterCell.createLocal(cellname("x"), 0, 2L, Long.MIN_VALUE);
+        leftCell = BufferCounterCell.createLocal(cellname("x"), 0, 1L, Long.MIN_VALUE);
+        rightCell = BufferCounterCell.createLocal(cellname("x"), 0, 2L, Long.MIN_VALUE);
 
         assert rightCell == leftCell.diff(rightCell);
         assert null      == rightCell.diff(leftCell);
 
         // timestampOfLastDelete
-        leftCell = CounterCell.createLocal(cellname("x"), 0, 1L, 1L);
-        rightCell = CounterCell.createLocal(cellname("x"), 0, 1L, 2L);
+        leftCell = BufferCounterCell.createLocal(cellname("x"), 0, 1L, 1L);
+        rightCell = BufferCounterCell.createLocal(cellname("x"), 0, 1L, 2L);
 
         assert rightCell == leftCell.diff(rightCell);
         assert null      == rightCell.diff(leftCell);
@@ -231,8 +231,8 @@ public class CounterCellTest extends SchemaLoader
         left.writeRemote(CounterId.fromInt(9), 1L, 0L);
         right = ContextState.wrap(ByteBufferUtil.clone(left.context));
 
-        leftCell  = new CounterCell(cellname("x"), left.context,  1L);
-        rightCell = new CounterCell(cellname("x"), right.context, 1L);
+        leftCell  = new BufferCounterCell(cellname("x"), left.context,  1L);
+        rightCell = new BufferCounterCell(cellname("x"), right.context, 1L);
         assert leftCell.diff(rightCell) == null;
 
         // greater than: left has superset of nodes (counts equal)
@@ -247,8 +247,8 @@ public class CounterCellTest extends SchemaLoader
         right.writeRemote(CounterId.fromInt(6), 2L, 0L);
         right.writeRemote(CounterId.fromInt(9), 1L, 0L);
 
-        leftCell  = new CounterCell(cellname("x"), left.context,  1L);
-        rightCell = new CounterCell(cellname("x"), right.context, 1L);
+        leftCell  = new BufferCounterCell(cellname("x"), left.context,  1L);
+        rightCell = new BufferCounterCell(cellname("x"), right.context, 1L);
         assert leftCell.diff(rightCell) == null;
 
         // less than: right has subset of nodes (counts equal)
@@ -265,8 +265,8 @@ public class CounterCellTest extends SchemaLoader
         right.writeRemote(CounterId.fromInt(6), 1L, 0L);
         right.writeRemote(CounterId.fromInt(9), 1L, 0L);
 
-        leftCell  = new CounterCell(cellname("x"), left.context,  1L);
-        rightCell = new CounterCell(cellname("x"), right.context, 1L);
+        leftCell  = new BufferCounterCell(cellname("x"), left.context,  1L);
+        rightCell = new BufferCounterCell(cellname("x"), right.context, 1L);
         assert rightCell == leftCell.diff(rightCell);
         assert leftCell  == rightCell.diff(leftCell);
     }
@@ -281,7 +281,7 @@ public class CounterCellTest extends SchemaLoader
         state.writeLocal(CounterId.fromInt(4), 4L, 4L);
 
         CellNameType type = new SimpleDenseCellNameType(UTF8Type.instance);
-        CounterCell original = new CounterCell(cellname("x"), state.context, 1L);
+        CounterCell original = new BufferCounterCell(cellname("x"), state.context, 1L);
         byte[] serialized;
         try (DataOutputBuffer bufOut = new DataOutputBuffer())
         {
@@ -315,8 +315,8 @@ public class CounterCellTest extends SchemaLoader
         state.writeRemote(CounterId.fromInt(3), 4L, 4L);
         state.writeLocal(CounterId.fromInt(4), 4L, 4L);
 
-        CounterCell original = new CounterCell(cellname("x"), state.context, 1L);
-        CounterCell cleared = new CounterCell(cellname("x"), cc.clearAllLocal(state.context), 1L);
+        CounterCell original = new BufferCounterCell(cellname("x"), state.context, 1L);
+        CounterCell cleared = new BufferCounterCell(cellname("x"), cc.clearAllLocal(state.context), 1L);
 
         original.updateDigest(digest1);
         cleared.updateDigest(digest2);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/test/unit/org/apache/cassandra/db/KeyCacheTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/KeyCacheTest.java b/test/unit/org/apache/cassandra/db/KeyCacheTest.java
index ec99fd1..e6745a1 100644
--- a/test/unit/org/apache/cassandra/db/KeyCacheTest.java
+++ b/test/unit/org/apache/cassandra/db/KeyCacheTest.java
@@ -120,10 +120,10 @@ public class KeyCacheTest extends SchemaLoader
         Mutation rm;
 
         // inserts
-        rm = new Mutation(KEYSPACE1, key1.key);
+        rm = new Mutation(KEYSPACE1, key1.getKey());
         rm.add(COLUMN_FAMILY1, Util.cellname("1"), ByteBufferUtil.EMPTY_BYTE_BUFFER, 0);
         rm.apply();
-        rm = new Mutation(KEYSPACE1, key2.key);
+        rm = new Mutation(KEYSPACE1, key2.getKey());
         rm.add(COLUMN_FAMILY1, Util.cellname("2"), ByteBufferUtil.EMPTY_BYTE_BUFFER, 0);
         rm.apply();
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/8541cca7/test/unit/org/apache/cassandra/db/KeyCollisionTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/KeyCollisionTest.java b/test/unit/org/apache/cassandra/db/KeyCollisionTest.java
index 57f077f..1869872 100644
--- a/test/unit/org/apache/cassandra/db/KeyCollisionTest.java
+++ b/test/unit/org/apache/cassandra/db/KeyCollisionTest.java
@@ -73,10 +73,10 @@ public class KeyCollisionTest extends SchemaLoader
 
         List<Row> rows = cfs.getRangeSlice(new Bounds<RowPosition>(dk("k2"), dk("key2")), null, new IdentityQueryFilter(), 10000);
         assert rows.size() == 4 : "Expecting 4 keys, got " + rows.size();
-        assert rows.get(0).key.key.equals(ByteBufferUtil.bytes("k2"));
-        assert rows.get(1).key.key.equals(ByteBufferUtil.bytes("k3"));
-        assert rows.get(2).key.key.equals(ByteBufferUtil.bytes("key1"));
-        assert rows.get(3).key.key.equals(ByteBufferUtil.bytes("key2"));
+        assert rows.get(0).key.getKey().equals(ByteBufferUtil.bytes("k2"));
+        assert rows.get(1).key.getKey().equals(ByteBufferUtil.bytes("k3"));
+        assert rows.get(2).key.getKey().equals(ByteBufferUtil.bytes("key1"));
+        assert rows.get(3).key.getKey().equals(ByteBufferUtil.bytes("key2"));
     }
 
     private void insert(String... keys)
@@ -102,7 +102,7 @@ public class KeyCollisionTest extends SchemaLoader
 
         public DecoratedKey decorateKey(ByteBuffer key)
         {
-            return new DecoratedKey(getToken(key), key);
+            return new BufferDecoratedKey(getToken(key), key);
         }
 
         public Token midpoint(Token ltoken, Token rtoken)