You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by jb...@apache.org on 2014/01/31 00:09:26 UTC

[1/6] remove Table.switchlock and introduce o.a.c.utils.memory package patch by Benedict Elliott Smith; reviewed by jbellis for CASSANDRA-5549

Updated Branches:
  refs/heads/trunk 00fc31868 -> 4b54b8acd


http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/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 185d3b3..cd837c8 100644
--- a/test/unit/org/apache/cassandra/db/ArrayBackedSortedColumnsTest.java
+++ b/test/unit/org/apache/cassandra/db/ArrayBackedSortedColumnsTest.java
@@ -35,8 +35,8 @@ import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.db.composites.*;
 import org.apache.cassandra.db.filter.ColumnSlice;
+import org.apache.cassandra.utils.memory.HeapAllocator;
 import org.apache.cassandra.db.marshal.Int32Type;
-import org.apache.cassandra.utils.HeapAllocator;
 
 public class ArrayBackedSortedColumnsTest extends SchemaLoader
 {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/test/unit/org/apache/cassandra/db/ColumnFamilyTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/ColumnFamilyTest.java b/test/unit/org/apache/cassandra/db/ColumnFamilyTest.java
index 42f6066..cd79217 100644
--- a/test/unit/org/apache/cassandra/db/ColumnFamilyTest.java
+++ b/test/unit/org/apache/cassandra/db/ColumnFamilyTest.java
@@ -37,7 +37,7 @@ import static org.apache.cassandra.Util.cellname;
 import static org.junit.Assert.assertEquals;
 
 import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.HeapAllocator;
+import org.apache.cassandra.utils.memory.HeapAllocator;
 
 
 public class ColumnFamilyTest extends SchemaLoader

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/test/unit/org/apache/cassandra/db/CommitLogTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/CommitLogTest.java b/test/unit/org/apache/cassandra/db/CommitLogTest.java
index b51c17e..577692d 100644
--- a/test/unit/org/apache/cassandra/db/CommitLogTest.java
+++ b/test/unit/org/apache/cassandra/db/CommitLogTest.java
@@ -117,7 +117,7 @@ public class CommitLogTest extends SchemaLoader
         assert CommitLog.instance.activeSegments() == 2 : "Expecting 2 segments, got " + CommitLog.instance.activeSegments();
 
         UUID cfid2 = rm2.getColumnFamilyIds().iterator().next();
-        CommitLog.instance.discardCompletedSegments(cfid2, CommitLog.instance.getContext().get());
+        CommitLog.instance.discardCompletedSegments(cfid2, CommitLog.instance.getContext());
 
         // Assert we still have both our segment
         assert CommitLog.instance.activeSegments() == 2 : "Expecting 2 segments, got " + CommitLog.instance.activeSegments();
@@ -141,7 +141,7 @@ public class CommitLogTest extends SchemaLoader
         // "Flush": this won't delete anything
         UUID cfid1 = rm.getColumnFamilyIds().iterator().next();
         CommitLog.instance.sync(true);
-        CommitLog.instance.discardCompletedSegments(cfid1, CommitLog.instance.getContext().get());
+        CommitLog.instance.discardCompletedSegments(cfid1, CommitLog.instance.getContext());
 
         assert CommitLog.instance.activeSegments() == 1 : "Expecting 1 segment, got " + CommitLog.instance.activeSegments();
 
@@ -160,7 +160,7 @@ public class CommitLogTest extends SchemaLoader
         // didn't write anything on cf1 since last flush (and we flush cf2)
 
         UUID cfid2 = rm2.getColumnFamilyIds().iterator().next();
-        CommitLog.instance.discardCompletedSegments(cfid2, CommitLog.instance.getContext().get());
+        CommitLog.instance.discardCompletedSegments(cfid2, CommitLog.instance.getContext());
 
         // Assert we still have both our segment
         assert CommitLog.instance.activeSegments() == 1 : "Expecting 1 segment, got " + CommitLog.instance.activeSegments();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/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 24d8f1c..6949340 100644
--- a/test/unit/org/apache/cassandra/db/CounterCellTest.java
+++ b/test/unit/org/apache/cassandra/db/CounterCellTest.java
@@ -27,6 +27,8 @@ import java.net.UnknownHostException;
 import java.nio.ByteBuffer;
 import java.util.Arrays;
 
+import org.apache.cassandra.utils.memory.AbstractAllocator;
+import org.apache.cassandra.utils.memory.HeapAllocator;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -87,7 +89,7 @@ public class CounterCellTest extends SchemaLoader
 
         ByteBuffer context;
 
-        Allocator allocator = HeapAllocator.instance;
+        AbstractAllocator allocator = HeapAllocator.instance;
 
         // tombstone + tombstone
         left  = new DeletedCell(cellname("x"), 1, 1L);
@@ -207,7 +209,7 @@ public class CounterCellTest extends SchemaLoader
     @Test
     public void testDiff() throws UnknownHostException
     {
-        Allocator allocator = HeapAllocator.instance;
+        AbstractAllocator allocator = HeapAllocator.instance;
         ContextState left;
         ContextState right;
 
@@ -278,7 +280,7 @@ public class CounterCellTest extends SchemaLoader
     @Test
     public void testSerializeDeserialize() throws IOException
     {
-        Allocator allocator = HeapAllocator.instance;
+        AbstractAllocator allocator = HeapAllocator.instance;
         CounterContext.ContextState state = CounterContext.ContextState.allocate(0, 2, 2, allocator);
         state.writeRemote(CounterId.fromInt(1), 4L, 4L);
         state.writeLocal(CounterId.fromInt(2), 4L, 4L);
@@ -311,7 +313,7 @@ public class CounterCellTest extends SchemaLoader
     @Test
     public void testUpdateDigest() throws Exception
     {
-        Allocator allocator = HeapAllocator.instance;
+        AbstractAllocator allocator = HeapAllocator.instance;
         MessageDigest digest1 = MessageDigest.getInstance("md5");
         MessageDigest digest2 = MessageDigest.getInstance("md5");
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/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 2c37f03..3a91c8c 100644
--- a/test/unit/org/apache/cassandra/db/KeyCollisionTest.java
+++ b/test/unit/org/apache/cassandra/db/KeyCollisionTest.java
@@ -167,6 +167,12 @@ public class KeyCollisionTest extends SchemaLoader
             return new BigIntegerToken(BigInteger.valueOf(key.remaining()));
         }
 
+        @Override
+        public long getHeapSizeOf(BigIntegerToken token)
+        {
+            return 0;
+        }
+
         public Map<Token, Float> describeOwnership(List<Token> sortedTokens)
         {
             // allTokens will contain the count and be returned, sorted_ranges is shorthand for token<->token math.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/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 c1e4e8e..74bc098 100644
--- a/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java
+++ b/test/unit/org/apache/cassandra/db/RangeTombstoneTest.java
@@ -20,6 +20,9 @@ package org.apache.cassandra.db;
 
 import java.nio.ByteBuffer;
 import java.util.*;
+
+import org.apache.cassandra.utils.concurrent.OpOrder;
+import org.apache.cassandra.utils.memory.AbstractAllocator;
 import org.junit.Test;
 
 import com.google.common.collect.ImmutableMap;
@@ -385,17 +388,17 @@ public class RangeTombstoneTest extends SchemaLoader
             deletes.clear();
         }
 
-        public void delete(ByteBuffer rowKey, Cell col)
+        public void delete(ByteBuffer rowKey, Cell col, OpOrder.Group opGroup)
         {
             deletes.add(col);
         }
 
-        public void insert(ByteBuffer rowKey, Cell col)
+        public void insert(ByteBuffer rowKey, Cell col, OpOrder.Group opGroup)
         {
             inserts.add(col);
         }
 
-        public void update(ByteBuffer rowKey, Cell col){}
+        public void update(ByteBuffer rowKey, Cell col, OpOrder.Group opGroup){}
 
         public void init(){}
 
@@ -409,7 +412,12 @@ public class RangeTombstoneTest extends SchemaLoader
 
         public void forceBlockingFlush(){}
 
-        public long getLiveSize(){ return 0; }
+        @Override
+        public AbstractAllocator getOnHeapAllocator()
+        {
+            return null;
+        }
+
 
         public ColumnFamilyStore getIndexCfs(){ return null; }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/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 3222d29..87ae58b 100644
--- a/test/unit/org/apache/cassandra/db/SecondaryIndexCellSizeTest.java
+++ b/test/unit/org/apache/cassandra/db/SecondaryIndexCellSizeTest.java
@@ -21,6 +21,8 @@ package org.apache.cassandra.db;
 import java.nio.ByteBuffer;
 import java.util.Set;
 
+import org.apache.cassandra.utils.concurrent.OpOrder;
+import org.apache.cassandra.utils.memory.AbstractAllocator;
 import org.junit.Test;
 
 import org.apache.cassandra.exceptions.ConfigurationException;
@@ -68,56 +70,46 @@ public class SecondaryIndexCellSizeTest
 
     private class MockRowIndex extends PerRowSecondaryIndex
     {
-        @Override
         public void init()
         {
         }
 
-        @Override
         public void validateOptions() throws ConfigurationException
         {
         }
 
-        @Override
         public String getIndexName()
         {
             return null;
         }
 
-        @Override
         protected SecondaryIndexSearcher createSecondaryIndexSearcher(Set<ByteBuffer> columns)
         {
             return null;
         }
 
-        @Override
         public void forceBlockingFlush()
         {
         }
 
-        @Override
-        public long getLiveSize()
+        public AbstractAllocator getOnHeapAllocator()
         {
-            return 0;
+            return null;
         }
 
-        @Override
         public ColumnFamilyStore getIndexCfs()
         {
             return null;
         }
 
-        @Override
         public void removeIndex(ByteBuffer columnName)
         {
         }
 
-        @Override
         public void invalidate()
         {
         }
 
-        @Override
         public void truncateBlocking(long truncatedAt)
         {
         }
@@ -126,15 +118,14 @@ public class SecondaryIndexCellSizeTest
         {
         }
 
-        public void index(ByteBuffer rowKey)
+        public void delete(DecoratedKey key, OpOrder.Group opGroup)
         {
         }
 
-        public void delete(DecoratedKey key)
+        public void index(ByteBuffer rowKey)
         {
         }
 
-        @Override
         public void reload()
         {
         }
@@ -181,9 +172,9 @@ public class SecondaryIndexCellSizeTest
         }
 
         @Override
-        public long getLiveSize()
+        public AbstractAllocator getOnHeapAllocator()
         {
-            return 0;
+            return null;
         }
 
         @Override
@@ -208,17 +199,17 @@ public class SecondaryIndexCellSizeTest
         }
 
         @Override
-        public void delete(ByteBuffer rowKey, Cell col)
+        public void delete(ByteBuffer rowKey, Cell col, OpOrder.Group opGroup)
         {
         }
 
         @Override
-        public void insert(ByteBuffer rowKey, Cell col)
+        public void insert(ByteBuffer rowKey, Cell col, OpOrder.Group opGroup)
         {
         }
 
         @Override
-        public void update(ByteBuffer rowKey, Cell col)
+        public void update(ByteBuffer rowKey, Cell col, OpOrder.Group opGroup)
         {
         }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/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 ea5dd3e..fd491d5 100644
--- a/test/unit/org/apache/cassandra/db/context/CounterContextTest.java
+++ b/test/unit/org/apache/cassandra/db/context/CounterContextTest.java
@@ -30,6 +30,11 @@ import org.apache.cassandra.db.ClockAndCount;
 import org.apache.cassandra.db.context.CounterContext.Relationship;
 import org.apache.cassandra.Util;
 import org.apache.cassandra.utils.*;
+import org.apache.cassandra.utils.concurrent.OpOrder;
+import org.apache.cassandra.utils.memory.AbstractAllocator;
+import org.apache.cassandra.utils.memory.HeapAllocator;
+import org.apache.cassandra.utils.memory.HeapPool;
+import org.apache.cassandra.utils.memory.Pool;
 
 import static org.apache.cassandra.db.context.CounterContext.ContextState;
 
@@ -44,10 +49,12 @@ public class CounterContextTest
     private static final int countLength = 8;
     private static final int stepLength = idLength + clockLength + countLength;
 
+    private static final Pool POOL = new HeapPool(Integer.MAX_VALUE, 1f, null);
+
     /** Allocates 1 byte from a new SlabAllocator and returns it. */
-    private Allocator bumpedSlab()
+    private AbstractAllocator bumpedSlab()
     {
-        SlabAllocator allocator = new SlabAllocator();
+        AbstractAllocator allocator = POOL.newAllocator(new OpOrder());
         allocator.allocate(1);
         return allocator;
     }
@@ -59,7 +66,7 @@ public class CounterContextTest
         runAllocate(bumpedSlab());
     }
 
-    private void runAllocate(Allocator allocator)
+    private void runAllocate(AbstractAllocator allocator)
     {
         ContextState allGlobal = ContextState.allocate(3, 0, 0, allocator);
         assertEquals(headerSizeLength + 3 * headerEltLength + 3 * stepLength, allGlobal.context.remaining());
@@ -81,7 +88,7 @@ public class CounterContextTest
         runDiff(bumpedSlab());
     }
 
-    private void runDiff(Allocator allocator)
+    private void runDiff(AbstractAllocator allocator)
     {
         ContextState left;
         ContextState right;
@@ -262,7 +269,7 @@ public class CounterContextTest
         runMerge(bumpedSlab());
     }
 
-    private void runMerge(Allocator allocator)
+    private void runMerge(AbstractAllocator allocator)
     {
         // note: local counts aggregated; remote counts are reconciled (i.e. take max)
         ContextState left = ContextState.allocate(0, 1, 3, allocator);
@@ -389,7 +396,7 @@ public class CounterContextTest
         runTotal(bumpedSlab());
     }
 
-    private void runTotal(Allocator allocator)
+    private void runTotal(AbstractAllocator allocator)
     {
         ContextState mixed = ContextState.allocate(0, 1, 4, allocator);
         mixed.writeRemote(CounterId.fromInt(1), 1L, 1L);
@@ -412,7 +419,7 @@ public class CounterContextTest
         ContextState state;
         ByteBuffer marked;
         ByteBuffer cleared;
-        Allocator allocator = HeapAllocator.instance;
+        AbstractAllocator allocator = HeapAllocator.instance;
 
         // mark/clear for remote-only contexts is a no-op
         state = ContextState.allocate(0, 0, 1, allocator);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/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 60cb19a..660a6e0 100644
--- a/test/unit/org/apache/cassandra/db/index/PerRowSecondaryIndexTest.java
+++ b/test/unit/org/apache/cassandra/db/index/PerRowSecondaryIndexTest.java
@@ -18,11 +18,12 @@
 package org.apache.cassandra.db.index;
 
 
-import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.Arrays;
 import java.util.Set;
 
+import org.apache.cassandra.utils.concurrent.OpOrder;
+import org.apache.cassandra.utils.memory.AbstractAllocator;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -137,7 +138,7 @@ public class PerRowSecondaryIndexTest extends SchemaLoader
         }
 
         @Override
-        public void delete(DecoratedKey key)
+        public void delete(DecoratedKey key, OpOrder.Group opGroup)
         {
         }
 
@@ -174,9 +175,9 @@ public class PerRowSecondaryIndexTest extends SchemaLoader
         }
 
         @Override
-        public long getLiveSize()
+        public AbstractAllocator getOnHeapAllocator()
         {
-            return 0;
+            return null;
         }
 
         @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/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 3c9d015..c7e7ee7 100644
--- a/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
+++ b/test/unit/org/apache/cassandra/streaming/StreamingTransferTest.java
@@ -50,7 +50,7 @@ import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.CounterId;
 import org.apache.cassandra.utils.FBUtilities;
-import org.apache.cassandra.utils.HeapAllocator;
+import org.apache.cassandra.utils.memory.HeapAllocator;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;


[3/6] remove Table.switchlock and introduce o.a.c.utils.memory package patch by Benedict Elliott Smith; reviewed by jbellis for CASSANDRA-5549

Posted by jb...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/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 bbdcf9f..66e549d 100644
--- a/src/java/org/apache/cassandra/db/index/SecondaryIndexManager.java
+++ b/src/java/org/apache/cassandra/db/index/SecondaryIndexManager.java
@@ -21,6 +21,8 @@ import java.nio.ByteBuffer;
 import java.util.*;
 import java.util.concurrent.*;
 
+import org.apache.cassandra.utils.concurrent.OpOrder;
+
 import org.apache.commons.lang3.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -312,8 +314,22 @@ public class SecondaryIndexManager
      */
     public void flushIndexesBlocking()
     {
+        // despatch flushes for all CFS backed indexes
+        List<Future<?>> wait = new ArrayList<>();
+        synchronized (baseCfs.getDataTracker())
+        {
+            for (SecondaryIndex index : indexesByColumn.values())
+                if (index.getIndexCfs() != null)
+                    wait.add(index.getIndexCfs().forceFlush());
+        }
+
+        // blockingFlush any non-CFS-backed indexes
         for (SecondaryIndex index : indexesByColumn.values())
-            index.forceBlockingFlush();
+            if (index.getIndexCfs() == null)
+                index.forceBlockingFlush();
+
+        // wait for the CFS-backed index flushes to complete
+        FBUtilities.waitOnFutures(wait);
     }
 
     /**
@@ -384,23 +400,12 @@ public class SecondaryIndexManager
     }
 
     /**
-     * @return total current ram size of all indexes
-     */
-    public long getTotalLiveSize()
-    {
-        long total = 0;
-        for (SecondaryIndex index : getIndexes())
-            total += index.getLiveSize();
-        return total;
-    }
-
-    /**
      * When building an index against existing data, add the given row to the index
      *
      * @param key the row key
      * @param cf the current rows data
      */
-    public void indexRow(ByteBuffer key, ColumnFamily cf)
+    public void indexRow(ByteBuffer key, ColumnFamily cf, OpOrder.Group opGroup)
     {
         // Update entire row only once per row level index
         Set<Class<? extends SecondaryIndex>> appliedRowLevelIndexes = null;
@@ -419,7 +424,7 @@ public class SecondaryIndexManager
             {
                 for (Cell cell : cf)
                     if (index.indexes(cell.name()))
-                        ((PerColumnSecondaryIndex) index).insert(key, cell);
+                        ((PerColumnSecondaryIndex) index).insert(key, cell, opGroup);
             }
         }
     }
@@ -430,7 +435,7 @@ public class SecondaryIndexManager
      * @param key the row key
      * @param indexedColumnsInRow all column names in row
      */
-    public void deleteFromIndexes(DecoratedKey key, List<Cell> indexedColumnsInRow)
+    public void deleteFromIndexes(DecoratedKey key, List<Cell> indexedColumnsInRow, OpOrder.Group opGroup)
     {
         // Update entire row only once per row level index
         Set<Class<? extends SecondaryIndex>> cleanedRowLevelIndexes = null;
@@ -448,11 +453,11 @@ public class SecondaryIndexManager
                     cleanedRowLevelIndexes = new HashSet<>();
 
                 if (cleanedRowLevelIndexes.add(index.getClass()))
-                    ((PerRowSecondaryIndex)index).delete(key);
+                    ((PerRowSecondaryIndex)index).delete(key, opGroup);
             }
             else
             {
-                ((PerColumnSecondaryIndex) index).delete(key.key, cell);
+                ((PerColumnSecondaryIndex) index).delete(key.key, cell, opGroup);
             }
         }
     }
@@ -464,19 +469,27 @@ public class SecondaryIndexManager
      * can get updated. Note: only a CF backed by AtomicSortedColumns implements
      * this behaviour fully, other types simply ignore the index updater.
      */
-    public Updater updaterFor(DecoratedKey key, ColumnFamily cf)
+    public Updater updaterFor(DecoratedKey key, ColumnFamily cf, OpOrder.Group opGroup)
     {
         return (indexesByColumn.isEmpty() && rowLevelIndexMap.isEmpty())
                 ? nullUpdater
-                : new StandardUpdater(key, cf);
+                : new StandardUpdater(key, cf, opGroup);
     }
 
     /**
      * Updated closure with only the modified row key.
      */
-    public Updater updaterFor(DecoratedKey key)
+    public Updater updaterFor(DecoratedKey key, OpOrder.Group opGroup)
     {
-        return updaterFor(key, null);
+        return updaterFor(key, null, opGroup);
+    }
+
+    /**
+     * Updated closure with only the modified row key.
+     */
+    public Updater gcUpdaterFor(DecoratedKey key)
+    {
+        return new GCUpdater(key);
     }
 
     /**
@@ -583,15 +596,65 @@ public class SecondaryIndexManager
         public void updateRowLevelIndexes();
     }
 
-    private class StandardUpdater implements Updater
+    private final class GCUpdater implements Updater
+    {
+        private final DecoratedKey key;
+
+        public GCUpdater(DecoratedKey key)
+        {
+            this.key = key;
+        }
+
+        public void insert(Cell cell)
+        {
+            throw new UnsupportedOperationException();
+        }
+
+        public void update(Cell oldCell, Cell newCell)
+        {
+            throw new UnsupportedOperationException();
+        }
+
+        public void remove(Cell cell)
+        {
+            if (cell.isMarkedForDelete(System.currentTimeMillis()))
+                return;
+
+            for (SecondaryIndex index : indexFor(cell.name()))
+            {
+                if (index instanceof PerColumnSecondaryIndex)
+                {
+                    OpOrder.Group opGroup = baseCfs.keyspace.writeOrder.start();
+                    try
+                    {
+                        ((PerColumnSecondaryIndex) index).delete(key.key, cell, opGroup);
+                    }
+                    finally
+                    {
+                        opGroup.finishOne();
+                    }
+                }
+            }
+        }
+
+        public void updateRowLevelIndexes()
+        {
+            for (SecondaryIndex index : rowLevelIndexMap.values())
+                ((PerRowSecondaryIndex) index).index(key.key, null);
+        }
+    }
+
+    private final class StandardUpdater implements Updater
     {
         private final DecoratedKey key;
         private final ColumnFamily cf;
+        private final OpOrder.Group opGroup;
 
-        public StandardUpdater(DecoratedKey key, ColumnFamily cf)
+        public StandardUpdater(DecoratedKey key, ColumnFamily cf, OpOrder.Group opGroup)
         {
             this.key = key;
             this.cf = cf;
+            this.opGroup = opGroup;
         }
 
         public void insert(Cell cell)
@@ -601,7 +664,7 @@ public class SecondaryIndexManager
 
             for (SecondaryIndex index : indexFor(cell.name()))
                 if (index instanceof PerColumnSecondaryIndex)
-                    ((PerColumnSecondaryIndex) index).insert(key.key, cell);
+                    ((PerColumnSecondaryIndex) index).insert(key.key, cell, opGroup);
         }
 
         public void update(Cell oldCell, Cell cell)
@@ -616,8 +679,8 @@ public class SecondaryIndexManager
                     // insert the new value before removing the old one, so we never have a period
                     // where the row is invisible to both queries (the opposite seems preferable); see CASSANDRA-5540
                     if (!cell.isMarkedForDelete(System.currentTimeMillis()))
-                        ((PerColumnSecondaryIndex) index).insert(key.key, cell);
-                    ((PerColumnSecondaryIndex) index).delete(key.key, oldCell);
+                        ((PerColumnSecondaryIndex) index).insert(key.key, cell, opGroup);
+                    ((PerColumnSecondaryIndex) index).delete(key.key, oldCell, opGroup);
                 }
             }
         }
@@ -629,7 +692,7 @@ public class SecondaryIndexManager
 
             for (SecondaryIndex index : indexFor(cell.name()))
                 if (index instanceof PerColumnSecondaryIndex)
-                   ((PerColumnSecondaryIndex) index).delete(key.key, cell);
+                   ((PerColumnSecondaryIndex) index).delete(key.key, cell, opGroup);
         }
 
         public void updateRowLevelIndexes()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/src/java/org/apache/cassandra/db/index/composites/CompositesIndex.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/index/composites/CompositesIndex.java b/src/java/org/apache/cassandra/db/index/composites/CompositesIndex.java
index c92f653..ec1118b 100644
--- a/src/java/org/apache/cassandra/db/index/composites/CompositesIndex.java
+++ b/src/java/org/apache/cassandra/db/index/composites/CompositesIndex.java
@@ -22,6 +22,7 @@ import java.util.HashMap;
 import java.util.Map;
 import java.util.Set;
 
+import org.apache.cassandra.utils.concurrent.OpOrder;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.db.*;
@@ -130,13 +131,21 @@ public abstract class CompositesIndex extends AbstractSimplePerColumnSecondaryIn
 
     public void delete(IndexedEntry entry)
     {
-        int localDeletionTime = (int) (System.currentTimeMillis() / 1000);
-        ColumnFamily cfi = ArrayBackedSortedColumns.factory.create(indexCfs.metadata);
-        cfi.addTombstone(entry.indexEntry, localDeletionTime, entry.timestamp);
-        indexCfs.apply(entry.indexValue, cfi, SecondaryIndexManager.nullUpdater);
-        if (logger.isDebugEnabled())
-            logger.debug("removed index entry for cleaned-up value {}:{}", entry.indexValue, cfi);
-
+        // start a mini-transaction for this delete, to ensure safe memtable updates
+        OpOrder.Group opGroup = baseCfs.keyspace.writeOrder.start();
+        try
+        {
+            int localDeletionTime = (int) (System.currentTimeMillis() / 1000);
+            ColumnFamily cfi = ArrayBackedSortedColumns.factory.create(indexCfs.metadata);
+            cfi.addTombstone(entry.indexEntry, localDeletionTime, entry.timestamp);
+            indexCfs.apply(entry.indexValue, cfi, SecondaryIndexManager.nullUpdater, opGroup, null);
+            if (logger.isDebugEnabled())
+                logger.debug("removed index entry for cleaned-up value {}:{}", entry.indexValue, cfi);
+        }
+        finally
+        {
+            opGroup.finishOne();
+        }
     }
 
     protected AbstractType<?> getExpressionComparator()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/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 932d446..2ed888f 100644
--- a/src/java/org/apache/cassandra/db/index/keys/KeysSearcher.java
+++ b/src/java/org/apache/cassandra/db/index/keys/KeysSearcher.java
@@ -24,6 +24,8 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Set;
 
+import org.apache.cassandra.utils.concurrent.OpOrder;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -38,7 +40,7 @@ import org.apache.cassandra.db.filter.QueryFilter;
 import org.apache.cassandra.db.index.*;
 import org.apache.cassandra.dht.AbstractBounds;
 import org.apache.cassandra.dht.Range;
-import org.apache.cassandra.utils.HeapAllocator;
+import org.apache.cassandra.utils.memory.HeapAllocator;
 
 public class KeysSearcher extends SecondaryIndexSearcher
 {
@@ -186,7 +188,15 @@ public class KeysSearcher extends SecondaryIndexSearcher
                         {
                             // delete the index entry w/ its own timestamp
                             Cell dummyCell = new Cell(primaryColumn, indexKey.key, cell.timestamp());
-                            ((PerColumnSecondaryIndex)index).delete(dk.key, dummyCell);
+                            OpOrder.Group opGroup = baseCfs.keyspace.writeOrder.start();
+                            try
+                            {
+                                ((PerColumnSecondaryIndex)index).delete(dk.key, dummyCell, opGroup);
+                            }
+                            finally
+                            {
+                                opGroup.finishOne();
+                            }
                             continue;
                         }
                         return new Row(dk, data);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/src/java/org/apache/cassandra/dht/ByteOrderedPartitioner.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/dht/ByteOrderedPartitioner.java b/src/java/org/apache/cassandra/dht/ByteOrderedPartitioner.java
index ccf7089..f46026f 100644
--- a/src/java/org/apache/cassandra/dht/ByteOrderedPartitioner.java
+++ b/src/java/org/apache/cassandra/dht/ByteOrderedPartitioner.java
@@ -17,14 +17,24 @@
  */
 package org.apache.cassandra.dht;
 
+import org.apache.cassandra.utils.ObjectSizes;
+
 import java.nio.ByteBuffer;
 
 public class ByteOrderedPartitioner extends AbstractByteOrderedPartitioner
 {
+    private static final long EMPTY_SIZE = ObjectSizes.measure(MINIMUM);
+
     public BytesToken getToken(ByteBuffer key)
     {
         if (key.remaining() == 0)
             return MINIMUM;
         return new BytesToken(key);
     }
+
+    @Override
+    public long getHeapSizeOf(BytesToken token)
+    {
+        return EMPTY_SIZE + ObjectSizes.sizeOfArray(token.token);
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/src/java/org/apache/cassandra/dht/IPartitioner.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/dht/IPartitioner.java b/src/java/org/apache/cassandra/dht/IPartitioner.java
index 46165b8..0ef242f 100644
--- a/src/java/org/apache/cassandra/dht/IPartitioner.java
+++ b/src/java/org/apache/cassandra/dht/IPartitioner.java
@@ -56,6 +56,13 @@ public interface IPartitioner<T extends Token>
     public T getToken(ByteBuffer key);
 
     /**
+     *
+     * @param token
+     * @return the on-heap memory used by the provided token
+     */
+    public long getHeapSizeOf(T token);
+
+    /**
      * @return a randomly generated token
      */
     public T getRandomToken();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/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 b24eede..2e75bd3 100644
--- a/src/java/org/apache/cassandra/dht/LocalPartitioner.java
+++ b/src/java/org/apache/cassandra/dht/LocalPartitioner.java
@@ -25,9 +25,12 @@ import java.util.Map;
 import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.ObjectSizes;
 
 public class LocalPartitioner extends AbstractPartitioner<LocalToken>
 {
+    private static final long EMPTY_SIZE = ObjectSizes.measure(new LocalToken(null, null));
+
     private final AbstractType<?> comparator;
 
     public LocalPartitioner(AbstractType<?> comparator)
@@ -55,6 +58,11 @@ public class LocalPartitioner extends AbstractPartitioner<LocalToken>
         return new LocalToken(comparator, key);
     }
 
+    public long getHeapSizeOf(LocalToken token)
+    {
+        return EMPTY_SIZE + ObjectSizes.sizeOnHeapOf(token.token);
+    }
+
     public LocalToken getRandomToken()
     {
         throw new UnsupportedOperationException();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/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 3a045d7..de22a34 100644
--- a/src/java/org/apache/cassandra/dht/Murmur3Partitioner.java
+++ b/src/java/org/apache/cassandra/dht/Murmur3Partitioner.java
@@ -32,6 +32,7 @@ import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.MurmurHash;
+import org.apache.cassandra.utils.ObjectSizes;
 
 /**
  * This class generates a BigIntegerToken using a Murmur3 hash.
@@ -41,6 +42,8 @@ public class Murmur3Partitioner extends AbstractPartitioner<LongToken>
     public static final LongToken MINIMUM = new LongToken(Long.MIN_VALUE);
     public static final long MAXIMUM = Long.MAX_VALUE;
 
+    private static final int HEAP_SIZE = (int) ObjectSizes.measureDeep(MINIMUM);
+
     public DecoratedKey decorateKey(ByteBuffer key)
     {
         return new DecoratedKey(getToken(key), key);
@@ -94,6 +97,11 @@ public class Murmur3Partitioner extends AbstractPartitioner<LongToken>
         return new LongToken(normalize(hash[0]));
     }
 
+    public long getHeapSizeOf(LongToken token)
+    {
+        return HEAP_SIZE;
+    }
+
     public LongToken getRandomToken()
     {
         return new LongToken(normalize(FBUtilities.threadLocalRandom().nextLong()));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/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 4c5a7d4..190851c 100644
--- a/src/java/org/apache/cassandra/dht/OrderPreservingPartitioner.java
+++ b/src/java/org/apache/cassandra/dht/OrderPreservingPartitioner.java
@@ -31,6 +31,7 @@ import org.apache.cassandra.gms.VersionedValue;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.ObjectSizes;
 import org.apache.cassandra.utils.Pair;
 
 public class OrderPreservingPartitioner extends AbstractPartitioner<StringToken>
@@ -39,6 +40,8 @@ public class OrderPreservingPartitioner extends AbstractPartitioner<StringToken>
 
     public static final BigInteger CHAR_MASK = new BigInteger("65535");
 
+    private static final long EMPTY_SIZE = ObjectSizes.measure(MINIMUM);
+
     public DecoratedKey decorateKey(ByteBuffer key)
     {
         return new DecoratedKey(getToken(key), key);
@@ -169,6 +172,11 @@ public class OrderPreservingPartitioner extends AbstractPartitioner<StringToken>
         return new StringToken(skey);
     }
 
+    public long getHeapSizeOf(StringToken token)
+    {
+        return EMPTY_SIZE + ObjectSizes.sizeOf(token.token);
+    }
+
     public Map<Token, Float> describeOwnership(List<Token> sortedTokens)
     {
         // allTokens will contain the count and be returned, sorted_ranges is shorthand for token<->token math.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/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 c9ddccf..a5b0ce9 100644
--- a/src/java/org/apache/cassandra/dht/RandomPartitioner.java
+++ b/src/java/org/apache/cassandra/dht/RandomPartitioner.java
@@ -29,6 +29,7 @@ import org.apache.cassandra.db.marshal.IntegerType;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.GuidGenerator;
+import org.apache.cassandra.utils.ObjectSizes;
 import org.apache.cassandra.utils.Pair;
 
 /**
@@ -40,6 +41,8 @@ public class RandomPartitioner extends AbstractPartitioner<BigIntegerToken>
     public static final BigIntegerToken MINIMUM = new BigIntegerToken("-1");
     public static final BigInteger MAXIMUM = new BigInteger("2").pow(127);
 
+    private static final int EMPTY_SIZE = (int) ObjectSizes.measureDeep(new BigIntegerToken(FBUtilities.hashToBigInteger(ByteBuffer.allocate(1))));
+
     public DecoratedKey decorateKey(ByteBuffer key)
     {
         return new DecoratedKey(getToken(key), key);
@@ -123,6 +126,11 @@ public class RandomPartitioner extends AbstractPartitioner<BigIntegerToken>
         return new BigIntegerToken(FBUtilities.hashToBigInteger(key));
     }
 
+    public long getHeapSizeOf(BigIntegerToken token)
+    {
+        return EMPTY_SIZE;
+    }
+
     public Map<Token, Float> describeOwnership(List<Token> sortedTokens)
     {
         Map<Token, Float> ownerships = new HashMap<Token, Float>();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/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 587d932..303f73c 100644
--- a/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
@@ -31,8 +31,8 @@ import org.apache.cassandra.db.context.CounterContext;
 import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
 import org.apache.cassandra.utils.CounterId;
-import org.apache.cassandra.utils.HeapAllocator;
 import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.memory.HeapAllocator;
 
 public abstract class AbstractSSTableSimpleWriter
 {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/src/java/org/apache/cassandra/io/sstable/IndexHelper.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/IndexHelper.java b/src/java/org/apache/cassandra/io/sstable/IndexHelper.java
index e644974..0d6a06e 100644
--- a/src/java/org/apache/cassandra/io/sstable/IndexHelper.java
+++ b/src/java/org/apache/cassandra/io/sstable/IndexHelper.java
@@ -155,6 +155,8 @@ public class IndexHelper
 
     public static class IndexInfo
     {
+        private static final long EMPTY_SIZE = ObjectSizes.measure(new IndexInfo(null, null, 0, 0));
+
         public final long width;
         public final Composite lastName;
         public final Composite firstName;
@@ -202,15 +204,9 @@ public class IndexHelper
             }
         }
 
-        public long memorySize()
+        public long excessHeapSize()
         {
-            return ObjectSizes.getFieldSize(// firstName
-                                            ObjectSizes.getReferenceSize() +
-                                            // lastName
-                                            ObjectSizes.getReferenceSize() +
-                                            TypeSizes.NATIVE.sizeof(offset) +
-                                            TypeSizes.NATIVE.sizeof(width))
-                   + firstName.memorySize() + lastName.memorySize();
+            return EMPTY_SIZE + firstName.unsharedHeapSize() + lastName.unsharedHeapSize();
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/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 dfccf28..0cc9d40 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTable.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTable.java
@@ -37,7 +37,7 @@ import org.apache.cassandra.io.FSWriteError;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.io.util.RandomAccessReader;
 import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.HeapAllocator;
+import org.apache.cassandra.utils.memory.HeapAllocator;
 import org.apache.cassandra.utils.Pair;
 
 /**
@@ -125,7 +125,7 @@ public abstract class SSTable
      */
     public static DecoratedKey getMinimalKey(DecoratedKey key)
     {
-        return key.key.position() > 0 || key.key.hasRemaining()
+        return key.key.position() > 0 || key.key.hasRemaining() || !key.key.hasArray()
                                        ? new DecoratedKey(key.token, HeapAllocator.instance.clone(key.key))
                                        : key;
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/src/java/org/apache/cassandra/io/sstable/metadata/MetadataCollector.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/metadata/MetadataCollector.java b/src/java/org/apache/cassandra/io/sstable/metadata/MetadataCollector.java
index e20015d..c61b766 100644
--- a/src/java/org/apache/cassandra/io/sstable/metadata/MetadataCollector.java
+++ b/src/java/org/apache/cassandra/io/sstable/metadata/MetadataCollector.java
@@ -31,6 +31,7 @@ import org.apache.cassandra.io.sstable.*;
 import org.apache.cassandra.utils.EstimatedHistogram;
 import org.apache.cassandra.utils.MurmurHash;
 import org.apache.cassandra.utils.StreamingHistogram;
+import org.apache.cassandra.utils.memory.HeapAllocator;
 
 public class MetadataCollector
 {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/src/java/org/apache/cassandra/io/util/SequentialWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/SequentialWriter.java b/src/java/org/apache/cassandra/io/util/SequentialWriter.java
index dc95676..a69391e 100644
--- a/src/java/org/apache/cassandra/io/util/SequentialWriter.java
+++ b/src/java/org/apache/cassandra/io/util/SequentialWriter.java
@@ -37,9 +37,6 @@ public class SequentialWriter extends OutputStream
     // absolute path to the given file
     private final String filePath;
 
-    // so we can use the write(int) path w/o tons of new byte[] allocations
-    private final byte[] singleByteBuffer = new byte[1];
-
     protected byte[] buffer;
     private final boolean skipIOCache;
     private final int fd;
@@ -112,8 +109,18 @@ public class SequentialWriter extends OutputStream
 
     public void write(int value) throws ClosedChannelException
     {
-        singleByteBuffer[0] = (byte) value;
-        write(singleByteBuffer, 0, 1);
+        if (current >= bufferOffset + buffer.length)
+            reBuffer();
+
+        assert current < bufferOffset + buffer.length
+                : String.format("File (%s) offset %d, buffer offset %d.", getPath(), current, bufferOffset);
+
+        buffer[bufferCursor()] = (byte) value;
+
+        validBufferBytes += 1;
+        current += 1;
+        isDirty = true;
+        syncNeeded = true;
     }
 
     public void write(byte[] buffer) throws ClosedChannelException

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/src/java/org/apache/cassandra/metrics/ColumnFamilyMetrics.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/metrics/ColumnFamilyMetrics.java b/src/java/org/apache/cassandra/metrics/ColumnFamilyMetrics.java
index d42e54d..f6e3030 100644
--- a/src/java/org/apache/cassandra/metrics/ColumnFamilyMetrics.java
+++ b/src/java/org/apache/cassandra/metrics/ColumnFamilyMetrics.java
@@ -24,7 +24,6 @@ import com.yammer.metrics.core.*;
 import com.yammer.metrics.util.RatioGauge;
 
 import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.io.sstable.SSTableReader;
 import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
 import org.apache.cassandra.utils.EstimatedHistogram;
@@ -34,10 +33,14 @@ import org.apache.cassandra.utils.EstimatedHistogram;
  */
 public class ColumnFamilyMetrics
 {
-    /** Total amount of data stored in the memtable, including column related overhead. */
-    public final Gauge<Long> memtableDataSize;
-    /** Total amount of data stored in the memtables (2i and pending flush memtables included). */
-    public final Gauge<Long> allMemtablesDataSize;
+    /** Total amount of data stored in the memtable that resides on-heap, including column related overhead and overwritten rows. */
+    public final Gauge<Long> memtableHeapSize;
+    /** Total amount of live data stored in the memtable, excluding any data structure overhead */
+    public final Gauge<Long> memtableLiveDataSize;
+    /** Total amount of data stored in the memtables (2i and pending flush memtables included) that resides on-heap. */
+    public final Gauge<Long> allMemtablesHeapSize;
+    /** Total amount of live data stored in the memtables (2i and pending flush memtables included) that resides off-heap, excluding any data structure overhead */
+    public final Gauge<Long> allMemtablesLiveDataSize;
     /** Total number of columns present in the memtable. */
     public final Gauge<Long> memtableColumnsCount;
     /** Number of times flush has resulted in the memtable being switched out. */
@@ -55,7 +58,7 @@ public class ColumnFamilyMetrics
     /** (Local) write metrics */
     public final LatencyMetrics writeLatency;
     /** Estimated number of tasks pending for this column family */
-    public final Gauge<Integer> pendingTasks;
+    public final Counter pendingFlushes;
     /** Number of SSTables on disk for this CF */
     public final Gauge<Integer> liveSSTableCount;
     /** Disk space used by SSTables belonging to this CF */
@@ -90,6 +93,9 @@ public class ColumnFamilyMetrics
     public final Timer coordinatorReadLatency;
     public final Timer coordinatorScanLatency;
 
+    /** Time spent waiting for free memtable space, either on- or off-heap */
+    public final Timer waitingOnFreeMemtableSpace;
+
     private final MetricNameFactory factory;
 
     public final Counter speculativeRetries;
@@ -111,21 +117,41 @@ public class ColumnFamilyMetrics
         {
             public Long value()
             {
-                return cfs.getDataTracker().getMemtable().getOperations();
+                return cfs.getDataTracker().getView().getCurrentMemtable().getOperations();
+            }
+        });
+        memtableHeapSize = Metrics.newGauge(factory.createMetricName("MemtableHeapSize"), new Gauge<Long>()
+        {
+            public Long value()
+            {
+                return cfs.getDataTracker().getView().getCurrentMemtable().getAllocator().owns();
+            }
+        });
+        memtableLiveDataSize = Metrics.newGauge(factory.createMetricName("MemtableLiveDataSize"), new Gauge<Long>()
+        {
+            public Long value()
+            {
+                return cfs.getDataTracker().getView().getCurrentMemtable().getLiveDataSize();
             }
         });
-        memtableDataSize = Metrics.newGauge(factory.createMetricName("MemtableDataSize"), new Gauge<Long>()
+        allMemtablesHeapSize = Metrics.newGauge(factory.createMetricName("AllMemtablesHeapSize"), new Gauge<Long>()
         {
             public Long value()
             {
-                return cfs.getDataTracker().getMemtable().getLiveSize();
+                long size = 0;
+                for (ColumnFamilyStore cfs2 : cfs.concatWithIndexes())
+                    size += cfs2.getDataTracker().getView().getCurrentMemtable().getAllocator().owns();
+                return size;
             }
         });
-        allMemtablesDataSize = Metrics.newGauge(factory.createMetricName("AllMemtablesDataSize"), new Gauge<Long>()
+        allMemtablesLiveDataSize = Metrics.newGauge(factory.createMetricName("AllMemtablesLiveDataSize"), new Gauge<Long>()
         {
             public Long value()
             {
-                return cfs.getTotalAllMemtablesLiveSize();
+                long size = 0;
+                for (ColumnFamilyStore cfs2 : cfs.concatWithIndexes())
+                    size += cfs2.getDataTracker().getView().getCurrentMemtable().getLiveDataSize();
+                return size;
             }
         });
         memtableSwitchCount = Metrics.newCounter(factory.createMetricName("MemtableSwitchCount"));
@@ -177,14 +203,7 @@ public class ColumnFamilyMetrics
         });
         readLatency = new LatencyMetrics(factory, "Read");
         writeLatency = new LatencyMetrics(factory, "Write");
-        pendingTasks = Metrics.newGauge(factory.createMetricName("PendingTasks"), new Gauge<Integer>()
-        {
-            public Integer value()
-            {
-                // TODO this actually isn't a good measure of pending tasks
-                return Keyspace.switchLock.getQueueLength();
-            }
-        });
+        pendingFlushes = Metrics.newCounter(factory.createMetricName("PendingFlushes"));
         liveSSTableCount = Metrics.newGauge(factory.createMetricName("LiveSSTableCount"), new Gauge<Integer>()
         {
             public Integer value()
@@ -319,7 +338,8 @@ public class ColumnFamilyMetrics
         liveScannedHistogram = Metrics.newHistogram(factory.createMetricName("LiveScannedHistogram"), true);
         coordinatorReadLatency = Metrics.newTimer(factory.createMetricName("CoordinatorReadLatency"), TimeUnit.MICROSECONDS, TimeUnit.SECONDS);
         coordinatorScanLatency = Metrics.newTimer(factory.createMetricName("CoordinatorScanLatency"), TimeUnit.MICROSECONDS, TimeUnit.SECONDS);
-        
+        waitingOnFreeMemtableSpace = Metrics.newTimer(factory.createMetricName("WaitingOnFreeMemtableSpace"), TimeUnit.MICROSECONDS, TimeUnit.SECONDS);
+
         trueSnapshotsSize = Metrics.newGauge(factory.createMetricName("SnapshotsSize"), new Gauge<Long>()
         {
             public Long value()
@@ -400,4 +420,5 @@ public class ColumnFamilyMetrics
             return new MetricName(groupName, type, metricName, keyspaceName + "." + columnFamilyName, mbeanName.toString());
         }
     }
+
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/src/java/org/apache/cassandra/metrics/CommitLogMetrics.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/metrics/CommitLogMetrics.java b/src/java/org/apache/cassandra/metrics/CommitLogMetrics.java
index 7c8ca61..e9c0719 100644
--- a/src/java/org/apache/cassandra/metrics/CommitLogMetrics.java
+++ b/src/java/org/apache/cassandra/metrics/CommitLogMetrics.java
@@ -20,9 +20,12 @@ package org.apache.cassandra.metrics;
 import com.yammer.metrics.Metrics;
 import com.yammer.metrics.core.Gauge;
 
+import com.yammer.metrics.core.Timer;
 import org.apache.cassandra.db.commitlog.AbstractCommitLogService;
 import org.apache.cassandra.db.commitlog.CommitLogSegmentManager;
 
+import java.util.concurrent.TimeUnit;
+
 /**
  * Metrics for commit log
  */
@@ -36,6 +39,10 @@ public class CommitLogMetrics
     public final Gauge<Long> pendingTasks;
     /** Current size used by all the commit log segments */
     public final Gauge<Long> totalCommitLogSize;
+    /** Time spent waiting for a CLS to be allocated - under normal conditions this should be zero */
+    public final Timer waitingOnSegmentAllocation;
+    /** The time spent waiting on CL sync; for Periodic this is only occurs when the sync is lagging its sync interval */
+    public final Timer waitingOnCommit;
 
     public CommitLogMetrics(final AbstractCommitLogService service, final CommitLogSegmentManager allocator)
     {
@@ -60,5 +67,7 @@ public class CommitLogMetrics
                 return allocator.bytesUsed();
             }
         });
+        waitingOnSegmentAllocation = Metrics.newTimer(factory.createMetricName("WaitingOnSegmentAllocation"), TimeUnit.MICROSECONDS, TimeUnit.SECONDS);
+        waitingOnCommit = Metrics.newTimer(factory.createMetricName("WaitingOnCommit"), TimeUnit.MICROSECONDS, TimeUnit.SECONDS);
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/src/java/org/apache/cassandra/service/CassandraDaemon.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/CassandraDaemon.java b/src/java/org/apache/cassandra/service/CassandraDaemon.java
index 424dbfa..2903fc4 100644
--- a/src/java/org/apache/cassandra/service/CassandraDaemon.java
+++ b/src/java/org/apache/cassandra/service/CassandraDaemon.java
@@ -45,7 +45,6 @@ import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Directories;
 import org.apache.cassandra.db.Keyspace;
-import org.apache.cassandra.db.MeteredFlusher;
 import org.apache.cassandra.db.SystemKeyspace;
 import org.apache.cassandra.db.commitlog.CommitLog;
 import org.apache.cassandra.db.compaction.CompactionManager;
@@ -258,10 +257,6 @@ public class CassandraDaemon
             logger.warn("Unable to start GCInspector (currently only supported on the Sun JVM)");
         }
 
-        // MeteredFlusher can block if flush queue fills up, so don't put on scheduledTasks
-        // Start it before commit log, so memtables can flush during commit log replay
-        StorageService.optionalTasks.scheduleWithFixedDelay(new MeteredFlusher(), 1000, 1000, TimeUnit.MILLISECONDS);
-
         // replay the log if necessary
         try
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/src/java/org/apache/cassandra/utils/Allocator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/Allocator.java b/src/java/org/apache/cassandra/utils/Allocator.java
deleted file mode 100644
index 7134353..0000000
--- a/src/java/org/apache/cassandra/utils/Allocator.java
+++ /dev/null
@@ -1,41 +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;
-
-import java.nio.ByteBuffer;
-
-public abstract class Allocator
-{
-    /**
-     * Allocate a slice of the given length.
-     */
-    public ByteBuffer clone(ByteBuffer buffer)
-    {
-        assert buffer != null;
-        ByteBuffer cloned = allocate(buffer.remaining());
-
-        cloned.mark();
-        cloned.put(buffer.duplicate());
-        cloned.reset();
-        return cloned;
-    }
-
-    public abstract ByteBuffer allocate(int size);
-
-    public abstract long getMinimumSize();
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/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 182ed21..0a94cc0 100644
--- a/src/java/org/apache/cassandra/utils/FBUtilities.java
+++ b/src/java/org/apache/cassandra/utils/FBUtilities.java
@@ -245,13 +245,6 @@ public class FBUtilities
         return out;
     }
 
-    public static BigInteger hashToBigInteger(ByteBuffer data)
-    {
-        byte[] result = hash(data);
-        BigInteger hash = new BigInteger(result);
-        return hash.abs();
-    }
-
     public static byte[] hash(ByteBuffer... data)
     {
         MessageDigest messageDigest = localMD5Digest.get();
@@ -266,6 +259,11 @@ public class FBUtilities
         return messageDigest.digest();
     }
 
+    public static BigInteger hashToBigInteger(ByteBuffer data)
+    {
+        return new BigInteger(hash(data)).abs();
+    }
+
     @Deprecated
     public static void serialize(TSerializer serializer, TBase struct, DataOutput out)
     throws IOException

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/src/java/org/apache/cassandra/utils/HeapAllocator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/HeapAllocator.java b/src/java/org/apache/cassandra/utils/HeapAllocator.java
deleted file mode 100644
index 3d19b98..0000000
--- a/src/java/org/apache/cassandra/utils/HeapAllocator.java
+++ /dev/null
@@ -1,41 +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;
-
-import java.nio.ByteBuffer;
-
-public final class HeapAllocator extends Allocator
-{
-    public static final HeapAllocator instance = new HeapAllocator();
-
-    /**
-     * Normally you should use HeapAllocator.instance, since there is no per-Allocator state.
-     * This is exposed so that the reflection done by Memtable works when SlabAllocator is disabled.
-     */
-    public HeapAllocator() {}
-
-    public ByteBuffer allocate(int size)
-    {
-        return ByteBuffer.allocate(size);
-    }
-
-    public long getMinimumSize()
-    {
-        return 0;
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/src/java/org/apache/cassandra/utils/ObjectSizes.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/ObjectSizes.java b/src/java/org/apache/cassandra/utils/ObjectSizes.java
index 28ddffd..3720385 100644
--- a/src/java/org/apache/cassandra/utils/ObjectSizes.java
+++ b/src/java/org/apache/cassandra/utils/ObjectSizes.java
@@ -21,242 +21,143 @@ package org.apache.cassandra.utils;
  */
 
 
-import java.lang.management.ManagementFactory;
-import java.lang.management.MemoryPoolMXBean;
 import java.nio.ByteBuffer;
 
+import org.github.jamm.MemoryLayoutSpecification;
 import org.github.jamm.MemoryMeter;
 
 /**
- * Modified version of the code from.
- * https://github.com/twitter/commons/blob/master
- * /src/java/com/twitter/common/objectsize/ObjectSizeCalculator.java
- *
- * Difference is that we don't use reflection.
+ * A convenience class for wrapping access to MemoryMeter
  */
 public class ObjectSizes
 {
-    public static final MemoryLayoutSpecification SPEC = getEffectiveMemoryLayoutSpecification();
-    private static final MemoryMeter meter = new MemoryMeter().omitSharedBufferOverhead();
+    private static final MemoryMeter meter = new MemoryMeter()
+                                             .omitSharedBufferOverhead()
+                                             .withGuessing(MemoryMeter.Guess.FALLBACK_UNSAFE);
+
+    private static final long BUFFER_EMPTY_SIZE = measure(ByteBufferUtil.EMPTY_BYTE_BUFFER);
+    private static final long STRING_EMPTY_SIZE = measure("");
 
     /**
-     * Describes constant memory overheads for various constructs in a JVM
-     * implementation.
+     * Memory a byte array consumes
+     * @param bytes byte array to get memory size
+     * @return heap-size of the array
      */
-    public interface MemoryLayoutSpecification
+    public static long sizeOfArray(byte[] bytes)
     {
-        int getArrayHeaderSize();
-
-        int getObjectHeaderSize();
-
-        int getObjectPadding();
-
-        int getReferenceSize();
-
-        int getSuperclassFieldPadding();
+        return sizeOfArray(bytes.length, 1);
     }
 
     /**
-     * Memory a class consumes, including the object header and the size of the fields.
-     * @param fieldsSize Total size of the primitive fields of a class
-     * @return Total in-memory size of the class
+     * Memory a long array consumes
+     * @param longs byte array to get memory size
+     * @return heap-size of the array
      */
-    public static long getFieldSize(long fieldsSize)
+    public static long sizeOfArray(long[] longs)
     {
-        return roundTo(SPEC.getObjectHeaderSize() + fieldsSize, SPEC.getObjectPadding());
+        return sizeOfArray(longs.length, 8);
     }
 
     /**
-     * Memory a super class consumes, given the primitive field sizes
-     * @param fieldsSize Total size of the primitive fields of the super class
-     * @return Total additional in-memory that the super class takes up
+     * Memory an int array consumes
+     * @param ints byte array to get memory size
+     * @return heap-size of the array
      */
-    public static long getSuperClassFieldSize(long fieldsSize)
+    public static long sizeOfArray(int[] ints)
     {
-        return roundTo(fieldsSize, SPEC.getSuperclassFieldPadding());
+        return sizeOfArray(ints.length, 4);
     }
 
     /**
-     * Memory an array will consume
-     * @param length Number of elements in the array
-     * @param elementSize In-memory size of each element's primitive stored
-     * @return In-memory size of the array
+     * Memory a reference array consumes
+     * @param length the length of the reference array
+     * @return heap-size of the array
      */
-    public static long getArraySize(int length, long elementSize)
+    public static long sizeOfReferenceArray(int length)
     {
-        return roundTo(SPEC.getArrayHeaderSize() + length * elementSize, SPEC.getObjectPadding());
+        return sizeOfArray(length, MemoryLayoutSpecification.SPEC.getReferenceSize());
     }
 
     /**
-     * Memory a byte array consumes
-     * @param bytes byte array to get memory size
-     * @return In-memory size of the array
+     * Memory a reference array consumes itself only
+     * @param objects the array to size
+     * @return heap-size of the array (excluding memory retained by referenced objects)
      */
-    public static long getArraySize(byte[] bytes)
+    public static long sizeOfArray(Object[] objects)
     {
-        return getArraySize(bytes.length, 1);
+        return sizeOfReferenceArray(objects.length);
+    }
+
+    private static long sizeOfArray(int length, long elementSize)
+    {
+        return MemoryLayoutSpecification.sizeOfArray(length, elementSize);
     }
 
     /**
      * Memory a ByteBuffer array consumes.
      */
-    public static long getArraySize(ByteBuffer[] array)
+    public static long sizeOnHeapOf(ByteBuffer[] array)
     {
         long allElementsSize = 0;
         for (int i = 0; i < array.length; i++)
             if (array[i] != null)
-                allElementsSize += getSize(array[i]);
+                allElementsSize += sizeOnHeapOf(array[i]);
 
-        return allElementsSize + getArraySize(array.length, getReferenceSize());
+        return allElementsSize + sizeOfArray(array);
     }
 
+    public static long sizeOnHeapExcludingData(ByteBuffer[] array)
+    {
+        return BUFFER_EMPTY_SIZE * array.length + sizeOfArray(array);
+    }
     /**
      * Memory a byte buffer consumes
      * @param buffer ByteBuffer to calculate in memory size
      * @return Total in-memory size of the byte buffer
      */
-    public static long getSize(ByteBuffer buffer)
+    public static long sizeOnHeapOf(ByteBuffer buffer)
     {
-        long size = 0;
-        /* BB Class */
-        // final byte[] hb;
-        // final int offset;
-        // boolean isReadOnly;
-        size += ObjectSizes.getFieldSize(1L + 4 + ObjectSizes.getReferenceSize() + ObjectSizes.getArraySize(buffer.capacity(), 1));
-        /* Super Class */
-        // private int mark;
-        // private int position;
-        // private int limit;
-        // private int capacity;
-        size += ObjectSizes.getSuperClassFieldSize(4L + 4 + 4 + 4 + 8);
-        return size;
+        if (buffer.isDirect())
+            return BUFFER_EMPTY_SIZE;
+        // if we're only referencing a sub-portion of the ByteBuffer, don't count the array overhead (assume it's slab
+        // allocated, so amortized over all the allocations the overhead is negligible and better to undercount than over)
+        if (buffer.capacity() > buffer.remaining())
+            return buffer.remaining();
+        return BUFFER_EMPTY_SIZE + sizeOfArray(buffer.capacity(), 1);
     }
 
-    public static long roundTo(long x, int multiple)
+    public static long sizeOnHeapExcludingData(ByteBuffer buffer)
     {
-        return ((x + multiple - 1) / multiple) * multiple;
+        return BUFFER_EMPTY_SIZE;
     }
 
     /**
-     * @return Memory a reference consumes on the current architecture.
+     * Memory a String consumes
+     * @param str String to calculate memory size of
+     * @return Total in-memory size of the String
      */
-    public static int getReferenceSize()
+    public static long sizeOf(String str)
     {
-        return SPEC.getReferenceSize();
+        return STRING_EMPTY_SIZE + sizeOfArray(str.length(), 2);
     }
 
-    private static MemoryLayoutSpecification getEffectiveMemoryLayoutSpecification()
+    /**
+     * @param pojo the object to measure
+     * @return the size on the heap of the instance and all retained heap referenced by it, excluding portions of
+     * ByteBuffer that are not directly referenced by it but including any other referenced that may also be retained
+     * by other objects.
+     */
+    public static long measureDeep(Object pojo)
     {
-        final String dataModel = System.getProperty("sun.arch.data.model");
-        if ("32".equals(dataModel))
-        {
-            // Running with 32-bit data model
-            return new MemoryLayoutSpecification()
-            {
-                public int getArrayHeaderSize()
-                {
-                    return 12;
-                }
-
-                public int getObjectHeaderSize()
-                {
-                    return 8;
-                }
-
-                public int getObjectPadding()
-                {
-                    return 8;
-                }
-
-                public int getReferenceSize()
-                {
-                    return 4;
-                }
-
-                public int getSuperclassFieldPadding()
-                {
-                    return 4;
-                }
-            };
-        }
-
-        final String strVmVersion = System.getProperty("java.vm.version");
-        final int vmVersion = Integer.parseInt(strVmVersion.substring(0, strVmVersion.indexOf('.')));
-        if (vmVersion >= 17)
-        {
-            long maxMemory = 0;
-            for (MemoryPoolMXBean mp : ManagementFactory.getMemoryPoolMXBeans())
-            {
-                maxMemory += mp.getUsage().getMax();
-            }
-            if (maxMemory < 30L * 1024 * 1024 * 1024)
-            {
-                // HotSpot 17.0 and above use compressed OOPs below 30GB of RAM
-                // total for all memory pools (yes, including code cache).
-                return new MemoryLayoutSpecification()
-                {
-                    public int getArrayHeaderSize()
-                    {
-                        return 16;
-                    }
-
-                    public int getObjectHeaderSize()
-                    {
-                        return 12;
-                    }
-
-                    public int getObjectPadding()
-                    {
-                        return 8;
-                    }
-
-                    public int getReferenceSize()
-                    {
-                        return 4;
-                    }
-
-                    public int getSuperclassFieldPadding()
-                    {
-                        return 4;
-                    }
-                };
-            }
-        }
-
-        /* Worst case we over count. */
-
-        // In other cases, it's a 64-bit uncompressed OOPs object model
-        return new MemoryLayoutSpecification()
-        {
-            public int getArrayHeaderSize()
-            {
-                return 24;
-            }
-
-            public int getObjectHeaderSize()
-            {
-                return 16;
-            }
-
-            public int getObjectPadding()
-            {
-                return 8;
-            }
-
-            public int getReferenceSize()
-            {
-                return 8;
-            }
-
-            public int getSuperclassFieldPadding()
-            {
-                return 8;
-            }
-        };
+        return meter.measureDeep(pojo);
     }
 
-    public static long measureDeep(Object pojo)
+    /**
+     * @param pojo the object to measure
+     * @return the size on the heap of the instance only, excluding any referenced objects
+     */
+    public static long measure(Object pojo)
     {
-        return meter.measureDeep(pojo);
+        return meter.measure(pojo);
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/src/java/org/apache/cassandra/utils/SlabAllocator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/SlabAllocator.java b/src/java/org/apache/cassandra/utils/SlabAllocator.java
deleted file mode 100644
index dedf869..0000000
--- a/src/java/org/apache/cassandra/utils/SlabAllocator.java
+++ /dev/null
@@ -1,220 +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;
-
-import java.nio.ByteBuffer;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.atomic.AtomicReference;
-
-import com.google.common.base.Preconditions;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * The SlabAllocator is a bump-the-pointer allocator that allocates
- * large (2MB by default) regions and then doles them out to threads that request
- * slices into the array.
- * <p/>
- * The purpose of this class is to combat heap fragmentation in long lived
- * objects: by ensuring that all allocations with similar lifetimes
- * only to large regions of contiguous memory, we ensure that large blocks
- * get freed up at the same time.
- * <p/>
- * Otherwise, variable length byte arrays allocated end up
- * interleaved throughout the heap, and the old generation gets progressively
- * more fragmented until a stop-the-world compacting collection occurs.
- */
-public class SlabAllocator extends Allocator
-{
-    private static final Logger logger = LoggerFactory.getLogger(SlabAllocator.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
-
-    private final AtomicReference<Region> currentRegion = new AtomicReference<Region>();
-    private final AtomicInteger regionCount = new AtomicInteger(0);
-    private AtomicLong unslabbed = new AtomicLong(0);
-
-    public ByteBuffer allocate(int size)
-    {
-        assert size >= 0;
-        if (size == 0)
-            return ByteBufferUtil.EMPTY_BYTE_BUFFER;
-
-        // 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);
-            return ByteBuffer.allocate(size);
-        }
-
-        while (true)
-        {
-            Region region = getRegion();
-
-            // Try to allocate from this region
-            ByteBuffer cloned = region.allocate(size);
-            if (cloned != null)
-                return cloned;
-
-            // not enough space!
-            currentRegion.compareAndSet(region, null);
-        }
-    }
-
-    /**
-     * 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 an uninitialized region
-            // (which is cheap to allocate)
-            region = new Region(REGION_SIZE);
-            if (currentRegion.compareAndSet(null, region))
-            {
-                // we won race - now we need to actually do the expensive allocation step
-                region.init();
-                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.
-        }
-    }
-
-    /**
-     * @return a lower bound on how much space has been allocated
-     */
-    public long getMinimumSize()
-    {
-        return unslabbed.get() + (regionCount.get() - 1) * (long)REGION_SIZE;
-    }
-
-    /**
-     * 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 ByteBuffer data;
-
-        private static final int UNINITIALIZED = -1;
-        /**
-         * Offset for the next allocation, or the sentinel value -1
-         * which implies that the region is still uninitialized.
-         */
-        private AtomicInteger nextFreeOffset = new AtomicInteger(UNINITIALIZED);
-
-        /**
-         * Total number of allocations satisfied from this buffer
-         */
-        private AtomicInteger allocCount = new AtomicInteger();
-
-        /**
-         * Size of region in bytes
-         */
-        private final int size;
-
-        /**
-         * Create an uninitialized region. Note that memory is not allocated yet, so
-         * this is cheap.
-         *
-         * @param size in bytes
-         */
-        private Region(int size)
-        {
-            this.size = size;
-        }
-
-        /**
-         * Actually claim the memory for this region. This should only be called from
-         * the thread that constructed the region. It is thread-safe against other
-         * threads calling alloc(), who will block until the allocation is complete.
-         */
-        public void init()
-        {
-            assert nextFreeOffset.get() == UNINITIALIZED;
-            data = ByteBuffer.allocate(size);
-            assert data.remaining() == data.capacity();
-            // Mark that it's ready for use
-            boolean initted = nextFreeOffset.compareAndSet(UNINITIALIZED, 0);
-            // We should always succeed the above CAS since only one thread calls init()!
-            Preconditions.checkState(initted, "Multiple threads tried to init same region");
-        }
-
-        /**
-         * Try to allocate <code>size</code> bytes from the region.
-         *
-         * @return the successful allocation, or null to indicate not-enough-space
-         */
-        public ByteBuffer allocate(int size)
-        {
-            while (true)
-            {
-                int oldOffset = nextFreeOffset.get();
-                if (oldOffset == UNINITIALIZED)
-                {
-                    // The region doesn't have its data allocated yet.
-                    // Since we found this in currentRegion, we know that whoever
-                    // CAS-ed it there is allocating it right now. So spin-loop
-                    // shouldn't spin long!
-                    Thread.yield();
-                    continue;
-                }
-
-                if (oldOffset + size > data.capacity()) // capacity == remaining
-                    return null;
-
-                // Try to atomically claim this region
-                if (nextFreeOffset.compareAndSet(oldOffset, oldOffset + size))
-                {
-                    // we got the alloc
-                    allocCount.incrementAndGet();
-                    return (ByteBuffer) data.duplicate().position(oldOffset).limit(oldOffset + size);
-                }
-                // we raced and lost alloc, try again
-            }
-        }
-
-        @Override
-        public String toString()
-        {
-            return "Region@" + System.identityHashCode(this) +
-                   " allocs=" + allocCount.get() + "waste=" +
-                   (data.capacity() - nextFreeOffset.get());
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/src/java/org/apache/cassandra/utils/WaitQueue.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/WaitQueue.java b/src/java/org/apache/cassandra/utils/WaitQueue.java
deleted file mode 100644
index 01b2559..0000000
--- a/src/java/org/apache/cassandra/utils/WaitQueue.java
+++ /dev/null
@@ -1,264 +0,0 @@
-package org.apache.cassandra.utils;
-
-import java.util.Iterator;
-import java.util.concurrent.ConcurrentLinkedQueue;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
-import java.util.concurrent.locks.AbstractQueuedSynchronizer;
-import java.util.concurrent.locks.LockSupport;
-
-/**
- * <p>A relatively easy to use utility for general purpose thread signalling.</p>
- * <p>Usage on a thread awaiting a state change using a WaitQueue q is:</p>
- * <pre>
- * {@code
- *      while (!conditionMet())
- *          WaitSignal s = q.register();
- *              if (!conditionMet())    // or, perhaps more correctly, !conditionChanged()
- *                  s.await();
- *              else
- *                  s.cancel();
- * }
- * </pre>
- * A signalling thread, AFTER changing the state, then calls q.signal() to wake up one, or q.signalAll()
- * to wake up all, waiting threads.
- *
- * <p>A few notes on utilisation:</p>
- * <p>1. A thread will only exit await() when it has been signalled, but this does
- * not guarantee the condition has not been altered since it was signalled,
- * and depending on your design it is likely the outer condition will need to be
- * checked in a loop, though this is not always the case.</p>
- * <p>2. Each signal is single use, so must be re-registered after each await(). This is true even if it times out.</p>
- * <p>3. If you choose not to wait on the signal (because the condition has been met before you waited on it)
- * you must cancel() the signal if the signalling thread uses signal() to awake waiters; otherwise signals will be
- * lost</p>
- * <p>4. Care must be taken when selecting conditionMet() to ensure we are waiting on the condition that actually
- * indicates progress is possible. In some complex cases it may be tempting to wait on a condition that is only indicative
- * of local progress, not progress on the task we are aiming to complete, and a race may leave us waiting for a condition
- * to be met that we no longer need.
- * <p>5. This scheme is not fair</p>
- * <p>6. Only the thread that calls register() may call await()</p>
- * <p>To understand intuitively how this class works, the idea is simply that a thread, once it considers itself
- * incapable of making progress, registers itself to be awoken once that condition changes. However, that condition
- * could have changed between checking and registering (in which case a thread updating the state would have been unable to signal it),
- * so before going to sleep on the signal, it checks the condition again, sleeping only if it hasn't changed.</p>
- */
-// TODO : switch to a Lock Free queue
-public final class WaitQueue
-{
-    public final class Signal
-    {
-        private final Thread thread = Thread.currentThread();
-        volatile int signalled;
-
-        private boolean isSignalled()
-        {
-            return signalled == 1;
-        }
-
-        public boolean isCancelled()
-        {
-            return signalled == -1;
-        }
-
-        private boolean signal()
-        {
-            if (signalledUpdater.compareAndSet(this, 0, 1))
-            {
-                LockSupport.unpark(thread);
-                return true;
-            }
-            return false;
-        }
-
-        public void awaitUninterruptibly()
-        {
-            assert !isCancelled();
-            if (thread != Thread.currentThread())
-                throw new IllegalStateException();
-            boolean interrupted = false;
-            while (!isSignalled())
-            {
-                if (Thread.interrupted())
-                    interrupted = true;
-                LockSupport.park();
-            }
-            if (interrupted)
-                thread.interrupt();
-        }
-
-        public void await() throws InterruptedException
-        {
-            assert !isCancelled();
-            while (!isSignalled())
-            {
-                if (Thread.interrupted())
-                {
-                    checkAndClear();
-                    throw new InterruptedException();
-                }
-                if (thread != Thread.currentThread())
-                    throw new IllegalStateException();
-                LockSupport.park();
-            }
-        }
-
-        public long awaitNanos(long nanosTimeout) throws InterruptedException
-        {
-            assert signalled != -1;
-            long start = System.nanoTime();
-            while (!isSignalled())
-            {
-                if (Thread.interrupted())
-                {
-                    checkAndClear();
-                    throw new InterruptedException();
-                }
-                LockSupport.parkNanos(nanosTimeout);
-            }
-            return nanosTimeout - (System.nanoTime() - start);
-        }
-
-        public boolean await(long time, TimeUnit unit) throws InterruptedException
-        {
-            // ignores nanos atm
-            long until = System.currentTimeMillis() + unit.toMillis(time);
-            if (until < 0)
-                until = Long.MAX_VALUE;
-            return awaitUntil(until);
-        }
-
-        public boolean awaitUntil(long until) throws InterruptedException
-        {
-            assert !isCancelled();
-            while (until < System.currentTimeMillis() && !isSignalled())
-            {
-                if (Thread.interrupted())
-                {
-                    checkAndClear();
-                    throw new InterruptedException();
-                }
-                LockSupport.parkUntil(until);
-            }
-            return checkAndClear();
-        }
-
-        private boolean checkAndClear()
-        {
-            if (isSignalled())
-            {
-                signalled = -1;
-                return true;
-            }
-            else if (signalledUpdater.compareAndSet(this, 0, -1))
-            {
-                cleanUpCancelled();
-                return false;
-            }
-            else
-            {
-                // must now be signalled, as checkAndClear() can only be called by
-                // owning thread if used correctly
-                signalled = -1;
-                return true;
-            }
-        }
-
-        public void cancel()
-        {
-            if (signalled < 0)
-                return;
-            if (!signalledUpdater.compareAndSet(this, 0, -1))
-            {
-                signalled = -1;
-                signal();
-                cleanUpCancelled();
-            }
-        }
-
-    }
-
-    private static final AtomicIntegerFieldUpdater signalledUpdater = AtomicIntegerFieldUpdater.newUpdater(Signal.class, "signalled");
-
-    // the waiting signals
-    private final ConcurrentLinkedQueue<Signal> queue = new ConcurrentLinkedQueue<>();
-
-    /**
-     * The calling thread MUST be the thread that uses the signal (for now)
-     * @return
-     */
-    public Signal register()
-    {
-        Signal signal = new Signal();
-        queue.add(signal);
-        return signal;
-    }
-
-    /**
-     * Signal one waiting thread
-     */
-    public void signal()
-    {
-        if (queue.isEmpty())
-            return;
-        Iterator<Signal> iter = queue.iterator();
-        while (iter.hasNext())
-        {
-            Signal next = iter.next();
-            if (next.signal())
-            {
-                iter.remove();
-                return;
-            }
-        }
-    }
-
-    /**
-     * Signal all waiting threads
-     */
-    public void signalAll()
-    {
-        if (queue.isEmpty())
-            return;
-        Iterator<Signal> iter = queue.iterator();
-        while (iter.hasNext())
-        {
-            Signal next = iter.next();
-            if (next.signal())
-                iter.remove();
-        }
-    }
-
-    private void cleanUpCancelled()
-    {
-        Iterator<Signal> iter = queue.iterator();
-        while (iter.hasNext())
-        {
-            Signal next = iter.next();
-            if (next.isCancelled())
-                iter.remove();
-        }
-    }
-
-    /**
-     * Return how many threads are waiting
-     * @return
-     */
-    public int getWaiting()
-    {
-        if (queue.isEmpty())
-            return 0;
-        Iterator<Signal> iter = queue.iterator();
-        int count = 0;
-        while (iter.hasNext())
-        {
-            Signal next = iter.next();
-            if (next.isCancelled())
-                iter.remove();
-            else
-                count++;
-        }
-        return count;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/src/java/org/apache/cassandra/utils/btree/BTree.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/btree/BTree.java b/src/java/org/apache/cassandra/utils/btree/BTree.java
index 1721fb0..eb7e558 100644
--- a/src/java/org/apache/cassandra/utils/btree/BTree.java
+++ b/src/java/org/apache/cassandra/utils/btree/BTree.java
@@ -5,6 +5,7 @@ import java.util.Collection;
 import java.util.Comparator;
 
 import com.google.common.base.Function;
+import org.apache.cassandra.utils.ObjectSizes;
 import com.google.common.collect.Collections2;
 
 public class BTree
@@ -73,7 +74,7 @@ public class BTree
      * @param <V>
      * @return
      */
-    public static <V> Object[] build(Collection<V> source, Comparator<V> comparator, boolean sorted)
+    public static <V> Object[] build(Collection<V> source, Comparator<V> comparator, boolean sorted, UpdateFunction<V> updateF)
     {
         int size = source.size();
 
@@ -84,6 +85,12 @@ public class BTree
             // inline sorting since we're already calling toArray
             if (!sorted)
                 Arrays.sort(values, 0, size, comparator);
+            if (updateF != null)
+            {
+                for (int i = 0 ; i < size ; i++)
+                    values[i] = updateF.apply(values[i]);
+                updateF.allocated(ObjectSizes.sizeOfArray(values));
+            }
             return values;
         }
 
@@ -105,7 +112,7 @@ public class BTree
      */
     public static <V> Object[] update(Object[] btree, Comparator<V> comparator, Collection<V> updateWith, boolean updateWithIsSorted)
     {
-        return update(btree, comparator, updateWith, updateWithIsSorted, null, null);
+        return update(btree, comparator, updateWith, updateWithIsSorted, null);
     }
 
     /**
@@ -115,9 +122,7 @@ public class BTree
      * @param comparator         the comparator that defines the ordering over the items in the tree
      * @param updateWith         the items to either insert / update
      * @param updateWithIsSorted if false, updateWith will be copied and sorted to facilitate construction
-     * @param replaceF           a function to apply to a pair we are swapping
-     * @param terminateEarly     a function that returns Boolean.TRUE if we should terminate before finishing our work.
-     *                           the argument to terminateEarly is ignored.
+     * @param updateF            the update function to apply to any pairs we are swapping, and maybe abort early
      * @param <V>
      * @return
      */
@@ -125,15 +130,10 @@ public class BTree
                                       Comparator<V> comparator,
                                       Collection<V> updateWith,
                                       boolean updateWithIsSorted,
-                                      ReplaceFunction<V> replaceF,
-                                      Function<?, Boolean> terminateEarly)
+                                      UpdateFunction<V> updateF)
     {
         if (btree.length == 0)
-        {
-            if (replaceF != null)
-                updateWith = Collections2.transform(updateWith, replaceF);
-            return build(updateWith, comparator, updateWithIsSorted);
-        }
+            return build(updateWith, comparator, updateWithIsSorted, updateF);
 
         if (!updateWithIsSorted)
             updateWith = sorted(updateWith, comparator, updateWith.size());
@@ -167,13 +167,13 @@ public class BTree
                 {
                     // apply replaceF if it matches an existing element
                     btreeOffset++;
-                    if (replaceF != null)
-                        v = replaceF.apply((V) btree[i], v);
+                    if (updateF != null)
+                        v = updateF.apply((V) btree[i], v);
                 }
-                else if (replaceF != null)
+                else if (updateF != null)
                 {
                     // new element but still need to apply replaceF to handle indexing and size-tracking
-                    v = replaceF.apply(v);
+                    v = updateF.apply(v);
                 }
 
                 merged[mergedCount++] = v;
@@ -187,19 +187,15 @@ public class BTree
                 mergedCount += count;
             }
 
-            if (mergedCount > FAN_FACTOR)
-            {
-                // TODO this code will never execute since QUICK_MERGE_LIMIT == FAN_FACTOR
-                int mid = (mergedCount >> 1) & ~1; // divide by two, rounding down to an even number
-                return new Object[] { merged[mid],
-                                      Arrays.copyOfRange(merged, 0, mid),
-                                      Arrays.copyOfRange(merged, 1 + mid, mergedCount + ((mergedCount + 1) & 1)), };
-            }
+            assert mergedCount <= FAN_FACTOR;
 
-            return Arrays.copyOfRange(merged, 0, mergedCount + (mergedCount & 1));
+            Object[] r = Arrays.copyOfRange(merged, 0, mergedCount + (mergedCount & 1));
+            if (updateF != null)
+                updateF.allocated(ObjectSizes.sizeOfArray(r) - (btree.length == 0 ? 0 : ObjectSizes.sizeOfArray(btree)));
+            return r;
         }
 
-        return modifier.get().update(btree, comparator, updateWith, replaceF, terminateEarly);
+        return modifier.get().update(btree, comparator, updateWith, updateF);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/src/java/org/apache/cassandra/utils/btree/BTreeSet.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/btree/BTreeSet.java b/src/java/org/apache/cassandra/utils/btree/BTreeSet.java
index 1730cfc..4452663 100644
--- a/src/java/org/apache/cassandra/utils/btree/BTreeSet.java
+++ b/src/java/org/apache/cassandra/utils/btree/BTreeSet.java
@@ -20,7 +20,7 @@ public class BTreeSet<V> implements NavigableSet<V>
 
     public BTreeSet<V> update(Collection<V> updateWith, boolean isSorted)
     {
-        return new BTreeSet<>(BTree.update(tree, comparator, updateWith, isSorted, null, null), comparator);
+        return new BTreeSet<>(BTree.update(tree, comparator, updateWith, isSorted, null), comparator);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/src/java/org/apache/cassandra/utils/btree/Builder.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/btree/Builder.java b/src/java/org/apache/cassandra/utils/btree/Builder.java
index 0865146..5ba923b 100644
--- a/src/java/org/apache/cassandra/utils/btree/Builder.java
+++ b/src/java/org/apache/cassandra/utils/btree/Builder.java
@@ -3,9 +3,6 @@ package org.apache.cassandra.utils.btree;
 import java.util.Collection;
 import java.util.Comparator;
 
-import com.google.common.base.Function;
-
-import static org.apache.cassandra.utils.btree.BTree.EMPTY_BRANCH;
 import static org.apache.cassandra.utils.btree.BTree.EMPTY_LEAF;
 import static org.apache.cassandra.utils.btree.BTree.FAN_SHIFT;
 import static org.apache.cassandra.utils.btree.BTree.POSITIVE_INFINITY;
@@ -40,21 +37,21 @@ final class Builder
      * we assume @param source has been sorted, e.g. by BTree.update, so the update of each key resumes where
      * the previous left off.
      */
-    public <V> Object[] update(Object[] btree, Comparator<V> comparator, Collection<V> source, ReplaceFunction<V> replaceF, Function<?, Boolean> terminateEarly)
+    public <V> Object[] update(Object[] btree, Comparator<V> comparator, Collection<V> source, UpdateFunction<V> updateF)
     {
         NodeBuilder current = rootBuilder;
-        current.reset(btree, POSITIVE_INFINITY);
+        current.reset(btree, POSITIVE_INFINITY, updateF, comparator);
 
         for (V key : source)
         {
             while (true)
             {
-                if (terminateEarly != null && terminateEarly.apply(null) == Boolean.TRUE)
+                if (updateF != null && updateF.abortEarly())
                 {
                     rootBuilder.clear();
                     return null;
                 }
-                NodeBuilder next = current.update(key, comparator, replaceF);
+                NodeBuilder next = current.update(key);
                 if (next == null)
                     break;
                 // we were in a subtree from a previous key that didn't contain this new key;
@@ -66,7 +63,7 @@ final class Builder
         // finish copying any remaining keys from the original btree
         while (true)
         {
-            NodeBuilder next = current.update(POSITIVE_INFINITY, comparator, replaceF);
+            NodeBuilder next = current.update(POSITIVE_INFINITY);
             if (next == null)
                 break;
             current = next;
@@ -88,9 +85,9 @@ final class Builder
         while ((size >>= FAN_SHIFT) > 0)
             current = current.ensureChild();
 
-        current.reset(EMPTY_LEAF, POSITIVE_INFINITY);
+        current.reset(EMPTY_LEAF, POSITIVE_INFINITY, null, null);
         for (V key : source)
-            current.addNewKey(key, null);
+            current.addNewKey(key);
 
         current = current.ascendToRoot();
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/src/java/org/apache/cassandra/utils/btree/NodeBuilder.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/btree/NodeBuilder.java b/src/java/org/apache/cassandra/utils/btree/NodeBuilder.java
index e526394..5a47f35 100644
--- a/src/java/org/apache/cassandra/utils/btree/NodeBuilder.java
+++ b/src/java/org/apache/cassandra/utils/btree/NodeBuilder.java
@@ -1,5 +1,7 @@
 package org.apache.cassandra.utils.btree;
 
+import org.apache.cassandra.utils.ObjectSizes;
+
 import java.util.Arrays;
 import java.util.Comparator;
 
@@ -38,6 +40,9 @@ final class NodeBuilder
     // the index of the first child node in copyFrom that has not yet been copied into the build arrays
     private int copyFromChildPosition;
 
+    private UpdateFunction updateFunction;
+    private Comparator comparator;
+
     // upper bound of range owned by this level; lets us know if we need to ascend back up the tree
     // for the next key we update when bsearch gives an insertion point past the end of the values
     // in the current node
@@ -51,7 +56,7 @@ final class NodeBuilder
         {
             if (current.upperBound != null)
             {
-                current.reset(null, null);
+                current.reset(null, null, null, null);
                 Arrays.fill(current.buildKeys, 0, current.maxBuildKeyPosition, null);
                 Arrays.fill(current.buildChildren, 0, current.maxBuildChildPosition, null);
                 current.maxBuildChildPosition = current.maxBuildKeyPosition = 0;
@@ -61,10 +66,12 @@ final class NodeBuilder
     }
 
     // reset counters/setup to copy from provided node
-    void reset(Object[] copyFrom, Object upperBound)
+    void reset(Object[] copyFrom, Object upperBound, UpdateFunction updateFunction, Comparator comparator)
     {
         this.copyFrom = copyFrom;
         this.upperBound = upperBound;
+        this.updateFunction = updateFunction;
+        this.comparator = comparator;
         maxBuildKeyPosition = Math.max(maxBuildKeyPosition, buildKeyPosition);
         maxBuildChildPosition = Math.max(maxBuildChildPosition, buildChildPosition);
         buildKeyPosition = 0;
@@ -81,7 +88,7 @@ final class NodeBuilder
      * a parent if we do not -- we got here from an earlier key -- and we need to ascend back up),
      * or null if we finished the update in this node.
      */
-    <V> NodeBuilder update(Object key, Comparator<V> comparator, ReplaceFunction<V> replaceF)
+    <V> NodeBuilder update(Object key)
     {
         assert copyFrom != null;
         int copyFromKeyEnd = getKeyEnd(copyFrom);
@@ -104,9 +111,9 @@ final class NodeBuilder
             if (owns)
             {
                 if (found)
-                    replaceNextKey(key, replaceF);
+                    replaceNextKey(key);
                 else
-                    addNewKey(key, replaceF); // handles splitting parent if necessary via ensureRoom
+                    addNewKey(key); // handles splitting parent if necessary via ensureRoom
 
                 // done, so return null
                 return null;
@@ -122,7 +129,7 @@ final class NodeBuilder
             if (found)
             {
                 copyKeys(i);
-                replaceNextKey(key, replaceF);
+                replaceNextKey(key);
                 copyChildren(i + 1);
                 return null;
             }
@@ -135,7 +142,7 @@ final class NodeBuilder
                 // so descend into the owning child
                 Object newUpperBound = i < copyFromKeyEnd ? copyFrom[i] : upperBound;
                 Object[] descendInto = (Object[]) copyFrom[copyFromKeyEnd + i];
-                ensureChild().reset(descendInto, newUpperBound);
+                ensureChild().reset(descendInto, newUpperBound, updateFunction, comparator);
                 return child;
             }
             else
@@ -177,7 +184,7 @@ final class NodeBuilder
     Object[] toNode()
     {
         assert buildKeyPosition <= FAN_FACTOR && buildKeyPosition > 0 : buildKeyPosition;
-        return buildFromRange(0, buildKeyPosition, isLeaf(copyFrom));
+        return buildFromRange(0, buildKeyPosition, isLeaf(copyFrom), false);
     }
 
     // finish up this level and pass any constructed children up to our parent, ensuring a parent exists
@@ -189,12 +196,12 @@ final class NodeBuilder
         {
             // split current node and move the midpoint into parent, with the two halves as children
             int mid = buildKeyPosition / 2;
-            parent.addExtraChild(buildFromRange(0, mid, isLeaf), buildKeys[mid]);
-            parent.finishChild(buildFromRange(mid + 1, buildKeyPosition - (mid + 1), isLeaf));
+            parent.addExtraChild(buildFromRange(0, mid, isLeaf, true), buildKeys[mid]);
+            parent.finishChild(buildFromRange(mid + 1, buildKeyPosition - (mid + 1), isLeaf, false));
         }
         else
         {
-            parent.finishChild(buildFromRange(0, buildKeyPosition, isLeaf));
+            parent.finishChild(buildFromRange(0, buildKeyPosition, isLeaf, false));
         }
         return parent;
     }
@@ -215,23 +222,23 @@ final class NodeBuilder
     }
 
     // skips the next key in copyf, and puts the provided key in the builder instead
-    private <V> void replaceNextKey(Object with, ReplaceFunction<V> replaceF)
+    private <V> void replaceNextKey(Object with)
     {
         // (this first part differs from addNewKey in that we pass the replaced object to replaceF as well)
         ensureRoom(buildKeyPosition + 1);
-        if (replaceF != null)
-            with = replaceF.apply((V) copyFrom[copyFromKeyPosition], (V) with);
+        if (updateFunction != null)
+            with = updateFunction.apply((V) copyFrom[copyFromKeyPosition], (V) with);
         buildKeys[buildKeyPosition++] = with;
 
         copyFromKeyPosition++;
     }
 
     // puts the provided key in the builder, with no impact on treatment of data from copyf
-    <V> void addNewKey(Object key, ReplaceFunction<V> replaceF)
+    <V> void addNewKey(Object key)
     {
         ensureRoom(buildKeyPosition + 1);
-        if (replaceF != null)
-            key = replaceF.apply((V) key);
+        if (updateFunction != null)
+            key = updateFunction.apply((V) key);
         buildKeys[buildKeyPosition++] = key;
     }
 
@@ -269,7 +276,7 @@ final class NodeBuilder
             return;
 
         // flush even number of items so we don't waste leaf space repeatedly
-        Object[] flushUp = buildFromRange(0, FAN_FACTOR, isLeaf(copyFrom));
+        Object[] flushUp = buildFromRange(0, FAN_FACTOR, isLeaf(copyFrom), true);
         ensureParent().addExtraChild(flushUp, buildKeys[FAN_FACTOR]);
         int size = FAN_FACTOR + 1;
         assert size <= buildKeyPosition : buildKeyPosition + "," + nextBuildKeyPosition;
@@ -285,7 +292,7 @@ final class NodeBuilder
     }
 
     // builds and returns a node from the buffered objects in the given range
-    private Object[] buildFromRange(int offset, int keyLength, boolean isLeaf)
+    private Object[] buildFromRange(int offset, int keyLength, boolean isLeaf, boolean isExtra)
     {
         Object[] a;
         if (isLeaf)
@@ -299,6 +306,14 @@ final class NodeBuilder
             System.arraycopy(buildKeys, offset, a, 0, keyLength);
             System.arraycopy(buildChildren, offset, a, keyLength, keyLength + 1);
         }
+        if (updateFunction != null)
+        {
+            if (isExtra)
+                updateFunction.allocated(ObjectSizes.sizeOfArray(a));
+            else if (a.length != copyFrom.length)
+                updateFunction.allocated(ObjectSizes.sizeOfArray(a) -
+                        (copyFrom.length == 0 ? 0 : ObjectSizes.sizeOfArray(copyFrom)));
+        }
         return a;
     }
 
@@ -313,7 +328,7 @@ final class NodeBuilder
             parent.child = this;
         }
         if (parent.upperBound == null)
-            parent.reset(EMPTY_BRANCH, upperBound);
+            parent.reset(EMPTY_BRANCH, upperBound, updateFunction, comparator);
         return parent;
     }
 


[5/6] remove Table.switchlock and introduce o.a.c.utils.memory package patch by Benedict Elliott Smith; reviewed by jbellis for CASSANDRA-5549

Posted by jb...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/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 94af5c0..540f1ce 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@ -24,7 +24,6 @@ import java.nio.ByteBuffer;
 import java.util.*;
 import java.util.concurrent.*;
 import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.locks.Lock;
 import java.util.regex.Pattern;
 import javax.management.*;
@@ -32,6 +31,14 @@ import javax.management.*;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Function;
 import com.google.common.collect.*;
+import com.google.common.util.concurrent.*;
+import org.apache.cassandra.concurrent.NamedThreadFactory;
+import org.apache.cassandra.utils.concurrent.OpOrder;
+import org.apache.cassandra.concurrent.StageManager;
+import org.apache.cassandra.db.filter.ColumnSlice;
+import org.apache.cassandra.db.filter.SliceQueryFilter;
+import org.apache.cassandra.utils.memory.HeapAllocator;
+
 import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.Striped;
 import com.google.common.util.concurrent.Uninterruptibles;
@@ -46,8 +53,10 @@ import org.apache.cassandra.db.columniterator.OnDiskAtomIterator;
 import org.apache.cassandra.db.commitlog.CommitLog;
 import org.apache.cassandra.db.commitlog.ReplayPosition;
 import org.apache.cassandra.db.compaction.*;
+import org.apache.cassandra.db.filter.ExtendedFilter;
+import org.apache.cassandra.db.filter.IDiskAtomFilter;
+import org.apache.cassandra.db.filter.QueryFilter;
 import org.apache.cassandra.db.composites.CellName;
-import org.apache.cassandra.db.filter.*;
 import org.apache.cassandra.db.composites.CellNameType;
 import org.apache.cassandra.db.composites.Composite;
 import org.apache.cassandra.db.index.SecondaryIndex;
@@ -74,7 +83,19 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
 {
     private static final Logger logger = LoggerFactory.getLogger(ColumnFamilyStore.class);
 
-    public static final ExecutorService postFlushExecutor = new JMXEnabledThreadPoolExecutor("MemtablePostFlusher");
+    private static final ExecutorService flushExecutor = new JMXEnabledThreadPoolExecutor(DatabaseDescriptor.getFlushWriters(),
+                                                                                          StageManager.KEEPALIVE,
+                                                                                          TimeUnit.SECONDS,
+                                                                                          new LinkedBlockingQueue<Runnable>(),
+                                                                                          new NamedThreadFactory("MemtableFlushWriter"),
+                                                                                          "internal");
+    // post-flush executor is single threaded to provide guarantee that any flush Future on a CF will never return until prior flushes have completed
+    public static final ExecutorService postFlushExecutor = new JMXEnabledThreadPoolExecutor(1,
+                                                                                             StageManager.KEEPALIVE,
+                                                                                             TimeUnit.SECONDS,
+                                                                                             new LinkedBlockingQueue<Runnable>(),
+                                                                                             new NamedThreadFactory("MemtablePostFlush"),
+                                                                                             "internal");
 
     public final Keyspace keyspace;
     public final String name;
@@ -83,7 +104,14 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
     private final String mbeanName;
     private volatile boolean valid = true;
 
-    /* Memtables and SSTables on disk for this column family */
+    /**
+     * Memtables and SSTables on disk for this column family.
+     *
+     * We synchronize on the DataTracker to ensure isolation when we want to make sure
+     * that the memtable we're acting on doesn't change out from under us.  I.e., flush
+     * syncronizes on it to make sure it can submit on both executors atomically,
+     * so anyone else who wants to make sure flush doesn't interfere should as well.
+     */
     private final DataTracker data;
 
     /* This is used to generate the next index for a SSTable */
@@ -98,11 +126,6 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
 
     public final Directories directories;
 
-    /** ratio of in-memory memtable size, to serialized size */
-    volatile double liveRatio = 10.0; // reasonable default until we compute what it is based on actual data
-    /** ops count last time we computed liveRatio */
-    private final AtomicLong liveRatioComputedAt = new AtomicLong(32);
-
     public final ColumnFamilyMetrics metric;
     public volatile long sampleLatencyNanos;
 
@@ -128,8 +151,8 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
 
         // If the CF comparator has changed, we need to change the memtable,
         // because the old one still aliases the previous comparator.
-        if (getMemtableThreadSafe().initialComparator != metadata.comparator)
-            switchMemtable(true, true);
+        if (data.getView().getCurrentMemtable().initialComparator != metadata.comparator)
+            switchMemtable();
     }
 
     private void maybeReloadCompactionStrategy()
@@ -158,14 +181,23 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
             {
                 protected void runMayThrow() throws Exception
                 {
-                    if (getMemtableThreadSafe().isExpired())
+                    synchronized (data)
                     {
-                        // if memtable is already expired but didn't flush because it's empty,
-                        // then schedule another flush.
-                        if (isClean())
-                            scheduleFlush();
-                        else
-                            forceFlush(); // scheduleFlush() will be called by the constructor of the new memtable.
+                        Memtable current = data.getView().getCurrentMemtable();
+                        // if we're not expired, we've been hit by a scheduled flush for an already flushed memtable, so ignore
+                        if (current.isExpired())
+                        {
+                            if (current.isClean())
+                            {
+                                // if we're still clean, instead of swapping just reschedule a flush for later
+                                scheduleFlush();
+                            }
+                            else
+                            {
+                                // we'll be rescheduled by the constructor of the Memtable.
+                                forceFlush();
+                            }
+                        }
                     }
                 }
             };
@@ -737,133 +769,283 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
     }
 
     /**
-     * Switch and flush the current memtable, if it was dirty. The forceSwitch
-     * flag allow to force switching the memtable even if it is clean (though
-     * in that case we don't flush, as there is no point).
+     * Switches the memtable iff the live memtable is the one provided
+     *
+     * @param memtable
      */
-    public Future<?> switchMemtable(final boolean writeCommitLog, boolean forceSwitch)
-    {
-        /*
-         * If we can get the writelock, that means no new updates can come in and
-         * all ongoing updates to memtables have completed. We can get the tail
-         * of the log and use it as the starting position for log replay on recovery.
-         *
-         * This is why we Keyspace.switchLock needs to be global instead of per-Keyspace:
-         * we need to schedule discardCompletedSegments calls in the same order as their
-         * contexts (commitlog position) were read, even though the flush executor
-         * is multithreaded.
-         */
-        Keyspace.switchLock.writeLock().lock();
-        try
+    public Future<?> switchMemtableIfCurrent(Memtable memtable)
+    {
+        synchronized (data)
         {
-            final Future<ReplayPosition> ctx = writeCommitLog ? CommitLog.instance.getContext() : Futures.immediateFuture(ReplayPosition.NONE);
+            if (data.getView().getCurrentMemtable() == memtable)
+                return switchMemtable();
+        }
+        return Futures.immediateFuture(null);
+    }
 
-            // submit the memtable for any indexed sub-cfses, and our own.
-            final List<ColumnFamilyStore> icc = new ArrayList<ColumnFamilyStore>();
-            // don't assume that this.memtable is dirty; forceFlush can bring us here during index build even if it is not
+    /*
+     * switchMemtable puts Memtable.getSortedContents on the writer executor.  When the write is complete,
+     * we turn the writer into an SSTableReader and add it to ssTables where it is available for reads.
+     * This method does not block except for synchronizing on DataTracker, but the Future it returns will
+     * not complete until the Memtable (and all prior Memtables) have been successfully flushed, and the CL
+     * marked clean up to the position owned by the Memtable.
+     */
+    public ListenableFuture<?> switchMemtable()
+    {
+        logger.info("Enqueuing flush of {}", name);
+        synchronized (data)
+        {
+            Flush flush = new Flush(false);
+            flushExecutor.execute(flush);
+            ListenableFutureTask<?> task = ListenableFutureTask.create(flush.postFlush, null);
+            postFlushExecutor.submit(task);
+            return task;
+        }
+    }
+
+    public ListenableFuture<?> forceFlush()
+    {
+        return forceFlush(null);
+    }
+
+    /**
+     * Flush if there is unflushed data that was written to the CommitLog before @param flushIfDirtyBefore
+     * (inclusive).  If @param flushIfDirtyBefore is null, flush if there is any unflushed data.
+     *
+     * @return a Future such that when the future completes, all data inserted before forceFlush was called,
+     * will be flushed.
+     */
+    public ListenableFuture<?> forceFlush(ReplayPosition flushIfDirtyBefore)
+    {
+        // we synchronize on the data tracker to ensure we don't race against other calls to switchMemtable(),
+        // unnecessarily queueing memtables that are about to be made clean
+        synchronized (data)
+        {
+            // during index build, 2ary index memtables can be dirty even if parent is not.  if so,
+            // we want to flush the 2ary index ones too.
+            boolean clean = true;
             for (ColumnFamilyStore cfs : concatWithIndexes())
+                clean &= cfs.data.getView().getCurrentMemtable().isCleanAfter(flushIfDirtyBefore);
+
+            if (clean)
             {
-                if (forceSwitch || !cfs.getMemtableThreadSafe().isClean())
-                    icc.add(cfs);
+                // We could have a memtable for this column family that is being
+                // flushed. Make sure the future returned wait for that so callers can
+                // assume that any data inserted prior to the call are fully flushed
+                // when the future returns (see #5241).
+                ListenableFutureTask<?> task = ListenableFutureTask.create(new Runnable()
+                {
+                    public void run()
+                    {
+                        logger.debug("forceFlush requested but everything is clean in {}", name);
+                    }
+                }, null);
+                postFlushExecutor.execute(task);
+                return task;
             }
 
-            final CountDownLatch latch = new CountDownLatch(icc.size());
-            for (ColumnFamilyStore cfs : icc)
+            return switchMemtable();
+        }
+    }
+
+    public void forceBlockingFlush()
+    {
+        FBUtilities.waitOnFuture(forceFlush());
+    }
+
+    /**
+     * Both synchronises custom secondary indexes and provides ordering guarantees for futures on switchMemtable/flush
+     * etc, which expect to be able to wait until the flush (and all prior flushes) requested have completed.
+     */
+    private final class PostFlush implements Runnable
+    {
+        final boolean flushSecondaryIndexes;
+        final OpOrder.Barrier writeBarrier;
+        final CountDownLatch latch = new CountDownLatch(1);
+        volatile ReplayPosition lastReplayPosition;
+
+        private PostFlush(boolean flushSecondaryIndexes, OpOrder.Barrier writeBarrier)
+        {
+            this.writeBarrier = writeBarrier;
+            this.flushSecondaryIndexes = flushSecondaryIndexes;
+        }
+
+        public void run()
+        {
+            writeBarrier.await();
+
+            /**
+             * we can flush 2is as soon as the barrier completes, as they will be consistent with (or ahead of) the
+             * flushed memtables and CL position, which is as good as we can guarantee.
+             * TODO: SecondaryIndex should support setBarrier(), so custom implementations can co-ordinate exactly
+             * with CL as we do with memtables/CFS-backed SecondaryIndexes.
+             */
+
+            if (flushSecondaryIndexes)
             {
-                Memtable memtable = cfs.data.switchMemtable();
-                // With forceSwitch it's possible to get a clean memtable here.
-                // In that case, since we've switched it already, just remove
-                // it from the memtable pending flush right away.
-                if (memtable.isClean())
+                for (SecondaryIndex index : indexManager.getIndexesNotBackedByCfs())
                 {
-                    cfs.replaceFlushed(memtable, null);
-                    latch.countDown();
-                }
-                else
-                {
-                    logger.info("Enqueuing flush of {}", memtable);
-                    memtable.flushAndSignal(latch, ctx);
+                    // flush any non-cfs backed indexes
+                    logger.info("Flushing SecondaryIndex {}", index);
+                    index.forceBlockingFlush();
                 }
             }
 
-            if (metric.memtableSwitchCount.count() == Long.MAX_VALUE)
-                metric.memtableSwitchCount.clear();
-            metric.memtableSwitchCount.inc();
+            try
+            {
+                // we wait on the latch for the lastReplayPosition to be set, and so that waiters
+                // on this task can rely on all prior flushes being complete
+                latch.await();
+            }
+            catch (InterruptedException e)
+            {
+                throw new IllegalStateException();
+            }
 
-            // when all the memtables have been written, including for indexes, mark the flush in the commitlog header.
-            // a second executor makes sure the onMemtableFlushes get called in the right order,
-            // while keeping the wait-for-flush (future.get) out of anything latency-sensitive.
-            return postFlushExecutor.submit(new WrappedRunnable()
+            // must check lastReplayPosition != null because Flush may find that all memtables are clean
+            // and so not set a lastReplayPosition
+            if (lastReplayPosition != null)
             {
-                public void runMayThrow() throws InterruptedException, ExecutionException
-                {
-                    latch.await();
+                CommitLog.instance.discardCompletedSegments(metadata.cfId, lastReplayPosition);
+            }
 
-                    if (!icc.isEmpty())
-                    {
-                        //only valid when memtables exist
+            metric.pendingFlushes.dec();
+        }
+    }
 
-                        for (SecondaryIndex index : indexManager.getIndexesNotBackedByCfs())
-                        {
-                            // flush any non-cfs backed indexes
-                            logger.info("Flushing SecondaryIndex {}", index);
-                            index.forceBlockingFlush();
-                        }
-                    }
+    /**
+     * Should only be constructed/used from switchMemtable() or truncate(), with ownership of the DataTracker monitor.
+     * In the constructor the current memtable(s) are swapped, and a barrer on outstanding writes is issued;
+     * when run by the flushWriter the barrier is waited on to ensure all outstanding writes have completed
+     * before all memtables are immediately written, and the CL is either immediately marked clean or, if
+     * there are custom secondary indexes, the post flush clean up is left to update those indexes and mark
+     * the CL clean
+     */
+    private final class Flush implements Runnable
+    {
+        final OpOrder.Barrier writeBarrier;
+        final List<Memtable> memtables;
+        final PostFlush postFlush;
+        final boolean truncate;
+
+        private Flush(boolean truncate)
+        {
+            // if true, we won't flush, we'll just wait for any outstanding writes, switch the memtable, and discard
+            this.truncate = truncate;
+
+            metric.pendingFlushes.inc();
+            /**
+             * To ensure correctness of switch without blocking writes, run() needs to wait for all write operations
+             * started prior to the switch to complete. We do this by creating a Barrier on the writeOrdering
+             * that all write operations register themselves with, and assigning this barrier to the memtables,
+             * after which we *.issue()* the barrier. This barrier is used to direct write operations started prior
+             * to the barrier.issue() into the memtable we have switched out, and any started after to its replacement.
+             * In doing so it also tells the write operations to update the lastReplayPosition of the memtable, so
+             * that we know the CL position we are dirty to, which can be marked clean when we complete.
+             */
+            writeBarrier = keyspace.writeOrder.newBarrier();
+            memtables = new ArrayList<>();
+
+            // submit flushes for the memtable for any indexed sub-cfses, and our own
+            final ReplayPosition minReplayPosition = CommitLog.instance.getContext();
+            for (ColumnFamilyStore cfs : concatWithIndexes())
+            {
+                // switch all memtables, regardless of their dirty status, setting the barrier
+                // so that we can reach a coordinated decision about cleanliness once they
+                // are no longer possible to be modified
+                Memtable mt = cfs.data.switchMemtable(truncate);
+                mt.setDiscarding(writeBarrier, minReplayPosition);
+                memtables.add(mt);
+            }
 
-                    if (writeCommitLog)
-                    {
-                        // if we're not writing to the commit log, we are replaying the log, so marking
-                        // the log header with "you can discard anything written before the context" is not valid
-                        CommitLog.instance.discardCompletedSegments(metadata.cfId, ctx.get());
-                    }
-                }
-            });
+            writeBarrier.issue();
+            postFlush = new PostFlush(!truncate, writeBarrier);
         }
-        finally
+
+        public void run()
         {
-            Keyspace.switchLock.writeLock().unlock();
-        }
-    }
+            // mark writes older than the barrier as blocking progress, permitting them to exceed our memory limit
+            // if they are stuck waiting on it, then wait for them all to complete
+            writeBarrier.markBlocking();
+            writeBarrier.await();
 
-    private boolean isClean()
-    {
-        // during index build, 2ary index memtables can be dirty even if parent is not.  if so,
-        // we want flushLargestMemtables to flush the 2ary index ones too.
-        for (ColumnFamilyStore cfs : concatWithIndexes())
-            if (!cfs.getMemtableThreadSafe().isClean())
-                return false;
+            // mark all memtables as flushing, removing them from the live memtable list, and
+            // remove any memtables that are already clean from the set we need to flush
+            Iterator<Memtable> iter = memtables.iterator();
+            while (iter.hasNext())
+            {
+                Memtable memtable = iter.next();
+                memtable.cfs.data.markFlushing(memtable);
+                if (memtable.isClean() || truncate)
+                {
+                    memtable.cfs.replaceFlushed(memtable, null);
+                    memtable.setDiscarded();
+                    iter.remove();
+                }
+            }
 
-        return true;
+            if (memtables.isEmpty())
+            {
+                postFlush.latch.countDown();
+                return;
+            }
+
+            metric.memtableSwitchCount.inc();
+
+            for (final Memtable memtable : memtables)
+            {
+                // flush the memtable
+                MoreExecutors.sameThreadExecutor().execute(memtable.flushRunnable());
+                memtable.setDiscarded();
+            }
+
+            // signal the post-flush we've done our work
+            postFlush.lastReplayPosition = memtables.get(0).getLastReplayPosition();
+            postFlush.latch.countDown();
+        }
     }
 
     /**
-     * @return a future, with a guarantee that any data inserted prior to the forceFlush() call is fully flushed
-     *         by the time future.get() returns. Never returns null.
+     * Finds the largest memtable, as a percentage of *either* on- or off-heap memory limits, and immediately
+     * queues it for flushing. If the memtable selected is flushed before this completes, no work is done.
      */
-    public Future<?> forceFlush()
+    public static class FlushLargestColumnFamily implements Runnable
     {
-        if (isClean())
+        public void run()
         {
-            // We could have a memtable for this column family that is being
-            // flushed. Make sure the future returned wait for that so callers can
-            // assume that any data inserted prior to the call are fully flushed
-            // when the future returns (see #5241).
-            return postFlushExecutor.submit(new Runnable()
+            float largestRatio = 0f;
+            Memtable largest = null;
+            for (ColumnFamilyStore cfs : ColumnFamilyStore.all())
             {
-                public void run()
+                // we take a reference to the current main memtable for the CF prior to snapping its ownership ratios
+                // to ensure we have some ordering guarantee for performing the switchMemtableIf(), i.e. we will only
+                // swap if the memtables we are measuring here haven't already been swapped by the time we try to swap them
+                Memtable current = cfs.getDataTracker().getView().getCurrentMemtable();
+
+                // find the total ownership ratio for the memtable and all SecondaryIndexes owned by this CF,
+                // both on- and off-heap, and select the largest of the two ratios to weight this CF
+                float onHeap = 0f;
+                onHeap += current.getAllocator().ownershipRatio();
+
+                for (SecondaryIndex index : cfs.indexManager.getIndexes())
                 {
-                    logger.debug("forceFlush requested but everything is clean in {}", name);
+                    if (index.getOnHeapAllocator() != null)
+                        onHeap += index.getOnHeapAllocator().ownershipRatio();
                 }
-            });
-        }
 
-        return switchMemtable(true, false);
-    }
+                if (onHeap > largestRatio)
+                {
+                    largest = current;
+                    largestRatio = onHeap;
+                }
+            }
 
-    public void forceBlockingFlush()
-    {
-        FBUtilities.waitOnFuture(forceFlush());
+            if (largest != null)
+            {
+                largest.cfs.switchMemtableIfCurrent(largest);
+                logger.info("Reclaiming {} of {} retained memtable bytes", largest.getAllocator().reclaiming(), Memtable.memoryPool.used());
+            }
+        }
     }
 
     public void maybeUpdateRowCache(DecoratedKey key)
@@ -882,28 +1064,14 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
      * param @ key - key for update/insert
      * param @ columnFamily - columnFamily changes
      */
-    public void apply(DecoratedKey key, ColumnFamily columnFamily, SecondaryIndexManager.Updater indexer)
+    public void apply(DecoratedKey key, ColumnFamily columnFamily, SecondaryIndexManager.Updater indexer, OpOrder.Group opGroup, ReplayPosition replayPosition)
     {
         long start = System.nanoTime();
 
-        Memtable mt = getMemtableThreadSafe();
-        mt.put(key, columnFamily, indexer);
+        Memtable mt = data.getMemtableFor(opGroup);
+        mt.put(key, columnFamily, indexer, opGroup, replayPosition);
         maybeUpdateRowCache(key);
         metric.writeLatency.addNano(System.nanoTime() - start);
-
-        // recompute liveRatio, if we have doubled the number of ops since last calculated
-        while (true)
-        {
-            long last = liveRatioComputedAt.get();
-            long operations = metric.writeLatency.latency.count();
-            if (operations < 2 * last)
-                break;
-            if (liveRatioComputedAt.compareAndSet(last, operations))
-            {
-                logger.debug("computing liveRatio of {} at {} ops", this, operations);
-                mt.updateLiveRatio();
-            }
-        }
     }
 
     /**
@@ -1149,35 +1317,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
 
     public long getMemtableDataSize()
     {
-        return metric.memtableDataSize.value();
-    }
-
-    public long getTotalMemtableLiveSize()
-    {
-        return getMemtableDataSize() + indexManager.getTotalLiveSize();
-    }
-
-    /**
-     * @return the live size of all the memtables (the current active one and pending flush).
-     */
-    public long getAllMemtablesLiveSize()
-    {
-        long size = 0;
-        for (Memtable mt : getDataTracker().getAllMemtables())
-            size += mt.getLiveSize();
-        return size;
-    }
-
-    /**
-     * @return the size of all the memtables, including the pending flush ones and 2i memtables, if any.
-     */
-    public long getTotalAllMemtablesLiveSize()
-    {
-        long size = getAllMemtablesLiveSize();
-        if (indexManager.hasIndexes())
-            for (ColumnFamilyStore index : indexManager.getIndexesBackedByCfs())
-                size += index.getAllMemtablesLiveSize();
-        return size;
+        return metric.memtableHeapSize.value();
     }
 
     public int getMemtableSwitchCount()
@@ -1185,11 +1325,6 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         return (int) metric.memtableSwitchCount.count();
     }
 
-    Memtable getMemtableThreadSafe()
-    {
-        return data.getMemtable();
-    }
-
     /**
      * Package protected for access from the CompactionManager.
      */
@@ -1245,7 +1380,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
 
     public int getPendingTasks()
     {
-        return metric.pendingTasks.value();
+        return (int) metric.pendingFlushes.count();
     }
 
     public long getWriteCount()
@@ -1472,7 +1607,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
             // retry w/ new view
         }
 
-        return new ViewFragment(sstables, Iterables.concat(Collections.singleton(view.memtable), view.memtablesPendingFlush));
+        return new ViewFragment(sstables, view.getAllMemtables());
     }
 
     /**
@@ -2049,22 +2184,16 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
             // that was part of the flushed we forced; otherwise on a tie, it won't get deleted.
             Uninterruptibles.sleepUninterruptibly(1, TimeUnit.MILLISECONDS);
         }
-
-        // nuke the memtable data w/o writing to disk first
-        Keyspace.switchLock.writeLock().lock();
-        try
+        else
         {
-            for (ColumnFamilyStore cfs : concatWithIndexes())
+            // just nuke the memtable data w/o writing to disk first
+            synchronized (data)
             {
-                Memtable mt = cfs.getMemtableThreadSafe();
-                if (!mt.isClean())
-                    mt.cfs.data.renewMemtable();
+                final Flush flush = new Flush(true);
+                flushExecutor.execute(flush);
+                postFlushExecutor.submit(flush.postFlush);
             }
         }
-        finally
-        {
-            Keyspace.switchLock.writeLock().unlock();
-        }
 
         Runnable truncateRunnable = new Runnable()
         {
@@ -2337,12 +2466,9 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
 
     public Iterable<ColumnFamilyStore> concatWithIndexes()
     {
-        return Iterables.concat(indexManager.getIndexesBackedByCfs(), Collections.singleton(this));
-    }
-
-    public Set<Memtable> getMemtablesPendingFlush()
-    {
-        return data.getMemtablesPendingFlush();
+        // we return the main CFS first, which we rely on for simplicity in switchMemtable(), for getting the
+        // latest replay position
+        return Iterables.concat(Collections.singleton(this), indexManager.getIndexesBackedByCfs());
     }
 
     public List<String> getBuiltIndexes()
@@ -2381,17 +2507,13 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
      */
     public long oldestUnflushedMemtable()
     {
-        DataTracker.View view = data.getView();
-        long oldest = view.memtable.creationTime();
-        for (Memtable memtable : view.memtablesPendingFlush)
-            oldest = Math.min(oldest, memtable.creationTime());
-        return oldest;
+        return data.getView().getOldestMemtable().creationTime();
     }
 
     public boolean isEmpty()
     {
         DataTracker.View view = data.getView();
-        return view.sstables.isEmpty() && view.memtable.getOperations() == 0 && view.memtablesPendingFlush.isEmpty();
+        return view.sstables.isEmpty() && view.getCurrentMemtable().getOperations() == 0 && view.getCurrentMemtable() == view.getOldestMemtable();
     }
 
     private boolean isRowCacheEnabled()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/src/java/org/apache/cassandra/db/ColumnFamilyStoreMBean.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStoreMBean.java b/src/java/org/apache/cassandra/db/ColumnFamilyStoreMBean.java
index fc1a7b1..f248ccf 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStoreMBean.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStoreMBean.java
@@ -35,7 +35,7 @@ public interface ColumnFamilyStoreMBean
      * Returns the total amount of data stored in the memtable, including
      * column related overhead.
      *
-     * @see org.apache.cassandra.metrics.ColumnFamilyMetrics#memtableDataSize
+     * @see org.apache.cassandra.metrics.ColumnFamilyMetrics#memtableHeapSize
      * @return The size in bytes.
      * @deprecated
      */
@@ -146,7 +146,7 @@ public interface ColumnFamilyStoreMBean
     public double getRecentWriteLatencyMicros();
 
     /**
-     * @see org.apache.cassandra.metrics.ColumnFamilyMetrics#pendingTasks
+     * @see org.apache.cassandra.metrics.ColumnFamilyMetrics#pendingFlushes
      * @return the estimated number of tasks pending for this column family
      */
     @Deprecated

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/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 426d876..e097914 100644
--- a/src/java/org/apache/cassandra/db/CounterCell.java
+++ b/src/java/org/apache/cassandra/db/CounterCell.java
@@ -20,15 +20,15 @@ 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.utils.memory.HeapAllocator;
 
 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.serializers.MarshalException;
-import org.apache.cassandra.utils.Allocator;
-import org.apache.cassandra.utils.*;
 
 /**
  * A column that represents a partitioned counter.
@@ -142,7 +142,7 @@ public class CounterCell extends Cell
     }
 
     @Override
-    public Cell reconcile(Cell cell, Allocator allocator)
+    public Cell reconcile(Cell cell, AbstractAllocator allocator)
     {
         // live + tombstone: track last tombstone
         if (cell.isMarkedForDelete(Long.MIN_VALUE)) // cannot be an expired cell, so the current time is irrelevant
@@ -190,7 +190,7 @@ public class CounterCell extends Cell
     }
 
     @Override
-    public Cell localCopy(ColumnFamilyStore cfs, Allocator allocator)
+    public Cell localCopy(ColumnFamilyStore cfs, AbstractAllocator allocator)
     {
         return new CounterCell(name.copy(allocator), allocator.clone(value), timestamp, timestampOfLastDelete);
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/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 6884c80..5d96c70 100644
--- a/src/java/org/apache/cassandra/db/CounterMutation.java
+++ b/src/java/org/apache/cassandra/db/CounterMutation.java
@@ -33,6 +33,9 @@ import org.apache.cassandra.exceptions.WriteTimeoutException;
 import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.net.MessageOut;
 import org.apache.cassandra.net.MessagingService;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.memory.AbstractAllocator;
+import org.apache.cassandra.utils.memory.HeapAllocator;
 import org.apache.cassandra.service.CacheService;
 import org.apache.cassandra.tracing.Tracing;
 import org.apache.cassandra.utils.*;
@@ -134,7 +137,7 @@ public class CounterMutation implements IMutation
     // Replaces all the CounterUpdateCell-s with updated regular CounterCell-s
     private ColumnFamily processModifications(ColumnFamily changesCF)
     {
-        Allocator allocator = HeapAllocator.instance;
+        AbstractAllocator allocator = HeapAllocator.instance;
         ColumnFamilyStore cfs = Keyspace.open(getKeyspaceName()).getColumnFamilyStore(changesCF.id());
 
         ColumnFamily resultCF = changesCF.cloneMeShallow();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/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 f7a0ef1..b7ca2cb 100644
--- a/src/java/org/apache/cassandra/db/CounterUpdateCell.java
+++ b/src/java/org/apache/cassandra/db/CounterUpdateCell.java
@@ -20,9 +20,9 @@ package org.apache.cassandra.db;
 import java.nio.ByteBuffer;
 
 import org.apache.cassandra.db.composites.CellName;
-import org.apache.cassandra.db.composites.CellNameType;
-import org.apache.cassandra.utils.Allocator;
+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.
@@ -56,7 +56,7 @@ public class CounterUpdateCell extends Cell
     }
 
     @Override
-    public Cell reconcile(Cell cell, Allocator allocator)
+    public Cell reconcile(Cell cell, AbstractAllocator allocator)
     {
         // The only time this could happen is if a batchAdd ships two
         // increment for the same cell. Hence we simply sums the delta.
@@ -78,7 +78,7 @@ public class CounterUpdateCell extends Cell
     }
 
     @Override
-    public Cell localCopy(ColumnFamilyStore cfs, Allocator allocator)
+    public Cell localCopy(ColumnFamilyStore cfs, AbstractAllocator allocator)
     {
         throw new UnsupportedOperationException();
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/src/java/org/apache/cassandra/db/DataTracker.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/DataTracker.java b/src/java/org/apache/cassandra/db/DataTracker.java
index 8fad5dd..f2aae50 100644
--- a/src/java/org/apache/cassandra/db/DataTracker.java
+++ b/src/java/org/apache/cassandra/db/DataTracker.java
@@ -24,6 +24,9 @@ import java.util.concurrent.atomic.AtomicReference;
 
 import com.google.common.base.Predicate;
 import com.google.common.collect.*;
+
+import org.apache.cassandra.utils.concurrent.OpOrder;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -51,23 +54,22 @@ public class DataTracker
         this.init();
     }
 
-    public Memtable getMemtable()
-    {
-        return view.get().memtable;
-    }
-
-    public Set<Memtable> getMemtablesPendingFlush()
+    // get the Memtable that the ordered writeOp should be directed to
+    public Memtable getMemtableFor(OpOrder.Group opGroup)
     {
-        return view.get().memtablesPendingFlush;
-    }
+        // since any new memtables appended to the list after we fetch it will be for operations started
+        // after us, we can safely assume that we will always find the memtable that 'accepts' us;
+        // if the barrier for any memtable is set whilst we are reading the list, it must accept us.
 
-    /**
-     * @return the active memtable and all the memtables that are pending flush.
-     */
-    public Iterable<Memtable> getAllMemtables()
-    {
-        View snapshot = view.get();
-        return Iterables.concat(snapshot.memtablesPendingFlush, Collections.singleton(snapshot.memtable));
+        // there may be multiple memtables in the list that would 'accept' us, however we only ever choose
+        // the oldest such memtable, as accepts() only prevents us falling behind (i.e. ensures we don't
+        // assign operations to a memtable that was retired/queued before we started)
+        for (Memtable memtable : view.get().liveMemtables)
+        {
+            if (memtable.accepts(opGroup))
+                return memtable;
+        }
+        throw new AssertionError(view.get().liveMemtables.toString());
     }
 
     public Set<SSTableReader> getSSTables()
@@ -98,46 +100,41 @@ public class DataTracker
     }
 
     /**
-     * Switch the current memtable.
-     * This atomically adds the current memtable to the memtables pending
-     * flush and replace it with a fresh memtable.
+     * Switch the current memtable. This atomically appends a new memtable to the end of the list of active memtables,
+     * returning the previously last memtable. It leaves the previous Memtable in the list of live memtables until
+     * discarding(memtable) is called. These two methods must be synchronized/paired, i.e. m = switchMemtable
+     * must be followed by discarding(m), they cannot be interleaved.
      *
-     * @return the previous current memtable (the one added to the pending
-     * flush)
+     * @return the previously active memtable
      */
-    public Memtable switchMemtable()
+    public Memtable switchMemtable(boolean truncating)
     {
-        // atomically change the current memtable
         Memtable newMemtable = new Memtable(cfstore);
         Memtable toFlushMemtable;
         View currentView, newView;
         do
         {
             currentView = view.get();
-            toFlushMemtable = currentView.memtable;
+            toFlushMemtable = currentView.getCurrentMemtable();
             newView = currentView.switchMemtable(newMemtable);
         }
         while (!view.compareAndSet(currentView, newView));
 
+        if (truncating)
+            notifyRenewed(newMemtable);
+
         return toFlushMemtable;
     }
 
-    /**
-     * Renew the current memtable without putting the old one for a flush.
-     * Used when we flush but a memtable is clean (in which case we must
-     * change it because it was frozen).
-     */
-    public void renewMemtable()
+    public void markFlushing(Memtable memtable)
     {
-        Memtable newMemtable = new Memtable(cfstore);
         View currentView, newView;
         do
         {
             currentView = view.get();
-            newView = currentView.renewMemtable(newMemtable);
+            newView = currentView.markFlushing(memtable);
         }
         while (!view.compareAndSet(currentView, newView));
-        notifyRenewed(currentView.memtable);
     }
 
     public void replaceFlushed(Memtable memtable, SSTableReader sstable)
@@ -320,11 +317,12 @@ public class DataTracker
     /** (Re)initializes the tracker, purging all references. */
     void init()
     {
-        view.set(new View(new Memtable(cfstore),
-                          Collections.<Memtable>emptySet(),
-                          Collections.<SSTableReader>emptySet(),
-                          Collections.<SSTableReader>emptySet(),
-                          SSTableIntervalTree.empty()));
+        view.set(new View(
+                ImmutableList.of(new Memtable(cfstore)),
+                ImmutableList.<Memtable>of(),
+                Collections.<SSTableReader>emptySet(),
+                Collections.<SSTableReader>emptySet(),
+                SSTableIntervalTree.empty()));
     }
 
     /**
@@ -533,66 +531,128 @@ public class DataTracker
      * flush, the sstables for a column family, and the sstables that are active
      * in compaction (a subset of the sstables).
      */
-    static class View
-    {
-        public final Memtable memtable;
-        public final Set<Memtable> memtablesPendingFlush;
+    public static class View
+    {
+        /**
+         * ordinarily a list of size 1, but when preparing to flush will contain both the memtable we will flush
+         * and the new replacement memtable, until all outstanding write operations on the old table complete.
+         * The last item in the list is always the "current" memtable.
+         */
+        private final List<Memtable> liveMemtables;
+        /**
+         * contains all memtables that are no longer referenced for writing and are queued for / in the process of being
+         * flushed. In chronologically ascending order.
+         */
+        private final List<Memtable> flushingMemtables;
         public final Set<SSTableReader> compacting;
         public final Set<SSTableReader> sstables;
         public final SSTableIntervalTree intervalTree;
 
-        View(Memtable memtable, Set<Memtable> pendingFlush, Set<SSTableReader> sstables, Set<SSTableReader> compacting, SSTableIntervalTree intervalTree)
+        View(List<Memtable> liveMemtables, List<Memtable> flushingMemtables, Set<SSTableReader> sstables, Set<SSTableReader> compacting, SSTableIntervalTree intervalTree)
         {
-            this.memtable = memtable;
-            this.memtablesPendingFlush = pendingFlush;
+            this.liveMemtables = liveMemtables;
+            this.flushingMemtables = flushingMemtables;
             this.sstables = sstables;
             this.compacting = compacting;
             this.intervalTree = intervalTree;
         }
 
+        public Memtable getOldestMemtable()
+        {
+            if (!flushingMemtables.isEmpty())
+                return flushingMemtables.get(0);
+            return liveMemtables.get(0);
+        }
+
+        public Memtable getCurrentMemtable()
+        {
+            return liveMemtables.get(liveMemtables.size() - 1);
+        }
+
+        public Iterable<Memtable> getMemtablesPendingFlush()
+        {
+            if (liveMemtables.size() == 1)
+                return flushingMemtables;
+            return Iterables.concat(liveMemtables.subList(0, 1), flushingMemtables);
+        }
+
+        /**
+         * @return the active memtable and all the memtables that are pending flush.
+         */
+        public Iterable<Memtable> getAllMemtables()
+        {
+            return Iterables.concat(flushingMemtables, liveMemtables);
+        }
+
         public Sets.SetView<SSTableReader> nonCompactingSStables()
         {
             return Sets.difference(ImmutableSet.copyOf(sstables), compacting);
         }
 
-        public View switchMemtable(Memtable newMemtable)
+        View switchMemtable(Memtable newMemtable)
         {
-            Set<Memtable> newPending = ImmutableSet.<Memtable>builder().addAll(memtablesPendingFlush).add(memtable).build();
-            return new View(newMemtable, newPending, sstables, compacting, intervalTree);
+            List<Memtable> newLiveMemtables = ImmutableList.<Memtable>builder().addAll(liveMemtables).add(newMemtable).build();
+            return new View(newLiveMemtables, flushingMemtables, sstables, compacting, intervalTree);
         }
 
-        public View renewMemtable(Memtable newMemtable)
+        View markFlushing(Memtable toFlushMemtable)
         {
-            return new View(newMemtable, memtablesPendingFlush, sstables, compacting, intervalTree);
+            List<Memtable> live = liveMemtables, flushing = flushingMemtables;
+
+            // since we can have multiple flushes queued, we may occasionally race and start a flush out of order,
+            // so must locate it in the list to remove, rather than just removing from the beginning
+            int i = live.indexOf(toFlushMemtable);
+            assert i < live.size() - 1;
+            List<Memtable> newLive = ImmutableList.<Memtable>builder()
+                                                  .addAll(live.subList(0, i))
+                                                  .addAll(live.subList(i + 1, live.size()))
+                                                  .build();
+
+            // similarly, if we out-of-order markFlushing once, we may afterwards need to insert a memtable into the
+            // flushing list in a position other than the end, though this will be rare
+            i = flushing.size();
+            while (i > 0 && flushing.get(i - 1).creationTime() > toFlushMemtable.creationTime())
+                i--;
+            List<Memtable> newFlushing = ImmutableList.<Memtable>builder()
+                                                      .addAll(flushing.subList(0, i))
+                                                      .add(toFlushMemtable)
+                                                      .addAll(flushing.subList(i, flushing.size()))
+                                                      .build();
+
+            return new View(newLive, newFlushing, sstables, compacting, intervalTree);
         }
 
-        public View replaceFlushed(Memtable flushedMemtable, SSTableReader newSSTable)
+        View replaceFlushed(Memtable flushedMemtable, SSTableReader newSSTable)
         {
-            Set<Memtable> newPending = ImmutableSet.copyOf(Sets.difference(memtablesPendingFlush, Collections.singleton(flushedMemtable)));
+            int index = flushingMemtables.indexOf(flushedMemtable);
+            List<Memtable> newQueuedMemtables = ImmutableList.<Memtable>builder()
+                                                             .addAll(flushingMemtables.subList(0, index))
+                                                             .addAll(flushingMemtables.subList(index + 1, flushingMemtables.size()))
+                                                             .build();
             Set<SSTableReader> newSSTables = newSSTable == null
-                                           ? sstables
-                                           : newSSTables(newSSTable);
+                                             ? sstables
+                                             : newSSTables(newSSTable);
             SSTableIntervalTree intervalTree = buildIntervalTree(newSSTables);
-            return new View(memtable, newPending, newSSTables, compacting, intervalTree);
+            return new View(liveMemtables, newQueuedMemtables, newSSTables, compacting, intervalTree);
         }
 
-        public View replace(Collection<SSTableReader> oldSSTables, Iterable<SSTableReader> replacements)
+        View replace(Collection<SSTableReader> oldSSTables, Iterable<SSTableReader> replacements)
         {
             Set<SSTableReader> newSSTables = newSSTables(oldSSTables, replacements);
             SSTableIntervalTree intervalTree = buildIntervalTree(newSSTables);
-            return new View(memtable, memtablesPendingFlush, newSSTables, compacting, intervalTree);
+            return new View(liveMemtables, flushingMemtables, newSSTables, compacting, intervalTree);
         }
 
-        public View markCompacting(Collection<SSTableReader> tomark)
+        View markCompacting(Collection<SSTableReader> tomark)
         {
             Set<SSTableReader> compactingNew = ImmutableSet.<SSTableReader>builder().addAll(compacting).addAll(tomark).build();
-            return new View(memtable, memtablesPendingFlush, sstables, compactingNew, intervalTree);
+            return new View(liveMemtables, flushingMemtables, sstables, compactingNew, intervalTree);
         }
 
-        public View unmarkCompacting(Iterable<SSTableReader> tounmark)
+        View unmarkCompacting(Iterable<SSTableReader> tounmark)
         {
             Set<SSTableReader> compactingNew = ImmutableSet.copyOf(Sets.difference(compacting, ImmutableSet.copyOf(tounmark)));
-            return new View(memtable, memtablesPendingFlush, sstables, compactingNew, intervalTree);
+            return new View(liveMemtables, flushingMemtables, sstables, compactingNew, intervalTree);
         }
 
         private Set<SSTableReader> newSSTables(SSTableReader newSSTable)
@@ -621,7 +681,7 @@ public class DataTracker
         @Override
         public String toString()
         {
-            return String.format("View(pending_count=%d, sstables=%s, compacting=%s)", memtablesPendingFlush.size(), sstables, compacting);
+            return String.format("View(pending_count=%d, sstables=%s, compacting=%s)", liveMemtables.size() + flushingMemtables.size() - 1, sstables, compacting);
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/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 13d1358..64a6df3 100644
--- a/src/java/org/apache/cassandra/db/DeletedCell.java
+++ b/src/java/org/apache/cassandra/db/DeletedCell.java
@@ -25,8 +25,9 @@ import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.composites.CellName;
 import org.apache.cassandra.io.util.DataOutputBuffer;
 import org.apache.cassandra.serializers.MarshalException;
-import org.apache.cassandra.utils.Allocator;
+import org.apache.cassandra.utils.memory.AbstractAllocator;
 import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.memory.HeapAllocator;
 
 public class DeletedCell extends Cell
 {
@@ -89,7 +90,7 @@ public class DeletedCell extends Cell
     }
 
     @Override
-    public Cell reconcile(Cell cell, Allocator allocator)
+    public Cell reconcile(Cell cell, AbstractAllocator allocator)
     {
         if (cell instanceof DeletedCell)
             return super.reconcile(cell, allocator);
@@ -97,7 +98,7 @@ public class DeletedCell extends Cell
     }
 
     @Override
-    public Cell localCopy(ColumnFamilyStore cfs, Allocator allocator)
+    public Cell localCopy(ColumnFamilyStore cfs, AbstractAllocator allocator)
     {
         return new DeletedCell(name.copy(allocator), allocator.clone(value), timestamp);
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/src/java/org/apache/cassandra/db/DeletionInfo.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/DeletionInfo.java b/src/java/org/apache/cassandra/db/DeletionInfo.java
index 5c62132..fc452a1 100644
--- a/src/java/org/apache/cassandra/db/DeletionInfo.java
+++ b/src/java/org/apache/cassandra/db/DeletionInfo.java
@@ -25,16 +25,20 @@ import java.util.*;
 import com.google.common.base.Objects;
 import com.google.common.collect.Iterators;
 
+import org.apache.cassandra.cache.IMeasurableMemory;
 import org.apache.cassandra.db.composites.CType;
 import org.apache.cassandra.db.composites.Composite;
 import org.apache.cassandra.io.IVersionedSerializer;
+import org.apache.cassandra.utils.ObjectSizes;
 
 /**
  * A combination of a top-level (or row) tombstone and range tombstones describing the deletions
  * within a {@link ColumnFamily} (or row).
  */
-public class DeletionInfo
+public class DeletionInfo implements IMeasurableMemory
 {
+    private static final long EMPTY_SIZE = ObjectSizes.measure(new DeletionInfo(0, 0));
+
     /**
      * This represents a deletion of the entire row.  We can't represent this within the RangeTombstoneList, so it's
      * kept separately.  This also slightly optimizes the common case of a full row deletion.
@@ -317,6 +321,12 @@ public class DeletionInfo
         return Objects.hashCode(topLevel, ranges);
     }
 
+    @Override
+    public long unsharedHeapSize()
+    {
+        return EMPTY_SIZE + topLevel.unsharedHeapSize() + (ranges == null ? 0 : ranges.unsharedHeapSize());
+    }
+
     public static class Serializer implements IVersionedSerializer<DeletionInfo>
     {
         private final RangeTombstoneList.Serializer rtlSerializer;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/src/java/org/apache/cassandra/db/DeletionTime.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/DeletionTime.java b/src/java/org/apache/cassandra/db/DeletionTime.java
index c60b423..39db398 100644
--- a/src/java/org/apache/cassandra/db/DeletionTime.java
+++ b/src/java/org/apache/cassandra/db/DeletionTime.java
@@ -24,6 +24,7 @@ import java.io.IOException;
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Objects;
 
+import org.apache.cassandra.cache.IMeasurableMemory;
 import org.apache.cassandra.io.ISerializer;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.utils.ObjectSizes;
@@ -31,8 +32,10 @@ import org.apache.cassandra.utils.ObjectSizes;
 /**
  * A top-level (row) tombstone.
  */
-public class DeletionTime implements Comparable<DeletionTime>
+public class DeletionTime implements Comparable<DeletionTime>, IMeasurableMemory
 {
+    private static final long EMPTY_SIZE = ObjectSizes.measure(new DeletionTime(0, 0));
+
     /**
      * A special DeletionTime that signifies that there is no top-level (row) tombstone.
      */
@@ -105,10 +108,9 @@ public class DeletionTime implements Comparable<DeletionTime>
         return atom.maxTimestamp() <= markedForDeleteAt;
     }
 
-    public long memorySize()
+    public long unsharedHeapSize()
     {
-        long fields = TypeSizes.NATIVE.sizeof(markedForDeleteAt) + TypeSizes.NATIVE.sizeof(localDeletionTime);
-        return ObjectSizes.getFieldSize(fields);
+        return EMPTY_SIZE;
     }
 
     public static class Serializer implements ISerializer<DeletionTime>

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/src/java/org/apache/cassandra/db/EmptyColumns.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/EmptyColumns.java b/src/java/org/apache/cassandra/db/EmptyColumns.java
index 5498353..5021f39 100644
--- a/src/java/org/apache/cassandra/db/EmptyColumns.java
+++ b/src/java/org/apache/cassandra/db/EmptyColumns.java
@@ -28,7 +28,7 @@ import java.util.Iterator;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.composites.CellName;
 import org.apache.cassandra.db.filter.ColumnSlice;
-import org.apache.cassandra.utils.Allocator;
+import org.apache.cassandra.utils.memory.AbstractAllocator;
 
 import com.google.common.base.Function;
 import com.google.common.collect.Iterators;
@@ -63,12 +63,12 @@ public class EmptyColumns extends AbstractThreadUnsafeSortedColumns
         return factory;
     }
 
-    public void addColumn(Cell cell, Allocator allocator)
+    public void addColumn(Cell cell, AbstractAllocator allocator)
     {
         throw new UnsupportedOperationException();
     }
 
-    public void addAll(ColumnFamily cm, Allocator allocator, Function<Cell, Cell> transformation)
+    public void addAll(ColumnFamily cm, AbstractAllocator allocator, Function<Cell, Cell> transformation)
     {
         throw new UnsupportedOperationException();
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/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 a2f68da..6742630 100644
--- a/src/java/org/apache/cassandra/db/ExpiringCell.java
+++ b/src/java/org/apache/cassandra/db/ExpiringCell.java
@@ -26,7 +26,8 @@ import org.apache.cassandra.db.composites.CellName;
 import org.apache.cassandra.db.composites.CellNameType;
 import org.apache.cassandra.io.util.DataOutputBuffer;
 import org.apache.cassandra.serializers.MarshalException;
-import org.apache.cassandra.utils.Allocator;
+import org.apache.cassandra.utils.memory.AbstractAllocator;
+import org.apache.cassandra.utils.memory.HeapAllocator;
 
 /**
  * Alternative to Cell that have an expiring time.
@@ -132,7 +133,7 @@ public class ExpiringCell extends Cell
     }
 
     @Override
-    public Cell localCopy(ColumnFamilyStore cfs, Allocator allocator)
+    public Cell localCopy(ColumnFamilyStore cfs, AbstractAllocator allocator)
     {
         return new ExpiringCell(name.copy(allocator), allocator.clone(value), timestamp, timeToLive, localExpirationTime);
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/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 c380e45..fb0f273 100644
--- a/src/java/org/apache/cassandra/db/Keyspace.java
+++ b/src/java/org/apache/cassandra/db/Keyspace.java
@@ -23,10 +23,12 @@ import java.util.*;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.Future;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 import com.google.common.base.Function;
 import com.google.common.collect.Iterables;
+
+import org.apache.cassandra.utils.concurrent.OpOrder;
+import org.apache.cassandra.db.commitlog.ReplayPosition;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -54,14 +56,6 @@ public class Keyspace
 
     private static final Logger logger = LoggerFactory.getLogger(Keyspace.class);
 
-    /**
-     * accesses to CFS.memtable should acquire this for thread safety.
-     * CFS.maybeSwitchMemtable should aquire the writeLock; see that method for the full explanation.
-     * <p/>
-     * (Enabling fairness in the RRWL is observed to decrease throughput, so we leave it off.)
-     */
-    public static final ReentrantReadWriteLock switchLock = new ReentrantReadWriteLock();
-
     // It is possible to call Keyspace.open without a running daemon, so it makes sense to ensure
     // proper directories here as well as in CassandraDaemon.
     static
@@ -71,6 +65,7 @@ public class Keyspace
     }
 
     public final KSMetaData metadata;
+    public final OpOrder writeOrder = new OpOrder();
 
     /* ColumnFamilyStore per column family */
     private final ConcurrentMap<UUID, ColumnFamilyStore> columnFamilyStores = new ConcurrentHashMap<UUID, ColumnFamilyStore>();
@@ -343,15 +338,21 @@ public class Keyspace
      */
     public void apply(Mutation mutation, boolean writeCommitLog, boolean updateIndexes)
     {
-        // write the mutation to the commitlog and memtables
-        Tracing.trace("Acquiring switchLock read lock");
-        switchLock.readLock().lock();
+        final OpOrder.Group opGroup = writeOrder.start();
         try
         {
+            // write the mutation to the commitlog and memtables
+            final ReplayPosition replayPosition;
             if (writeCommitLog)
             {
                 Tracing.trace("Appending to commitlog");
-                CommitLog.instance.add(mutation);
+                replayPosition = CommitLog.instance.add(mutation);
+            }
+            else
+            {
+                // we don't need the replayposition, but grab one anyway so that it stays stack allocated.
+                // (the JVM will not stack allocate if the object may be null.)
+                replayPosition = CommitLog.instance.getContext();
             }
 
             DecoratedKey key = StorageService.getPartitioner().decorateKey(mutation.key());
@@ -365,12 +366,13 @@ public class Keyspace
                 }
 
                 Tracing.trace("Adding to {} memtable", cf.metadata().cfName);
-                cfs.apply(key, cf, updateIndexes ? cfs.indexManager.updaterFor(key, cf) : SecondaryIndexManager.nullUpdater);
+                SecondaryIndexManager.Updater updater = updateIndexes ? cfs.indexManager.updaterFor(key, opGroup) : SecondaryIndexManager.nullUpdater;
+                cfs.apply(key, cf, updater, opGroup, replayPosition);
             }
         }
         finally
         {
-            switchLock.readLock().unlock();
+            opGroup.finishOne();
         }
     }
 
@@ -389,11 +391,11 @@ public class Keyspace
         if (logger.isDebugEnabled())
             logger.debug("Indexing row {} ", cfs.metadata.getKeyValidator().getString(key.key));
 
-        Collection<SecondaryIndex> indexes = cfs.indexManager.getIndexesByNames(idxNames);
-
-        switchLock.readLock().lock();
+        final OpOrder.Group opGroup = cfs.keyspace.writeOrder.start();
         try
         {
+            Collection<SecondaryIndex> indexes = cfs.indexManager.getIndexesByNames(idxNames);
+
             Iterator<ColumnFamily> pager = QueryPagers.pageRowLocally(cfs, key.key, DEFAULT_PAGE_SIZE);
             while (pager.hasNext())
             {
@@ -404,12 +406,12 @@ public class Keyspace
                     if (cfs.indexManager.indexes(cell.name(), indexes))
                         cf2.addColumn(cell);
                 }
-                cfs.indexManager.indexRow(key.key, cf2);
+                cfs.indexManager.indexRow(key.key, cf2, opGroup);
             }
         }
         finally
         {
-            switchLock.readLock().unlock();
+            opGroup.finishOne();
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/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 3761826..49a3f92 100644
--- a/src/java/org/apache/cassandra/db/Memtable.java
+++ b/src/java/org/apache/cassandra/db/Memtable.java
@@ -21,75 +21,50 @@ import java.io.File;
 import java.util.*;
 import java.util.concurrent.*;
 import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
 
-import com.google.common.base.Function;
 import com.google.common.base.Throwables;
-import org.cliffc.high_scale_lib.NonBlockingHashSet;
+
+import org.apache.cassandra.db.commitlog.CommitLog;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.concurrent.OpOrder;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.index.SecondaryIndexManager;
+import org.apache.cassandra.dht.LongToken;
+import org.apache.cassandra.io.util.DiskAwareRunnable;
+
+import org.apache.cassandra.utils.memory.AbstractAllocator;
+import org.apache.cassandra.utils.memory.ContextAllocator;
+import org.apache.cassandra.utils.ObjectSizes;
+import org.apache.cassandra.utils.memory.Pool;
+import org.apache.cassandra.utils.memory.PoolAllocator;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import org.apache.cassandra.concurrent.JMXEnabledThreadPoolExecutor;
-import org.apache.cassandra.concurrent.NamedThreadFactory;
-import org.apache.cassandra.concurrent.StageManager;
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.db.composites.CellNameType;
 import org.apache.cassandra.db.commitlog.ReplayPosition;
-import org.apache.cassandra.db.index.SecondaryIndexManager;
+import org.apache.cassandra.db.composites.CellNameType;
 import org.apache.cassandra.io.sstable.SSTableReader;
 import org.apache.cassandra.io.sstable.SSTableWriter;
 import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
-import org.apache.cassandra.io.util.DiskAwareRunnable;
-import org.apache.cassandra.utils.Allocator;
-import org.github.jamm.MemoryMeter;
 
 public class Memtable
 {
     private static final Logger logger = LoggerFactory.getLogger(Memtable.class);
 
-    /*
-     * switchMemtable puts Memtable.getSortedContents on the writer executor.  When the write is complete,
-     * we turn the writer into an SSTableReader and add it to ssTables where it is available for reads.
-     *
-     * There are two other things that switchMemtable does.
-     * First, it puts the Memtable into memtablesPendingFlush, where it stays until the flush is complete
-     * and it's been added as an SSTableReader to ssTables_.  Second, it adds an entry to commitLogUpdater
-     * that waits for the flush to complete, then calls onMemtableFlush.  This allows multiple flushes
-     * to happen simultaneously on multicore systems, while still calling onMF in the correct order,
-     * which is necessary for replay in case of a restart since CommitLog assumes that when onMF is
-     * called, all data up to the given context has been persisted to SSTables.
-     */
-    private static final ExecutorService flushWriter
-            = new JMXEnabledThreadPoolExecutor(DatabaseDescriptor.getFlushWriters(),
-                                               StageManager.KEEPALIVE,
-                                               TimeUnit.SECONDS,
-                                               new LinkedBlockingQueue<Runnable>(DatabaseDescriptor.getFlushQueueSize()),
-                                               new NamedThreadFactory("FlushWriter"),
-                                               "internal");
-
-    // size in memory can never be less than serialized size
-    private static final double MIN_SANE_LIVE_RATIO = 1.0;
-    // max liveratio seen w/ 1-byte columns on a 64-bit jvm was 19. If it gets higher than 64 something is probably broken.
-    private static final double MAX_SANE_LIVE_RATIO = 64.0;
-
-    // We need to take steps to avoid retaining inactive membtables in memory, because counting is slow (can be
-    // minutes, for a large memtable and a busy server).  A strictly FIFO Memtable queue could keep memtables
-    // alive waiting for metering after they're flushed and would otherwise be GC'd.  Instead, the approach we take
-    // is to enqueue the CFS instead of the memtable, and to meter whatever the active memtable is when the executor
-    // starts to work on it.  We use a Set to make sure we don't enqueue redundant tasks for the same CFS.
-    private static final Set<ColumnFamilyStore> meteringInProgress = new NonBlockingHashSet<ColumnFamilyStore>();
-    private static final ExecutorService meterExecutor = new JMXEnabledThreadPoolExecutor(1,
-                                                                                          Integer.MAX_VALUE,
-                                                                                          TimeUnit.MILLISECONDS,
-                                                                                          new LinkedBlockingQueue<Runnable>(),
-                                                                                          new NamedThreadFactory("MemoryMeter"),
-                                                                                          "internal");
-    private final MemoryMeter meter;
-
-    volatile static ColumnFamilyStore activelyMeasuring;
-
-    private final AtomicLong currentSize = new AtomicLong(0);
+    static final Pool memoryPool = DatabaseDescriptor.getMemtableAllocatorPool();
+    private static final int ROW_OVERHEAD_HEAP_SIZE;
+
+    private final PoolAllocator allocator;
+    private final AtomicLong liveDataSize = new AtomicLong(0);
     private final AtomicLong currentOperations = new AtomicLong(0);
 
+    // the write barrier for directing writes to this memtable during a switch
+    private volatile OpOrder.Barrier writeBarrier;
+    // the last ReplayPosition owned by this Memtable; all ReplayPositions lower are owned by this or an earlier Memtable
+    private final AtomicReference<ReplayPosition> lastReplayPosition = new AtomicReference<>();
+    // the "first" ReplayPosition owned by this Memtable; this is inaccurate, and only used as a convenience to prevent CLSM flushing wantonly
+    private final ReplayPosition minReplayPosition = CommitLog.instance.getContext();
+
     // We index the memtable by RowPosition only for the purpose of being able
     // to select key range using Token.KeyBound. However put() ensures that we
     // actually only store DecoratedKey.
@@ -98,16 +73,6 @@ public class Memtable
     private final long creationTime = System.currentTimeMillis();
     private final long creationNano = System.nanoTime();
 
-    private final Allocator allocator = DatabaseDescriptor.getMemtableAllocator();
-    // We really only need one column by allocator but one by memtable is not a big waste and avoids needing allocators to know about CFS
-    private final Function<Cell, Cell> localCopyFunction = new Function<Cell, Cell>()
-    {
-        public Cell apply(Cell c)
-        {
-            return c.localCopy(cfs, allocator);
-        }
-    };
-
     // Record the comparator of the CFS at the creation of the memtable. This
     // is only used when a user update the CF comparator, to know if the
     // memtable was created with the new or old comparator.
@@ -116,31 +81,19 @@ public class Memtable
     public Memtable(ColumnFamilyStore cfs)
     {
         this.cfs = cfs;
+        this.allocator = memoryPool.newAllocator(cfs.keyspace.writeOrder);
         this.initialComparator = cfs.metadata.comparator;
         this.cfs.scheduleFlush();
-
-        Callable<Set<Object>> provider = new Callable<Set<Object>>()
-        {
-            public Set<Object> call() throws Exception
-            {
-                // avoid counting this once for each row
-                Set<Object> set = Collections.newSetFromMap(new IdentityHashMap<Object, Boolean>());
-                set.add(Memtable.this.cfs.metadata);
-                return set;
-            }
-        };
-        meter = new MemoryMeter().omitSharedBufferOverhead().withTrackerProvider(provider);
     }
 
-    public long getLiveSize()
+    public AbstractAllocator getAllocator()
     {
-        long estimatedSize = (long) (currentSize.get() * cfs.liveRatio);
-
-        // liveRatio is just an estimate; we can get a lower bound directly from the allocator
-        if (estimatedSize < allocator.getMinimumSize())
-            return allocator.getMinimumSize();
+        return allocator;
+    }
 
-        return estimatedSize;
+    public long getLiveDataSize()
+    {
+        return liveDataSize.get();
     }
 
     public long getOperations()
@@ -148,54 +101,109 @@ public class Memtable
         return currentOperations.get();
     }
 
-    /**
-     * Should only be called by ColumnFamilyStore.apply.  NOT a public API.
-     * (CFS handles locking to avoid submitting an op
-     *  to a flushing memtable.  Any other way is unsafe.)
-    */
-    void put(DecoratedKey key, ColumnFamily columnFamily, SecondaryIndexManager.Updater indexer)
+    void setDiscarding(OpOrder.Barrier writeBarrier, ReplayPosition minLastReplayPosition)
     {
-        resolve(key, columnFamily, indexer);
+        assert this.writeBarrier == null;
+        this.lastReplayPosition.set(minLastReplayPosition);
+        this.writeBarrier = writeBarrier;
+        allocator.setDiscarding();
     }
 
-    public void updateLiveRatio() throws RuntimeException
+    void setDiscarded()
     {
-        if (!MemoryMeter.isInitialized())
-        {
-            // hack for openjdk.  we log a warning about this in the startup script too.
-            logger.error("MemoryMeter uninitialized (jamm not specified as java agent); assuming liveRatio of {}.  "
-                         + " Usually this means cassandra-env.sh disabled jamm because you are using a buggy JRE; "
-                         + " upgrade to the Sun JRE instead", cfs.liveRatio);
-            return;
-        }
+        allocator.setDiscarded();
+    }
 
-        if (!meteringInProgress.add(cfs))
-        {
-            logger.debug("Metering already pending or active for {}; skipping liveRatio update", cfs);
-            return;
-        }
+    public boolean accepts(OpOrder.Group opGroup)
+    {
+        OpOrder.Barrier barrier = this.writeBarrier;
+        return barrier == null || barrier.isAfter(opGroup);
+    }
+
+    public boolean isLive()
+    {
+        return allocator.isLive();
+    }
+
+    public boolean isClean()
+    {
+        return rows.isEmpty();
+    }
 
-        meterExecutor.submit(new MeteringRunnable(cfs));
+    public boolean isCleanAfter(ReplayPosition position)
+    {
+        return isClean() || (position != null && minReplayPosition.compareTo(position) >= 0);
     }
 
-    private void resolve(DecoratedKey key, ColumnFamily cf, SecondaryIndexManager.Updater indexer)
+    /**
+     * @return true if this memtable is expired. Expiration time is determined by CF's memtable_flush_period_in_ms.
+     */
+    public boolean isExpired()
+    {
+        int period = cfs.metadata.getMemtableFlushPeriod();
+        return period > 0 && (System.nanoTime() - creationNano >= TimeUnit.MILLISECONDS.toNanos(period));
+    }
+
+    /**
+     * Should only be called by ColumnFamilyStore.apply via Keyspace.apply, which supplies the appropriate
+     * OpOrdering.
+     *
+     * replayPosition should only be null if this is a secondary index, in which case it is *expected* to be null
+     */
+    void put(DecoratedKey key, ColumnFamily cf, SecondaryIndexManager.Updater indexer, OpOrder.Group opGroup, ReplayPosition replayPosition)
     {
+        if (replayPosition != null && writeBarrier != null)
+        {
+            // if the writeBarrier is set, we want to maintain lastReplayPosition; this is an optimisation to avoid
+            // casing it for every write, but still ensure it is correct when writeBarrier.await() completes.
+            // we clone the replay position so that the object passed in does not "escape", permitting stack allocation
+            replayPosition = replayPosition.clone();
+            while (true)
+            {
+                ReplayPosition last = lastReplayPosition.get();
+                if (last.compareTo(replayPosition) >= 0)
+                    break;
+                if (lastReplayPosition.compareAndSet(last, replayPosition))
+                    break;
+            }
+        }
+
         AtomicBTreeColumns previous = rows.get(key);
 
         if (previous == null)
         {
             AtomicBTreeColumns empty = cf.cloneMeShallow(AtomicBTreeColumns.factory, false);
+            final DecoratedKey cloneKey = new DecoratedKey(key.token, allocator.clone(key.key, opGroup));
             // We'll add the columns later. This avoids wasting works if we get beaten in the putIfAbsent
-            previous = rows.putIfAbsent(new DecoratedKey(key.token, allocator.clone(key.key)), empty);
+            previous = rows.putIfAbsent(cloneKey, empty);
             if (previous == null)
+            {
                 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);
+            }
+            else
+            {
+                allocator.free(cloneKey.key);
+            }
         }
 
-        long sizeDelta = previous.addAllWithSizeDelta(cf, allocator, localCopyFunction, indexer);
-        currentSize.addAndGet(sizeDelta);
+        ContextAllocator contextAllocator = allocator.wrap(opGroup, cfs);
+        AtomicBTreeColumns.Delta delta = previous.addAllWithSizeDelta(cf, contextAllocator, contextAllocator, indexer, new AtomicBTreeColumns.Delta());
+        liveDataSize.addAndGet(delta.dataSize());
         currentOperations.addAndGet((cf.getColumnCount() == 0)
                                     ? cf.isMarkedForDelete() ? 1 : 0
                                     : cf.getColumnCount());
+
+        // 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
@@ -211,15 +219,15 @@ public class Memtable
         return builder.toString();
     }
 
-    public void flushAndSignal(final CountDownLatch latch, final Future<ReplayPosition> context)
+    public FlushRunnable flushRunnable()
     {
-        flushWriter.execute(new FlushRunnable(latch, context));
+        return new FlushRunnable(lastReplayPosition.get());
     }
 
     public String toString()
     {
-        return String.format("Memtable-%s@%s(%s/%s serialized/live bytes, %s ops)",
-                             cfs.name, hashCode(), currentSize, getLiveSize(), currentOperations);
+        return String.format("Memtable-%s@%s(%s serialized bytes, %s ops, %.0f%% of heap limit)",
+                             cfs.name, hashCode(), liveDataSize, currentOperations, 100 * allocator.ownershipRatio());
     }
 
     /**
@@ -254,26 +262,12 @@ public class Memtable
             public void remove()
             {
                 iter.remove();
-                currentSize.addAndGet(-currentEntry.getValue().dataSize());
+                liveDataSize.addAndGet(-currentEntry.getValue().dataSize());
                 currentEntry = null;
             }
         };
     }
 
-    public boolean isClean()
-    {
-        return rows.isEmpty();
-    }
-
-    /**
-     * @return true if this memtable is expired. Expiration time is determined by CF's memtable_flush_period_in_ms.
-     */
-    public boolean isExpired()
-    {
-        int period = cfs.metadata.getMemtableFlushPeriod();
-        return period > 0 && (System.nanoTime() - creationNano >= TimeUnit.MILLISECONDS.toNanos(period));
-    }
-
     public ColumnFamily getColumnFamily(DecoratedKey key)
     {
         return rows.get(key);
@@ -284,15 +278,18 @@ public class Memtable
         return creationTime;
     }
 
+    public ReplayPosition getLastReplayPosition()
+    {
+        return lastReplayPosition.get();
+    }
+
     class FlushRunnable extends DiskAwareRunnable
     {
-        private final CountDownLatch latch;
-        private final Future<ReplayPosition> context;
+        private final ReplayPosition context;
         private final long estimatedSize;
 
-        FlushRunnable(CountDownLatch latch, Future<ReplayPosition> context)
+        FlushRunnable(ReplayPosition context)
         {
-            this.latch = latch;
             this.context = context;
 
             long keySize = 0;
@@ -304,7 +301,7 @@ public class Memtable
             }
             estimatedSize = (long) ((keySize // index entries
                                     + keySize // keys in data file
-                                    + currentSize.get()) // data
+                                    + liveDataSize.get()) // data
                                     * 1.2); // bloom filter and row index overhead
         }
 
@@ -319,7 +316,6 @@ public class Memtable
 
             SSTableReader sstable = writeSortedContents(context, sstableDirectory);
             cfs.replaceFlushed(Memtable.this, sstable);
-            latch.countDown();
         }
 
         protected Directories getDirectories()
@@ -327,7 +323,7 @@ public class Memtable
             return cfs.directories;
         }
 
-        private SSTableReader writeSortedContents(Future<ReplayPosition> context, File sstableDirectory)
+        private SSTableReader writeSortedContents(ReplayPosition context, File sstableDirectory)
         throws ExecutionException, InterruptedException
         {
             logger.info("Writing {}", Memtable.this.toString());
@@ -361,15 +357,16 @@ public class Memtable
                 {
                     ssTable = writer.closeAndOpenReader();
                     logger.info(String.format("Completed flushing %s (%d bytes) for commitlog position %s",
-                                              ssTable.getFilename(), new File(ssTable.getFilename()).length(), context.get()));
+                                              ssTable.getFilename(), new File(ssTable.getFilename()).length(), context));
                 }
                 else
                 {
                     writer.abort();
                     ssTable = null;
                     logger.info("Completed flushing; nothing needed to be retained.  Commitlog position was {}",
-                                context.get());
+                                context);
                 }
+
                 return ssTable;
             }
             catch (Throwable e)
@@ -381,7 +378,7 @@ public class Memtable
 
         public SSTableWriter createFlushWriter(String filename) throws ExecutionException, InterruptedException
         {
-            MetadataCollector sstableMetadataCollector = new MetadataCollector(cfs.metadata.comparator).replayPosition(context.get());
+            MetadataCollector sstableMetadataCollector = new MetadataCollector(cfs.metadata.comparator).replayPosition(context);
             return new SSTableWriter(filename,
                                      rows.size(),
                                      cfs.metadata,
@@ -390,62 +387,19 @@ public class Memtable
         }
     }
 
-    private static class MeteringRunnable implements Runnable
+    static
     {
-        // we might need to wait in the meter queue for a while.  measure whichever memtable is active at that point,
-        // rather than keeping the original memtable referenced (and thus un-freeable) until this runs.
-        private final ColumnFamilyStore cfs;
-
-        public MeteringRunnable(ColumnFamilyStore cfs)
-        {
-            this.cfs = cfs;
-        }
-
-        public void run()
-        {
-            try
-            {
-                activelyMeasuring = cfs;
-                Memtable memtable = cfs.getMemtableThreadSafe();
-
-                long start = System.nanoTime();
-                // ConcurrentSkipListMap has cycles, so measureDeep will have to track a reference to EACH object it visits.
-                // So to reduce the memory overhead of doing a measurement, we break it up to row-at-a-time.
-                long deepSize = memtable.meter.measure(memtable.rows);
-                int objects = 0;
-                for (Map.Entry<RowPosition, AtomicBTreeColumns> entry : memtable.rows.entrySet())
-                {
-                    deepSize += memtable.meter.measureDeep(entry.getKey()) + memtable.meter.measureDeep(entry.getValue());
-                    objects += entry.getValue().getColumnCount();
-                }
-                double newRatio = (double) deepSize / memtable.currentSize.get();
-
-                if (newRatio < MIN_SANE_LIVE_RATIO)
-                {
-                    logger.debug("setting live ratio to minimum of {} instead of {}", MIN_SANE_LIVE_RATIO, newRatio);
-                    newRatio = MIN_SANE_LIVE_RATIO;
-                }
-                if (newRatio > MAX_SANE_LIVE_RATIO)
-                {
-                    logger.warn("setting live ratio to maximum of {} instead of {}", MAX_SANE_LIVE_RATIO, newRatio);
-                    newRatio = MAX_SANE_LIVE_RATIO;
-                }
-
-                // we want to be very conservative about our estimate, since the penalty for guessing low is OOM
-                // death.  thus, higher estimates are believed immediately; lower ones are averaged w/ the old
-                if (newRatio > cfs.liveRatio)
-                    cfs.liveRatio = newRatio;
-                else
-                    cfs.liveRatio = (cfs.liveRatio + newRatio) / 2.0;
-
-                logger.info("{} liveRatio is {} (just-counted was {}).  calculation took {}ms for {} cells",
-                            cfs, cfs.liveRatio, newRatio, TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - start), objects);
-            }
-            finally
-            {
-                activelyMeasuring = null;
-                meteringInProgress.remove(cfs);
-            }
-        }
+        // calculate row overhead
+        int rowOverhead;
+        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);
+        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;
+        ROW_OVERHEAD_HEAP_SIZE = rowOverhead;
     }
 }


[4/6] remove Table.switchlock and introduce o.a.c.utils.memory package patch by Benedict Elliott Smith; reviewed by jbellis for CASSANDRA-5549

Posted by jb...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/src/java/org/apache/cassandra/db/MeteredFlusher.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/MeteredFlusher.java b/src/java/org/apache/cassandra/db/MeteredFlusher.java
deleted file mode 100644
index f1a3ac9..0000000
--- a/src/java/org/apache/cassandra/db/MeteredFlusher.java
+++ /dev/null
@@ -1,131 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.db;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.List;
-
-import com.google.common.collect.Iterables;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.cassandra.config.DatabaseDescriptor;
-
-public class MeteredFlusher implements Runnable
-{
-    private static final Logger logger = LoggerFactory.getLogger(MeteredFlusher.class);
-
-    public void run()
-    {
-        long totalMemtableBytesAllowed = DatabaseDescriptor.getTotalMemtableSpaceInMB() * 1048576L;
-
-        // first, find how much memory non-active memtables are using
-        long flushingBytes = Memtable.activelyMeasuring == null
-                           ? 0
-                           : Memtable.activelyMeasuring.getMemtableThreadSafe().getLiveSize();
-        flushingBytes += countFlushingBytes();
-        if (flushingBytes > 0)
-            logger.debug("Currently flushing {} bytes of {} max", flushingBytes, totalMemtableBytesAllowed);
-
-        // next, flush CFs using more than 1 / (maximum number of memtables it could have in the pipeline)
-        // of the total size allotted.  Then, flush other CFs in order of size if necessary.
-        long liveBytes = 0;
-        try
-        {
-            long totalMemtableBytesUnused = totalMemtableBytesAllowed - flushingBytes;
-            for (ColumnFamilyStore cfs : ColumnFamilyStore.all())
-            {
-                long size = cfs.getTotalMemtableLiveSize();
-                int maxInFlight = (int) Math.ceil((double) (1 // live memtable
-                                                            + 1 // potentially a flushed memtable being counted by jamm
-                                                            + DatabaseDescriptor.getFlushWriters()
-                                                            + DatabaseDescriptor.getFlushQueueSize())
-                                                  / (1 + cfs.indexManager.getIndexesBackedByCfs().size()));
-                if (cfs.getCompactionStrategy().isAffectedByMeteredFlusher() && totalMemtableBytesUnused > 0 && size > totalMemtableBytesUnused / maxInFlight)
-                {
-                    logger.info("flushing high-traffic column family {} (estimated {} bytes)", cfs, size);
-                    cfs.forceFlush();
-                }
-                else
-                {
-                    liveBytes += size;
-                }
-            }
-
-            if (flushingBytes + liveBytes <= totalMemtableBytesAllowed)
-                return;
-
-            logger.info("estimated {} live and {} flushing bytes used by all memtables", liveBytes, flushingBytes);
-
-            // sort memtables by size
-            List<ColumnFamilyStore> sorted = new ArrayList<ColumnFamilyStore>();
-            Iterables.addAll(sorted, ColumnFamilyStore.all());
-            Collections.sort(sorted, new Comparator<ColumnFamilyStore>()
-            {
-                public int compare(ColumnFamilyStore o1, ColumnFamilyStore o2)
-                {
-                    long size1 = o1.getTotalMemtableLiveSize();
-                    long size2 = o2.getTotalMemtableLiveSize();
-                    if (size1 < size2)
-                        return -1;
-                    if (size1 > size2)
-                        return 1;
-                    return 0;
-                }
-            });
-
-            // flush largest first until we get below our threshold.
-            // although it looks like liveBytes + flushingBytes will stay a constant, it will not if flushes finish
-            // while we loop, which is especially likely to happen if the flush queue fills up (so further forceFlush calls block)
-            while (!sorted.isEmpty())
-            {
-                flushingBytes = countFlushingBytes();
-                if (liveBytes + flushingBytes <= totalMemtableBytesAllowed)
-                    break;
-
-                ColumnFamilyStore cfs = sorted.remove(sorted.size() - 1);
-                if (cfs.getCompactionStrategy().isAffectedByMeteredFlusher())
-                {
-                    long size = cfs.getTotalMemtableLiveSize();
-                    if (size == 0)
-                        break;
-                    logger.info("flushing {} to free up {} bytes", cfs, size);
-                    liveBytes -= size;
-                    cfs.forceFlush();
-                }
-            }
-        }
-        finally
-        {
-            logger.trace("memtable memory usage is {} bytes with {} live", liveBytes + flushingBytes, liveBytes);
-        }
-    }
-
-    private long countFlushingBytes()
-    {
-        long flushingBytes = 0;
-        for (ColumnFamilyStore cfs : ColumnFamilyStore.all())
-        {
-            for (Memtable memtable : cfs.getMemtablesPendingFlush())
-                flushingBytes += memtable.getLiveSize();
-        }
-        return flushingBytes;
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/src/java/org/apache/cassandra/db/RangeTombstoneList.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/RangeTombstoneList.java b/src/java/org/apache/cassandra/db/RangeTombstoneList.java
index 4199ea4..1158e20 100644
--- a/src/java/org/apache/cassandra/db/RangeTombstoneList.java
+++ b/src/java/org/apache/cassandra/db/RangeTombstoneList.java
@@ -26,11 +26,13 @@ import java.util.Iterator;
 
 import com.google.common.collect.AbstractIterator;
 
+import org.apache.cassandra.cache.IMeasurableMemory;
 import org.apache.cassandra.db.composites.CType;
 import org.apache.cassandra.db.composites.Composite;
 import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.net.MessagingService;
 
+import org.apache.cassandra.utils.ObjectSizes;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -50,10 +52,12 @@ import org.slf4j.LoggerFactory;
  * The only use of the local deletion time is to know when a given tombstone can
  * be purged, which will be done by the purge() method.
  */
-public class RangeTombstoneList implements Iterable<RangeTombstone>
+public class RangeTombstoneList implements Iterable<RangeTombstone>, IMeasurableMemory
 {
     private static final Logger logger = LoggerFactory.getLogger(RangeTombstoneList.class);
 
+    private static long EMPTY_SIZE = ObjectSizes.measure(new RangeTombstoneList(null, 0));
+
     private final Comparator<Composite> comparator;
 
     // Note: we don't want to use a List for the markedAts and delTimes to avoid boxing. We could
@@ -63,9 +67,10 @@ public class RangeTombstoneList implements Iterable<RangeTombstone>
     private long[] markedAts;
     private int[] delTimes;
 
+    private long boundaryHeapSize;
     private int size;
 
-    private RangeTombstoneList(Comparator<Composite> comparator, Composite[] starts, Composite[] ends, long[] markedAts, int[] delTimes, int size)
+    private RangeTombstoneList(Comparator<Composite> comparator, Composite[] starts, Composite[] ends, long[] markedAts, int[] delTimes, long boundaryHeapSize, int size)
     {
         assert starts.length == ends.length && starts.length == markedAts.length && starts.length == delTimes.length;
         this.comparator = comparator;
@@ -74,11 +79,12 @@ public class RangeTombstoneList implements Iterable<RangeTombstone>
         this.markedAts = markedAts;
         this.delTimes = delTimes;
         this.size = size;
+        this.boundaryHeapSize = boundaryHeapSize;
     }
 
     public RangeTombstoneList(Comparator<Composite> comparator, int capacity)
     {
-        this(comparator, new Composite[capacity], new Composite[capacity], new long[capacity], new int[capacity], 0);
+        this(comparator, new Composite[capacity], new Composite[capacity], new long[capacity], new int[capacity], 0, 0);
     }
 
     public boolean isEmpty()
@@ -103,7 +109,7 @@ public class RangeTombstoneList implements Iterable<RangeTombstone>
                                       Arrays.copyOf(ends, size),
                                       Arrays.copyOf(markedAts, size),
                                       Arrays.copyOf(delTimes, size),
-                                      size);
+                                      boundaryHeapSize, size);
     }
 
     public void add(RangeTombstone tombstone)
@@ -138,6 +144,7 @@ public class RangeTombstoneList implements Iterable<RangeTombstone>
             int pos = Arrays.binarySearch(ends, 0, size, start, comparator);
             insertFrom((pos >= 0 ? pos+1 : -pos-1), start, end, markedAt, delTime);
         }
+        boundaryHeapSize += start.unsharedHeapSize() + end.unsharedHeapSize();
     }
 
     /**
@@ -375,6 +382,7 @@ public class RangeTombstoneList implements Iterable<RangeTombstone>
         System.arraycopy(src.markedAts, 0, dst.markedAts, 0, src.size);
         System.arraycopy(src.delTimes, 0, dst.delTimes, 0, src.size);
         dst.size = src.size;
+        dst.boundaryHeapSize = src.boundaryHeapSize;
     }
 
     /*
@@ -572,14 +580,31 @@ public class RangeTombstoneList implements Iterable<RangeTombstone>
         System.arraycopy(ends, i, ends, i+1, size - i);
         System.arraycopy(markedAts, i, markedAts, i+1, size - i);
         System.arraycopy(delTimes, i, delTimes, i+1, size - i);
+        // we set starts[i] to null to indicate the position is now empty, so that we update boundaryHeapSize
+        // when we set it
+        starts[i] = null;
     }
 
     private void setInternal(int i, Composite start, Composite end, long markedAt, int delTime)
     {
+        if (starts[i] != null)
+            boundaryHeapSize -= starts[i].unsharedHeapSize() + ends[i].unsharedHeapSize();
         starts[i] = start;
         ends[i] = end;
         markedAts[i] = markedAt;
         delTimes[i] = delTime;
+        boundaryHeapSize += start.unsharedHeapSize() + end.unsharedHeapSize();
+    }
+
+    @Override
+    public long unsharedHeapSize()
+    {
+        return EMPTY_SIZE
+                + boundaryHeapSize
+                + ObjectSizes.sizeOfArray(starts)
+                + ObjectSizes.sizeOfArray(ends)
+                + ObjectSizes.sizeOfArray(markedAts)
+                + ObjectSizes.sizeOfArray(delTimes);
     }
 
     public static class Serializer implements IVersionedSerializer<RangeTombstoneList>

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/src/java/org/apache/cassandra/db/RowIndexEntry.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/RowIndexEntry.java b/src/java/org/apache/cassandra/db/RowIndexEntry.java
index 3ddaae5..49dfec0 100644
--- a/src/java/org/apache/cassandra/db/RowIndexEntry.java
+++ b/src/java/org/apache/cassandra/db/RowIndexEntry.java
@@ -21,6 +21,7 @@ import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
 
@@ -36,6 +37,8 @@ import org.apache.cassandra.utils.ObjectSizes;
 
 public class RowIndexEntry implements IMeasurableMemory
 {
+    private static final long EMPTY_SIZE = ObjectSizes.measure(new RowIndexEntry(0));
+
     public final long position;
 
     public RowIndexEntry(long position)
@@ -81,9 +84,9 @@ public class RowIndexEntry implements IMeasurableMemory
         return Collections.emptyList();
     }
 
-    public long memorySize()
+    public long unsharedHeapSize()
     {
-        return ObjectSizes.getFieldSize(TypeSizes.NATIVE.sizeof(position));
+        return EMPTY_SIZE;
     }
 
     public static class Serializer
@@ -161,6 +164,9 @@ public class RowIndexEntry implements IMeasurableMemory
     {
         private final DeletionTime deletionTime;
         private final List<IndexHelper.IndexInfo> columnsIndex;
+        private static final long BASE_SIZE =
+                ObjectSizes.measure(new IndexedEntry(0, DeletionTime.LIVE, Arrays.<IndexHelper.IndexInfo>asList(null, null)))
+              + ObjectSizes.measure(new ArrayList<>(1));
 
         private IndexedEntry(long position, DeletionTime deletionTime, List<IndexHelper.IndexInfo> columnsIndex)
         {
@@ -197,19 +203,16 @@ public class RowIndexEntry implements IMeasurableMemory
         }
 
         @Override
-        public long memorySize()
+        public long unsharedHeapSize()
         {
             long entrySize = 0;
             for (IndexHelper.IndexInfo idx : columnsIndex)
-                entrySize += idx.memorySize();
-
-            return ObjectSizes.getSuperClassFieldSize(TypeSizes.NATIVE.sizeof(position))
-                   + ObjectSizes.getFieldSize(// deletionTime
-                                              ObjectSizes.getReferenceSize() +
-                                              // columnsIndex
-                                              ObjectSizes.getReferenceSize())
-                   + deletionTime.memorySize()
-                   + ObjectSizes.getArraySize(columnsIndex.size(), ObjectSizes.getReferenceSize()) + entrySize + 4;
+                entrySize += idx.excessHeapSize();
+
+            return BASE_SIZE
+                   + entrySize
+                   + deletionTime.unsharedHeapSize()
+                   + ObjectSizes.sizeOfReferenceArray(columnsIndex.size());
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/src/java/org/apache/cassandra/db/TreeMapBackedSortedColumns.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/TreeMapBackedSortedColumns.java b/src/java/org/apache/cassandra/db/TreeMapBackedSortedColumns.java
index 252f243..09c50fa 100644
--- a/src/java/org/apache/cassandra/db/TreeMapBackedSortedColumns.java
+++ b/src/java/org/apache/cassandra/db/TreeMapBackedSortedColumns.java
@@ -29,7 +29,7 @@ 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.filter.ColumnSlice;
-import org.apache.cassandra.utils.Allocator;
+import org.apache.cassandra.utils.memory.AbstractAllocator;
 
 public class TreeMapBackedSortedColumns extends AbstractThreadUnsafeSortedColumns
 {
@@ -80,7 +80,7 @@ public class TreeMapBackedSortedColumns extends AbstractThreadUnsafeSortedColumn
      * If we find an old cell that has the same name
      * the ask it to resolve itself else add the new cell
     */
-    public void addColumn(Cell cell, Allocator allocator)
+    public void addColumn(Cell cell, AbstractAllocator allocator)
     {
         CellName name = cell.name();
         // this is a slightly unusual way to structure this; a more natural way is shown in ThreadSafeSortedColumns,
@@ -98,7 +98,7 @@ public class TreeMapBackedSortedColumns extends AbstractThreadUnsafeSortedColumn
     /**
      * We need to go through each column in the column container and resolve it before adding
      */
-    public void addAll(ColumnFamily cm, Allocator allocator, Function<Cell, Cell> transformation)
+    public void addAll(ColumnFamily cm, AbstractAllocator allocator, Function<Cell, Cell> transformation)
     {
         delete(cm.deletionInfo());
         for (Cell cell : cm)
@@ -173,4 +173,5 @@ public class TreeMapBackedSortedColumns extends AbstractThreadUnsafeSortedColumn
     {
         return new ColumnSlice.NavigableMapIterator(map.descendingMap(), slices);
     }
+
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/src/java/org/apache/cassandra/db/UnsortedColumns.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/UnsortedColumns.java b/src/java/org/apache/cassandra/db/UnsortedColumns.java
index a4957ae..ddd7827 100644
--- a/src/java/org/apache/cassandra/db/UnsortedColumns.java
+++ b/src/java/org/apache/cassandra/db/UnsortedColumns.java
@@ -27,7 +27,7 @@ import com.google.common.collect.Iterables;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.composites.CellName;
 import org.apache.cassandra.db.filter.ColumnSlice;
-import org.apache.cassandra.utils.Allocator;
+import org.apache.cassandra.utils.memory.AbstractAllocator;
 
 /**
  * A ColumnFamily that allows inserting in any order, even unsorted.
@@ -79,12 +79,12 @@ public class UnsortedColumns extends AbstractThreadUnsafeSortedColumns
         cells.clear();
     }
 
-    public void addColumn(Cell cell, Allocator allocator)
+    public void addColumn(Cell cell, AbstractAllocator allocator)
     {
         cells.add(cell);
     }
 
-    public void addAll(ColumnFamily cm, Allocator allocator, Function<Cell, Cell> transformation)
+    public void addAll(ColumnFamily cm, AbstractAllocator allocator, Function<Cell, Cell> transformation)
     {
         delete(cm.deletionInfo());
         for (Cell cell : cm)
@@ -141,4 +141,5 @@ public class UnsortedColumns extends AbstractThreadUnsafeSortedColumns
     {
         throw new UnsupportedOperationException();
     }
+
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogService.java b/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogService.java
index a00e992..94802bf 100644
--- a/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogService.java
+++ b/src/java/org/apache/cassandra/db/commitlog/AbstractCommitLogService.java
@@ -17,7 +17,7 @@
  */
 package org.apache.cassandra.db.commitlog;
 
-import org.apache.cassandra.utils.WaitQueue;
+import org.apache.cassandra.utils.concurrent.WaitQueue;
 import org.slf4j.*;
 
 import java.util.concurrent.Semaphore;
@@ -155,9 +155,11 @@ public abstract class AbstractCommitLogService
     /**
      * Sync immediately, but don't block for the sync to cmplete
      */
-    public void requestExtraSync()
+    public WaitQueue.Signal requestExtraSync()
     {
-        haveWork.release();
+        WaitQueue.Signal signal = syncComplete.register();
+        haveWork.release(1);
+        return signal;
     }
 
     public void shutdown()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/src/java/org/apache/cassandra/db/commitlog/CommitLog.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLog.java b/src/java/org/apache/cassandra/db/commitlog/CommitLog.java
index b6b32a0..1181a6f 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLog.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLog.java
@@ -59,7 +59,7 @@ public class CommitLog implements CommitLogMBean
 
     public final CommitLogSegmentManager allocator;
     public final CommitLogArchiver archiver = new CommitLogArchiver();
-    private final CommitLogMetrics metrics;
+    final CommitLogMetrics metrics;
     final AbstractCommitLogService executor;
 
     private CommitLog()
@@ -151,9 +151,9 @@ public class CommitLog implements CommitLogMBean
      * @return a Future representing a ReplayPosition such that when it is ready,
      * all Allocations created prior to the getContext call will be written to the log
      */
-    public Future<ReplayPosition> getContext()
+    public ReplayPosition getContext()
     {
-        return Futures.immediateFuture(allocator.allocatingFrom().getContext());
+        return allocator.allocatingFrom().getContext();
     }
 
     /**
@@ -191,18 +191,26 @@ public class CommitLog implements CommitLogMBean
      *
      * @param mutation the Mutation to add to the log
      */
-    public void add(Mutation mutation)
+    public ReplayPosition add(Mutation mutation)
     {
+        Allocation alloc = add(mutation, new Allocation());
+        return alloc.getReplayPosition();
+    }
+
+    private Allocation add(Mutation mutation, Allocation alloc)
+    {
+        assert mutation != null;
+
         long size = Mutation.serializer.serializedSize(mutation, MessagingService.current_version);
 
         long totalSize = size + ENTRY_OVERHEAD_SIZE;
         if (totalSize > MAX_MUTATION_SIZE)
         {
             logger.warn("Skipping commitlog append of extremely large mutation ({} bytes)", totalSize);
-            return;
+            return alloc;
         }
 
-        Allocation alloc = allocator.allocate(mutation, (int) totalSize, new Allocation());
+        allocator.allocate(mutation, (int) totalSize, alloc);
         try
         {
             PureJavaCrc32 checksum = new PureJavaCrc32();
@@ -227,6 +235,7 @@ public class CommitLog implements CommitLogMBean
         }
 
         executor.finishWriteFor(alloc);
+        return alloc;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java
index 3acbc63..a47fcc0 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java
@@ -33,10 +33,8 @@ import java.util.UUID;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReadWriteLock;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import org.apache.cassandra.utils.concurrent.OpOrder;
 
 import org.cliffc.high_scale_lib.NonBlockingHashMap;
 import org.slf4j.Logger;
@@ -50,7 +48,7 @@ import org.apache.cassandra.db.Mutation;
 import org.apache.cassandra.io.FSWriteError;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.utils.PureJavaCrc32;
-import org.apache.cassandra.utils.WaitQueue;
+import org.apache.cassandra.utils.concurrent.WaitQueue;
 
 /*
  * A single commit log file on disk. Manages creation of the file and writing mutations to disk,
@@ -70,8 +68,8 @@ public class CommitLogSegment
     // The commit log (chained) sync marker/header size in bytes (int: length + long: checksum [segmentId, position])
     static final int SYNC_MARKER_SIZE = 4 + 8;
 
-    // The current AppendLock object - i.e. the one all threads adding new log records should use to synchronise
-    private final AtomicReference<AppendLock> appendLock = new AtomicReference<>(new AppendLock());
+    // The OpOrder used to order appends wrt sync
+    private final OpOrder appendOrder = new OpOrder();
 
     private final AtomicInteger allocatePosition = new AtomicInteger();
 
@@ -172,40 +170,29 @@ public class CommitLogSegment
      */
     boolean allocate(Mutation mutation, int size, Allocation alloc)
     {
-        final AppendLock appendLock = lockForAppend();
+        final OpOrder.Group commandOrder = appendOrder.start();
         try
         {
             int position = allocate(size);
             if (position < 0)
             {
-                appendLock.unlock();
+                commandOrder.finishOne();
                 return false;
             }
             alloc.buffer = (ByteBuffer) buffer.duplicate().position(position).limit(position + size);
             alloc.position = position;
             alloc.segment = this;
-            alloc.appendLock = appendLock;
+            alloc.appendOp = commandOrder;
             markDirty(mutation, position);
             return true;
         }
         catch (Throwable t)
         {
-            appendLock.unlock();
+            commandOrder.finishOne();
             throw t;
         }
     }
 
-    // obtain the current AppendLock and lock it for record appending
-    private AppendLock lockForAppend()
-    {
-        while (true)
-        {
-            AppendLock appendLock = this.appendLock.get();
-            if (appendLock.lock())
-                return appendLock;
-        }
-    }
-
     // allocate bytes in the segment, or return -1 if not enough space
     private int allocate(int size)
     {
@@ -221,20 +208,44 @@ public class CommitLogSegment
     }
 
     // ensures no more of this segment is writeable, by allocating any unused section at the end and marking it discarded
-    synchronized void discardUnusedTail()
+    void discardUnusedTail()
     {
-        if (discardedTailFrom > 0)
-            return;
-        while (true)
+        // we guard this with the OpOrdering instead of synchronised due to potential dead-lock with CLSM.advanceAllocatingFrom()
+        // this actually isn't strictly necessary, as currently all calls to discardUnusedTail occur within a block
+        // already protected by this OpOrdering, but to prevent future potential mistakes, we duplicate the protection here
+        // so that the contract between discardUnusedTail() and sync() is more explicit.
+        OpOrder.Group group = appendOrder.start();
+        try
         {
-            int prev = allocatePosition.get();
-            int next = buffer.capacity();
-            if (allocatePosition.compareAndSet(prev, next))
+            while (true)
             {
-                discardedTailFrom = prev;
-                return;
+                int prev = allocatePosition.get();
+                // we set allocatePosition past buffer.capacity() to make sure we always set discardedTailFrom
+                int next = buffer.capacity() + 1;
+                if (prev == next)
+                    return;
+                if (allocatePosition.compareAndSet(prev, next))
+                {
+                    discardedTailFrom = prev;
+                    return;
+                }
             }
         }
+        finally
+        {
+            group.finishOne();
+        }
+    }
+
+    /**
+     * Wait for any appends or discardUnusedTail() operations started before this method was called
+     */
+    private synchronized void waitForModifications()
+    {
+        // issue a barrier and wait for it
+        OpOrder.Barrier barrier = appendOrder.newBarrier();
+        barrier.issue();
+        barrier.await();
     }
 
     /**
@@ -259,6 +270,9 @@ public class CommitLogSegment
                 discardUnusedTail();
                 close = true;
 
+                // wait for modifications guards both discardedTailFrom, and any outstanding appends
+                waitForModifications();
+
                 if (discardedTailFrom < buffer.capacity() - SYNC_MARKER_SIZE)
                 {
                     // if there's room in the discard section to write an empty header, use that as the nextMarker
@@ -270,11 +284,12 @@ public class CommitLogSegment
                     nextMarker = buffer.capacity();
                 }
             }
+            else
+            {
+                waitForModifications();
+            }
 
-            // swap the append lock
-            AppendLock curAppendLock = appendLock.get();
-            appendLock.set(new AppendLock());
-            curAppendLock.expireAndWaitForCompletion();
+            assert nextMarker > lastSyncedOffset;
 
             // write previous sync marker to point to next sync marker
             // we don't chain the crcs here to ensure this method is idempotent if it fails
@@ -311,9 +326,9 @@ public class CommitLogSegment
         }
     }
 
-    public boolean isFullySynced()
+    public boolean isStillAllocating()
     {
-        return lastSyncedOffset == buffer.capacity();
+        return allocatePosition.get() < buffer.capacity();
     }
 
     /**
@@ -445,7 +460,7 @@ public class CommitLogSegment
     private void removeCleanFromDirty()
     {
         // if we're still allocating from this segment, don't touch anything since it can't be done thread-safely
-        if (!isFullySynced())
+        if (isStillAllocating())
             return;
 
         Iterator<Map.Entry<UUID, AtomicInteger>> iter = cfClean.entrySet().iterator();
@@ -463,7 +478,6 @@ public class CommitLogSegment
         }
     }
 
-
     /**
      * @return a collection of dirty CFIDs for this segment file.
      */
@@ -489,8 +503,9 @@ public class CommitLogSegment
      */
     public boolean isUnused()
     {
-        // if it's not fully synced, we assume we're still in use as the active allocatingFrom
-        if (!isFullySynced())
+        // if room to allocate, we're still in use as the active allocatingFrom,
+        // so we don't want to race with updates to cfClean with removeCleanFromDirty
+        if (isStillAllocating())
             return false;
 
         removeCleanFromDirty();
@@ -537,63 +552,15 @@ public class CommitLogSegment
     }
 
     /**
-     * A relatively simple class for synchronising flushes() with log message writers:
-     * Log writers take the readLock prior to allocating themselves space in the segment;
-     * once they complete writing the record they release the read lock. A call to sync()
-     * will first check the position we have allocated space up until, then allocate a new AppendLock object,
-     * take the writeLock of the previous AppendLock, and invalidate it for further log writes. All appends are
-     * redirected to the new AppendLock so they do not block; only the sync() blocks waiting to obtain the writeLock.
-     * Once it obtains the lock it is guaranteed that all writes up to the allocation position it checked at
-     * the start have been completely written to.
-     */
-    private static final class AppendLock
-    {
-        final ReadWriteLock syncLock = new ReentrantReadWriteLock();
-        final Lock logLock = syncLock.readLock();
-        // a map of Cfs with log records that have not been synced to disk, so cannot be marked clean yet
-
-        boolean expired;
-
-        // false if the lock could not be acquired for adding a log record;
-        // a new AppendLock object will already be available, so fetch appendLock().get()
-        // and retry
-        boolean lock()
-        {
-            if (!logLock.tryLock())
-                return false;
-            if (expired)
-            {
-                logLock.unlock();
-                return false;
-            }
-            return true;
-        }
-
-        // release the lock so that a appendLock() may complete
-        void unlock()
-        {
-            logLock.unlock();
-        }
-
-        void expireAndWaitForCompletion()
-        {
-            // wait for log records to complete (take writeLock)
-            syncLock.writeLock().lock();
-            expired = true;
-            // release lock immediately, though effectively a NOOP since we use tryLock() for log record appends
-            syncLock.writeLock().unlock();
-        }
-    }
-
-    /**
      * A simple class for tracking information about the portion of a segment that has been allocated to a log write.
      * The constructor leaves the fields uninitialized for population by CommitlogManager, so that it can be
      * stack-allocated by escape analysis in CommitLog.add.
      */
-    static final class Allocation
+    static class Allocation
     {
+
         private CommitLogSegment segment;
-        private AppendLock appendLock;
+        private OpOrder.Group appendOp;
         private int position;
         private ByteBuffer buffer;
 
@@ -608,19 +575,30 @@ public class CommitLogSegment
         }
 
         // markWritten() MUST be called once we are done with the segment or the CL will never flush
+        // but must not be called more than once
         void markWritten()
         {
-            appendLock.unlock();
+            appendOp.finishOne();
         }
 
         void awaitDiskSync()
         {
             while (segment.lastSyncedOffset < position)
             {
-                WaitQueue.Signal signal = segment.syncComplete.register();
+                WaitQueue.Signal signal = segment.syncComplete.register(CommitLog.instance.metrics.waitingOnCommit.time());
                 if (segment.lastSyncedOffset < position)
                     signal.awaitUninterruptibly();
+                else
+                    signal.cancel();
             }
         }
+
+        public ReplayPosition getReplayPosition()
+        {
+            // always allocate a ReplayPosition to let stack allocation do its magic. If we return null, we always
+            // have to allocate an object on the stack
+            return new ReplayPosition(segment == null ? -1 : segment.id, segment == null ? 0 : buffer.limit());
+        }
+
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java
index 4275362..bc87370 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManager.java
@@ -36,6 +36,8 @@ import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
 
 import com.google.common.collect.Iterables;
+import com.google.common.util.concurrent.*;
+import org.apache.cassandra.service.paxos.Commit;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -49,7 +51,7 @@ import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.Pair;
-import org.apache.cassandra.utils.WaitQueue;
+import org.apache.cassandra.utils.concurrent.WaitQueue;
 import org.apache.cassandra.utils.WrappedRunnable;
 
 import static org.apache.cassandra.db.commitlog.CommitLogSegment.Allocation;
@@ -135,7 +137,7 @@ public class CommitLogSegmentManager
                                 if (spaceToReclaim + unused >= 0)
                                     break;
                             }
-                            flushDataFrom(segmentsToRecycle);
+                            flushDataFrom(segmentsToRecycle, false);
                         }
 
                         try
@@ -239,26 +241,29 @@ public class CommitLogSegmentManager
             }
 
             // no more segments, so register to receive a signal when not empty
-            WaitQueue.Signal signal = hasAvailableSegments.register();
+            WaitQueue.Signal signal = hasAvailableSegments.register(CommitLog.instance.metrics.waitingOnSegmentAllocation.time());
 
             // trigger the management thread; this must occur after registering
             // the signal to ensure we are woken by any new segment creation
             wakeManager();
 
             // check if the queue has already been added to before waiting on the signal, to catch modifications
-            // that happened prior to registering the signal
-            if (availableSegments.isEmpty())
+            // that happened prior to registering the signal; *then* check to see if we've been beaten to making the change
+            if (!availableSegments.isEmpty() || allocatingFrom != old)
             {
-                // check to see if we've been beaten to it
+                signal.cancel();
+                // if we've been beaten, just stop immediately
                 if (allocatingFrom != old)
                     return;
-
-                // can only reach here if the queue hasn't been inserted into
-                // before we registered the signal, as we only remove items from the queue
-                // after updating allocatingFrom. Can safely block until we are signalled
-                // by the allocator that new segments have been published
-                signal.awaitUninterruptibly();
+                // otherwise try again, as there should be an available segment
+                continue;
             }
+
+            // can only reach here if the queue hasn't been inserted into
+            // before we registered the signal, as we only remove items from the queue
+            // after updating allocatingFrom. Can safely block until we are signalled
+            // by the allocator that new segments have been published
+            signal.awaitUninterruptibly();
         }
     }
 
@@ -282,13 +287,12 @@ public class CommitLogSegmentManager
      */
     void forceRecycleAll()
     {
-        CommitLogSegment last = allocatingFrom;
-        last.discardUnusedTail();
         List<CommitLogSegment> segmentsToRecycle = new ArrayList<>(activeSegments);
+        CommitLogSegment last = segmentsToRecycle.get(segmentsToRecycle.size() - 1);
         advanceAllocatingFrom(last);
 
         // flush and wait for all CFs that are dirty in segments up-to and including 'last'
-        Future<?> future = flushDataFrom(segmentsToRecycle);
+        Future<?> future = flushDataFrom(segmentsToRecycle, true);
         try
         {
             future.get();
@@ -443,10 +447,14 @@ public class CommitLogSegmentManager
      *
      * @return a Future that will finish when all the flushes are complete.
      */
-    private Future<?> flushDataFrom(Collection<CommitLogSegment> segments)
+    private Future<?> flushDataFrom(List<CommitLogSegment> segments, boolean force)
     {
+        if (segments.isEmpty())
+            return Futures.immediateFuture(null);
+        final ReplayPosition maxReplayPosition = segments.get(segments.size() - 1).getContext();
+
         // a map of CfId -> forceFlush() to ensure we only queue one flush per cf
-        final Map<UUID, Future<?>> flushes = new LinkedHashMap<>();
+        final Map<UUID, ListenableFuture<?>> flushes = new LinkedHashMap<>();
 
         for (CommitLogSegment segment : segments)
         {
@@ -464,29 +472,14 @@ public class CommitLogSegmentManager
                 {
                     String keyspace = pair.left;
                     final ColumnFamilyStore cfs = Keyspace.open(keyspace).getColumnFamilyStore(dirtyCFId);
-                    // Push the flush out to another thread to avoid potential deadlock: Table.add
-                    // acquires switchlock, and could be blocking for the manager thread.  So if the manager
-                    // thread itself tries to acquire switchlock (via flush -> switchMemtable) we'd have a problem.
-                    Runnable runnable = new Runnable()
-                    {
-                        public void run()
-                        {
-                            cfs.forceFlush();
-                        }
-                    };
-                    flushes.put(dirtyCFId, StorageService.optionalTasks.submit(runnable));
+                    // can safely call forceFlush here as we will only ever block (briefly) for other attempts to flush,
+                    // no deadlock possibility since switchLock removal
+                    flushes.put(dirtyCFId, force ? cfs.forceFlush() : cfs.forceFlush(maxReplayPosition));
                 }
             }
         }
 
-        return new FutureTask<>(new Callable<Object>()
-        {
-            public Object call()
-            {
-                FBUtilities.waitOnFutures(flushes.values());
-                return null;
-            }
-        });
+        return Futures.allAsList(flushes.values());
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/src/java/org/apache/cassandra/db/commitlog/PeriodicCommitLogService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/commitlog/PeriodicCommitLogService.java b/src/java/org/apache/cassandra/db/commitlog/PeriodicCommitLogService.java
index 19d2770..14bb367 100644
--- a/src/java/org/apache/cassandra/db/commitlog/PeriodicCommitLogService.java
+++ b/src/java/org/apache/cassandra/db/commitlog/PeriodicCommitLogService.java
@@ -18,7 +18,7 @@
 package org.apache.cassandra.db.commitlog;
 
 import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.utils.WaitQueue;
+import org.apache.cassandra.utils.concurrent.WaitQueue;
 
 class PeriodicCommitLogService extends AbstractCommitLogService
 {
@@ -39,9 +39,11 @@ class PeriodicCommitLogService extends AbstractCommitLogService
             pending.incrementAndGet();
             while (waitForSyncToCatchUp(started))
             {
-                WaitQueue.Signal signal = syncComplete.register();
+                WaitQueue.Signal signal = syncComplete.register(CommitLog.instance.metrics.waitingOnCommit.time());
                 if (waitForSyncToCatchUp(started))
                     signal.awaitUninterruptibly();
+                else
+                    signal.cancel();
             }
             pending.decrementAndGet();
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/src/java/org/apache/cassandra/db/commitlog/ReplayPosition.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/commitlog/ReplayPosition.java b/src/java/org/apache/cassandra/db/commitlog/ReplayPosition.java
index fb78ed3..e45e6e0 100644
--- a/src/java/org/apache/cassandra/db/commitlog/ReplayPosition.java
+++ b/src/java/org/apache/cassandra/db/commitlog/ReplayPosition.java
@@ -117,6 +117,11 @@ public class ReplayPosition implements Comparable<ReplayPosition>
                ')';
     }
 
+    public ReplayPosition clone()
+    {
+        return new ReplayPosition(segment, position);
+    }
+
     public static class ReplayPositionSerializer implements ISerializer<ReplayPosition>
     {
         public void serialize(ReplayPosition rp, DataOutput out) throws IOException

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/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 1414c3f..eeed45f 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
@@ -30,6 +30,9 @@ import javax.management.openmbean.TabularData;
 import com.google.common.base.Throwables;
 import com.google.common.collect.*;
 import com.google.common.util.concurrent.RateLimiter;
+
+import org.apache.cassandra.utils.concurrent.OpOrder;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -700,14 +703,14 @@ public class CompactionManager implements CompactionManagerMBean
                 if (indexedColumnsInRow != null && !indexedColumnsInRow.isEmpty())
                 {
                     // acquire memtable lock here because secondary index deletion may cause a race. See CASSANDRA-3712
-                    Keyspace.switchLock.readLock().lock();
+                    final OpOrder.Group opGroup = cfs.keyspace.writeOrder.start();
                     try
                     {
-                        cfs.indexManager.deleteFromIndexes(row.getKey(), indexedColumnsInRow);
+                        cfs.indexManager.deleteFromIndexes(row.getKey(), indexedColumnsInRow, opGroup);
                     }
                     finally
                     {
-                        Keyspace.switchLock.readLock().unlock();
+                        opGroup.finishOne();
                     }
                 }
                 return null;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionTask.java b/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
index cb0dcd5..1bb5df8 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionTask.java
@@ -22,8 +22,8 @@ import java.io.IOException;
 import java.util.*;
 import java.util.concurrent.TimeUnit;
 
-import com.google.common.base.Throwables;
-import com.google.common.collect.Sets;
+import com.google.common.base.*;
+import com.google.common.collect.*;
 import org.apache.commons.lang3.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -107,8 +107,14 @@ public class CompactionTask extends AbstractCompactionTask
             cfs.snapshotWithoutFlush(System.currentTimeMillis() + "-compact-" + cfs.name);
 
         // sanity check: all sstables must belong to the same cfs
-        for (SSTableReader sstable : toCompact)
-            assert sstable.descriptor.cfname.equals(cfs.name);
+        assert !Iterables.any(toCompact, new Predicate<SSTableReader>()
+        {
+            @Override
+            public boolean apply(SSTableReader sstable)
+            {
+                return !sstable.descriptor.cfname.equals(cfs.name);
+            }
+        });
 
         UUID taskId = SystemKeyspace.startCompaction(cfs, toCompact);
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/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 3e360ab..02901a3 100644
--- a/src/java/org/apache/cassandra/db/compaction/LazilyCompactedRow.java
+++ b/src/java/org/apache/cassandra/db/compaction/LazilyCompactedRow.java
@@ -65,7 +65,7 @@ public class LazilyCompactedRow extends AbstractCompactedRow
         super(rows.get(0).getKey());
         this.rows = rows;
         this.controller = controller;
-        indexer = controller.cfs.indexManager.updaterFor(key);
+        indexer = controller.cfs.indexManager.gcUpdaterFor(key);
 
         // Combine top-level tombstones, keeping the one with the highest markedForDeleteAt timestamp.  This may be
         // purged (depending on gcBefore), but we need to remember it to properly delete columns during the merge
@@ -96,7 +96,7 @@ public class LazilyCompactedRow extends AbstractCompactedRow
         // are shadowed by a row or range tombstone; removeDeletedColumnsOnly(cf, Integer.MIN_VALUE) will accomplish this
         // without purging tombstones.
         int overriddenGCBefore = shouldPurge ? controller.gcBefore : Integer.MIN_VALUE;
-        ColumnFamilyStore.removeDeletedColumnsOnly(cf, overriddenGCBefore, controller.cfs.indexManager.updaterFor(key));
+        ColumnFamilyStore.removeDeletedColumnsOnly(cf, overriddenGCBefore, controller.cfs.indexManager.gcUpdaterFor(key));
     }
 
     public RowIndexEntry write(long currentPosition, DataOutput out) throws IOException

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/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 9e56dc2..86cdc4e 100644
--- a/src/java/org/apache/cassandra/db/composites/BoundedComposite.java
+++ b/src/java/org/apache/cassandra/db/composites/BoundedComposite.java
@@ -19,12 +19,14 @@ package org.apache.cassandra.db.composites;
 
 import java.nio.ByteBuffer;
 
-import org.apache.cassandra.db.TypeSizes;
-import org.apache.cassandra.utils.Allocator;
+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
 {
+    private static final long EMPTY_SIZE = ObjectSizes.measure(new BoundedComposite(null, false));
+
     private final Composite wrapped;
     private final boolean isStart;
 
@@ -82,14 +84,19 @@ public class BoundedComposite extends AbstractComposite
         return bb;
     }
 
-    public long memorySize()
+    public long unsharedHeapSize()
     {
-        return ObjectSizes.getFieldSize(ObjectSizes.getReferenceSize() + TypeSizes.NATIVE.sizeof(isStart))
-             + wrapped.memorySize();
+        return EMPTY_SIZE + wrapped.unsharedHeapSize();
     }
 
-    public Composite copy(Allocator allocator)
+    public Composite copy(AbstractAllocator allocator)
     {
         return new BoundedComposite(wrapped.copy(allocator), isStart);
     }
+
+    @Override
+    public void free(PoolAllocator<?> allocator)
+    {
+        wrapped.free(allocator);
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/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 1ca50eb..d55243e 100644
--- a/src/java/org/apache/cassandra/db/composites/CellName.java
+++ b/src/java/org/apache/cassandra/db/composites/CellName.java
@@ -20,7 +20,7 @@ package org.apache.cassandra.db.composites;
 import java.nio.ByteBuffer;
 
 import org.apache.cassandra.cql3.ColumnIdentifier;
-import org.apache.cassandra.utils.Allocator;
+import org.apache.cassandra.utils.memory.AbstractAllocator;
 
 /**
  * A CellName is a Composite, but for which, for the sake of CQL3, we
@@ -70,5 +70,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(Allocator allocator);
+    public CellName copy(AbstractAllocator allocator);
+
+    public long excessHeapSizeExcludingData();
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/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 7001384..c82d170 100644
--- a/src/java/org/apache/cassandra/db/composites/Composite.java
+++ b/src/java/org/apache/cassandra/db/composites/Composite.java
@@ -21,7 +21,8 @@ import java.nio.ByteBuffer;
 
 import org.apache.cassandra.cache.IMeasurableMemory;
 import org.apache.cassandra.db.filter.ColumnSlice;
-import org.apache.cassandra.utils.Allocator;
+import org.apache.cassandra.utils.memory.AbstractAllocator;
+import org.apache.cassandra.utils.memory.PoolAllocator;
 
 /**
  * A composite value.
@@ -72,5 +73,6 @@ public interface Composite extends IMeasurableMemory
     public ByteBuffer toByteBuffer();
 
     public int dataSize();
-    public Composite copy(Allocator allocator);
+    public Composite copy(AbstractAllocator allocator);
+    public void free(PoolAllocator<?> allocator);
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/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 e02725c..f1b7eab 100644
--- a/src/java/org/apache/cassandra/db/composites/Composites.java
+++ b/src/java/org/apache/cassandra/db/composites/Composites.java
@@ -20,8 +20,9 @@ package org.apache.cassandra.db.composites;
 import java.nio.ByteBuffer;
 
 import org.apache.cassandra.db.filter.ColumnSlice;
-import org.apache.cassandra.utils.Allocator;
+import org.apache.cassandra.utils.memory.AbstractAllocator;
 import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.memory.PoolAllocator;
 
 public abstract class Composites
 {
@@ -92,7 +93,7 @@ public abstract class Composites
             return 0;
         }
 
-        public long memorySize()
+        public long unsharedHeapSize()
         {
             return 0;
         }
@@ -102,9 +103,15 @@ public abstract class Composites
             return true;
         }
 
-        public Composite copy(Allocator allocator)
+        public Composite copy(AbstractAllocator allocator)
         {
             return this;
         }
+
+        @Override
+        public void free(PoolAllocator<?> allocator)
+        {
+        }
+
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/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 0a258b3..54ec759 100644
--- a/src/java/org/apache/cassandra/db/composites/CompoundComposite.java
+++ b/src/java/org/apache/cassandra/db/composites/CompoundComposite.java
@@ -19,15 +19,17 @@ package org.apache.cassandra.db.composites;
 
 import java.nio.ByteBuffer;
 
-import org.apache.cassandra.db.TypeSizes;
-import org.apache.cassandra.utils.Allocator;
+import org.apache.cassandra.utils.memory.AbstractAllocator;
 import org.apache.cassandra.utils.ObjectSizes;
+import org.apache.cassandra.utils.memory.PoolAllocator;
 
 /**
  * A "truly-composite" Composite.
  */
 public class CompoundComposite extends AbstractComposite
 {
+    private static final long EMPTY_SIZE = ObjectSizes.measure(new CompoundComposite(null, 0));
+
     // We could use a List, but we'll create such object *a lot* and using a array+size is not
     // all that harder, so we save the List object allocation.
     final ByteBuffer[] elements;
@@ -49,7 +51,7 @@ public class CompoundComposite extends AbstractComposite
         return elements[i];
     }
 
-    protected ByteBuffer[] elementsCopy(Allocator allocator)
+    protected ByteBuffer[] elementsCopy(AbstractAllocator allocator)
     {
         ByteBuffer[] elementsCopy = new ByteBuffer[size];
         for (int i = 0; i < size; i++)
@@ -57,14 +59,26 @@ public class CompoundComposite extends AbstractComposite
         return elementsCopy;
     }
 
-    public long memorySize()
+    public long unsharedHeapSize()
+    {
+        return EMPTY_SIZE + ObjectSizes.sizeOnHeapOf(elements);
+    }
+
+    public long excessHeapSizeExcludingData()
     {
-        return ObjectSizes.getFieldSize(TypeSizes.NATIVE.sizeof(size))
-             + ObjectSizes.getArraySize(elements);
+        return EMPTY_SIZE + ObjectSizes.sizeOnHeapExcludingData(elements);
     }
 
-    public Composite copy(Allocator allocator)
+    public Composite copy(AbstractAllocator allocator)
     {
         return new CompoundComposite(elementsCopy(allocator), size);
     }
+
+    @Override
+    public void free(PoolAllocator<?> allocator)
+    {
+        for (ByteBuffer element : elements)
+            allocator.free(element);
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/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 10f1a11..26df34b 100644
--- a/src/java/org/apache/cassandra/db/composites/CompoundDenseCellName.java
+++ b/src/java/org/apache/cassandra/db/composites/CompoundDenseCellName.java
@@ -20,11 +20,14 @@ package org.apache.cassandra.db.composites;
 import java.nio.ByteBuffer;
 
 import org.apache.cassandra.cql3.ColumnIdentifier;
-import org.apache.cassandra.utils.Allocator;
+import org.apache.cassandra.utils.memory.AbstractAllocator;
 import org.apache.cassandra.utils.ObjectSizes;
 
 public class CompoundDenseCellName extends CompoundComposite implements CellName
 {
+
+    private static final long HEAP_SIZE = ObjectSizes.measure(new CompoundDenseCellName(new ByteBuffer[0]));
+
     // Not meant to be used directly, you should use the CellNameType method instead
     CompoundDenseCellName(ByteBuffer[] elements)
     {
@@ -58,13 +61,20 @@ public class CompoundDenseCellName extends CompoundComposite implements CellName
     }
 
     @Override
-    public long memorySize()
+    public long unsharedHeapSize()
     {
-        return ObjectSizes.getSuperClassFieldSize(super.memorySize());
+        return HEAP_SIZE + ObjectSizes.sizeOnHeapOf(elements);
     }
 
-    public CellName copy(Allocator allocator)
+    @Override
+    public long excessHeapSizeExcludingData()
+    {
+        return HEAP_SIZE + ObjectSizes.sizeOnHeapExcludingData(elements);
+    }
+
+    public CellName copy(AbstractAllocator allocator)
     {
         return new CompoundDenseCellName(elementsCopy(allocator));
     }
+
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/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 77b311e..dc525d6 100644
--- a/src/java/org/apache/cassandra/db/composites/CompoundSparseCellName.java
+++ b/src/java/org/apache/cassandra/db/composites/CompoundSparseCellName.java
@@ -20,13 +20,17 @@ package org.apache.cassandra.db.composites;
 import java.nio.ByteBuffer;
 
 import org.apache.cassandra.cql3.ColumnIdentifier;
-import org.apache.cassandra.utils.Allocator;
+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
 {
     private static final ByteBuffer[] EMPTY_PREFIX = new ByteBuffer[0];
 
+    private static final long HEAP_SIZE = ObjectSizes.measure(new CompoundSparseCellName(null));
+
     protected final ColumnIdentifier columnName;
 
     // Not meant to be used directly, you should use the CellNameType method instead
@@ -89,7 +93,7 @@ public class CompoundSparseCellName extends CompoundComposite implements CellNam
         return true;
     }
 
-    public CellName copy(Allocator allocator)
+    public CellName copy(AbstractAllocator allocator)
     {
         if (elements.length == 0)
             return this;
@@ -98,15 +102,10 @@ public class CompoundSparseCellName extends CompoundComposite implements CellNam
         return new CompoundSparseCellName(elementsCopy(allocator), columnName);
     }
 
-    @Override
-    public long memorySize()
-    {
-        return ObjectSizes.getSuperClassFieldSize(super.memorySize())
-             + ObjectSizes.getFieldSize(ObjectSizes.getReferenceSize()) + columnName.memorySize();
-    }
-
     public static class WithCollection extends CompoundSparseCellName
     {
+        private static final long HEAP_SIZE = ObjectSizes.measure(new WithCollection(null, ByteBufferUtil.EMPTY_BYTE_BUFFER));
+
         private final ByteBuffer collectionElement;
 
         WithCollection(ColumnIdentifier columnName, ByteBuffer collectionElement)
@@ -148,17 +147,29 @@ public class CompoundSparseCellName extends CompoundComposite implements CellNam
         }
 
         @Override
-        public CellName copy(Allocator allocator)
+        public CellName copy(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));
         }
 
         @Override
-        public long memorySize()
+        public long unsharedHeapSize()
+        {
+            return super.unsharedHeapSize() + ObjectSizes.sizeOnHeapOf(collectionElement);
+        }
+
+        @Override
+        public long excessHeapSizeExcludingData()
+        {
+            return super.excessHeapSizeExcludingData() + ObjectSizes.sizeOnHeapExcludingData(collectionElement);
+        }
+
+        @Override
+        public void free(PoolAllocator<?> allocator)
         {
-            return ObjectSizes.getSuperClassFieldSize(super.memorySize())
-                 + ObjectSizes.getFieldSize(ObjectSizes.getReferenceSize()) + ObjectSizes.getSize(collectionElement);
+            super.free(allocator);
+            allocator.free(collectionElement);
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/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 e88afe7..7fd6ddb 100644
--- a/src/java/org/apache/cassandra/db/composites/SimpleComposite.java
+++ b/src/java/org/apache/cassandra/db/composites/SimpleComposite.java
@@ -19,14 +19,17 @@ package org.apache.cassandra.db.composites;
 
 import java.nio.ByteBuffer;
 
-import org.apache.cassandra.utils.Allocator;
+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.
  */
 public class SimpleComposite extends AbstractComposite
 {
+    private static final long EMPTY_SIZE = ObjectSizes.measure(new SimpleComposite(ByteBuffer.allocate(1)));
+
     protected final ByteBuffer element;
 
     SimpleComposite(ByteBuffer element)
@@ -64,13 +67,19 @@ public class SimpleComposite extends AbstractComposite
         return element;
     }
 
-    public long memorySize()
+    public long unsharedHeapSize()
     {
-        return ObjectSizes.getFieldSize(ObjectSizes.getSize(element));
+        return EMPTY_SIZE + ObjectSizes.sizeOnHeapOf(element);
     }
 
-    public Composite copy(Allocator allocator)
+    public Composite copy(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/4b54b8ac/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 338e8a0..a72c458 100644
--- a/src/java/org/apache/cassandra/db/composites/SimpleDenseCellName.java
+++ b/src/java/org/apache/cassandra/db/composites/SimpleDenseCellName.java
@@ -20,11 +20,13 @@ package org.apache.cassandra.db.composites;
 import java.nio.ByteBuffer;
 
 import org.apache.cassandra.cql3.ColumnIdentifier;
-import org.apache.cassandra.utils.Allocator;
+import org.apache.cassandra.utils.memory.AbstractAllocator;
 import org.apache.cassandra.utils.ObjectSizes;
 
 public class SimpleDenseCellName extends SimpleComposite implements CellName
 {
+    private static final long EMPTY_SIZE = ObjectSizes.measure(new SimpleDenseCellName(ByteBuffer.allocate(1)));
+
     // Not meant to be used directly, you should use the CellNameType method instead
     SimpleDenseCellName(ByteBuffer element)
     {
@@ -58,16 +60,23 @@ public class SimpleDenseCellName extends SimpleComposite implements CellName
     }
 
     @Override
-    public long memorySize()
+    public long unsharedHeapSize()
+    {
+        return EMPTY_SIZE + ObjectSizes.sizeOnHeapOf(element);
+    }
+
+    @Override
+    public long excessHeapSizeExcludingData()
     {
-        return ObjectSizes.getSuperClassFieldSize(super.memorySize());
+        return EMPTY_SIZE + ObjectSizes.sizeOnHeapExcludingData(element);
     }
 
     // If cellnames were sharing some prefix components, this will break it, so
     // we might want to try to do better.
     @Override
-    public CellName copy(Allocator allocator)
+    public CellName copy(AbstractAllocator allocator)
     {
         return new SimpleDenseCellName(allocator.clone(element));
     }
+
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/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 8105683..7e6a357 100644
--- a/src/java/org/apache/cassandra/db/composites/SimpleSparseCellName.java
+++ b/src/java/org/apache/cassandra/db/composites/SimpleSparseCellName.java
@@ -20,11 +20,14 @@ package org.apache.cassandra.db.composites;
 import java.nio.ByteBuffer;
 
 import org.apache.cassandra.cql3.ColumnIdentifier;
-import org.apache.cassandra.utils.Allocator;
+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
 {
+    private static final long EMPTY_SIZE = ObjectSizes.measure(new SimpleSparseCellName(null));
+
     private final ColumnIdentifier columnName;
 
     // Not meant to be used directly, you should use the CellNameType method instead
@@ -84,16 +87,23 @@ public class SimpleSparseCellName extends AbstractComposite implements CellName
         return true;
     }
 
-    @Override
-    public long memorySize()
+    public long excessHeapSizeExcludingData()
     {
-        return ObjectSizes.getFieldSize(ObjectSizes.getReferenceSize()) + columnName.memorySize();
+        return EMPTY_SIZE + columnName.excessHeapSizeExcludingData();
     }
 
-    @Override
-    public CellName copy(Allocator allocator)
+    public long unsharedHeapSize()
     {
-        // We're interning those instance in SparceCellNameType so don't need to copy.
-        return this;
+        return EMPTY_SIZE + columnName.unsharedHeapSize();
+    }
+
+    public CellName copy(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/4b54b8ac/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 6d86ce2..9c99680 100644
--- a/src/java/org/apache/cassandra/db/composites/SimpleSparseCellNameType.java
+++ b/src/java/org/apache/cassandra/db/composites/SimpleSparseCellNameType.java
@@ -83,7 +83,7 @@ public class SimpleSparseCellNameType extends AbstractSimpleCellNameType
 
     public void addCQL3Column(ColumnIdentifier id)
     {
-        internedNames.put(id.bytes, new SimpleSparseCellName(id));
+        internedNames.put(id.bytes, new SimpleSparseInternedCellName(id));
     }
 
     public void removeCQL3Column(ColumnIdentifier id)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/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
new file mode 100644
index 0000000..6cc29dc
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/composites/SimpleSparseInternedCellName.java
@@ -0,0 +1,58 @@
+/*
+ * 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.composites;
+
+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
+{
+
+    // Not meant to be used directly, you should use the CellNameType method instead
+    SimpleSparseInternedCellName(ColumnIdentifier columnName)
+    {
+        super(columnName);
+    }
+
+    @Override
+    public long excessHeapSizeExcludingData()
+    {
+        return 0;
+    }
+
+    @Override
+    public long unsharedHeapSize()
+    {
+        return 0;
+    }
+
+    @Override
+    public CellName copy(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/4b54b8ac/src/java/org/apache/cassandra/db/context/CounterContext.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/context/CounterContext.java b/src/java/org/apache/cassandra/db/context/CounterContext.java
index d759f6d..0b66d3b 100644
--- a/src/java/org/apache/cassandra/db/context/CounterContext.java
+++ b/src/java/org/apache/cassandra/db/context/CounterContext.java
@@ -22,6 +22,8 @@ import java.security.MessageDigest;
 import java.util.ArrayList;
 import java.util.List;
 
+import org.apache.cassandra.serializers.MarshalException;
+import org.apache.cassandra.utils.memory.AbstractAllocator;
 import com.google.common.annotations.VisibleForTesting;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -102,7 +104,7 @@ public class CounterContext
     /**
      * Creates a counter context with a single global, 2.1+ shard (a result of increment).
      */
-    public ByteBuffer createGlobal(CounterId id, long clock, long count, Allocator allocator)
+    public ByteBuffer createGlobal(CounterId id, long clock, long count, AbstractAllocator allocator)
     {
         ContextState state = ContextState.allocate(1, 0, 0, allocator);
         state.writeGlobal(id, clock, count);
@@ -113,7 +115,7 @@ public class CounterContext
      * Creates a counter context with a single local shard.
      * For use by tests of compatibility with pre-2.1 counters only.
      */
-    public ByteBuffer createLocal(long count, Allocator allocator)
+    public ByteBuffer createLocal(long count, AbstractAllocator allocator)
     {
         ContextState state = ContextState.allocate(0, 1, 0, allocator);
         state.writeLocal(CounterId.getLocalId(), 1L, count);
@@ -124,7 +126,7 @@ public class CounterContext
      * Creates a counter context with a single remote shard.
      * For use by tests of compatibility with pre-2.1 counters only.
      */
-    public ByteBuffer createRemote(CounterId id, long clock, long count, Allocator allocator)
+    public ByteBuffer createRemote(CounterId id, long clock, long count, AbstractAllocator allocator)
     {
         ContextState state = ContextState.allocate(0, 0, 1, allocator);
         state.writeRemote(id, clock, count);
@@ -255,7 +257,7 @@ public class CounterContext
      * @param right counter context.
      * @param allocator An allocator for the merged value.
      */
-    public ByteBuffer merge(ByteBuffer left, ByteBuffer right, Allocator allocator)
+    public ByteBuffer merge(ByteBuffer left, ByteBuffer right, AbstractAllocator allocator)
     {
         int globalCount = 0;
         int localCount = 0;
@@ -699,7 +701,7 @@ public class CounterContext
          * Allocate a new context big enough for globalCount + localCount + remoteCount elements
          * and return the initial corresponding ContextState.
          */
-        public static ContextState allocate(int globalCount, int localCount, int remoteCount, Allocator allocator)
+        public static ContextState allocate(int globalCount, int localCount, int remoteCount, AbstractAllocator allocator)
         {
             int headerLength = HEADER_SIZE_LENGTH + (globalCount + localCount) * HEADER_ELT_LENGTH;
             int bodyLength = (globalCount + localCount + remoteCount) * STEP_LENGTH;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/src/java/org/apache/cassandra/db/context/IContext.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/context/IContext.java b/src/java/org/apache/cassandra/db/context/IContext.java
new file mode 100644
index 0000000..df484e7
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/context/IContext.java
@@ -0,0 +1,75 @@
+/*
+ * 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.context;
+
+import java.nio.ByteBuffer;
+
+import org.apache.cassandra.utils.memory.AbstractAllocator;
+
+/**
+ * An opaque commutative context.
+ *
+ * Maintains a ByteBuffer context that represents a partitioned commutative value.
+ */
+public interface IContext
+{
+    public static enum ContextRelationship
+    {
+        EQUAL,
+        GREATER_THAN,
+        LESS_THAN,
+        DISJOINT
+    };
+
+    /**
+     * Determine the relationship between two contexts.
+     *
+     * EQUAL:        Equal set of nodes and every count is equal.
+     * GREATER_THAN: Superset of nodes and every count is equal or greater than its corollary.
+     * LESS_THAN:    Subset of nodes and every count is equal or less than its corollary.
+     * DISJOINT:     Node sets are not equal and/or counts are not all greater or less than.
+     *
+     * @param left
+     *            context.
+     * @param right
+     *            context.
+     * @return the ContextRelationship between the contexts.
+     */
+    public ContextRelationship diff(ByteBuffer left, ByteBuffer right);
+
+    /**
+     * Return a context w/ an aggregated count for each node id.
+     *
+     * @param left
+     *            context.
+     * @param right
+     *            context.
+     * @param allocator
+     *            an allocator to allocate the new context from.
+     */
+    public ByteBuffer merge(ByteBuffer left, ByteBuffer right, AbstractAllocator allocator);
+
+    /**
+     * Human-readable String from context.
+     *
+     * @param context
+     *            context.
+     * @return a human-readable String of the context.
+     */
+    public String toString(ByteBuffer context);
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/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 0e096fe..f5ea49a 100644
--- a/src/java/org/apache/cassandra/db/filter/ColumnSlice.java
+++ b/src/java/org/apache/cassandra/db/filter/ColumnSlice.java
@@ -34,7 +34,8 @@ import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.io.ISerializer;
 import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.Allocator;
+import org.apache.cassandra.utils.memory.AbstractAllocator;
+import org.apache.cassandra.utils.memory.PoolAllocator;
 
 public class ColumnSlice
 {
@@ -298,12 +299,24 @@ public class ColumnSlice
             throw new UnsupportedOperationException();
         }
 
-        public CellName copy(Allocator allocator)
+        public CellName copy(AbstractAllocator allocator)
         {
             throw new UnsupportedOperationException();
         }
 
-        public long memorySize()
+        @Override
+        public long excessHeapSizeExcludingData()
+        {
+            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/4b54b8ac/src/java/org/apache/cassandra/db/filter/QueryFilter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/filter/QueryFilter.java b/src/java/org/apache/cassandra/db/filter/QueryFilter.java
index 8beb7ae..53a2180 100644
--- a/src/java/org/apache/cassandra/db/filter/QueryFilter.java
+++ b/src/java/org/apache/cassandra/db/filter/QueryFilter.java
@@ -26,7 +26,7 @@ import org.apache.cassandra.db.composites.CellName;
 import org.apache.cassandra.db.composites.Composite;
 import org.apache.cassandra.io.sstable.SSTableReader;
 import org.apache.cassandra.io.util.FileDataInput;
-import org.apache.cassandra.utils.HeapAllocator;
+import org.apache.cassandra.utils.memory.HeapAllocator;
 import org.apache.cassandra.utils.MergeIterator;
 
 public class QueryFilter

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/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 5aaf07c..5987d7a 100644
--- a/src/java/org/apache/cassandra/db/index/AbstractSimplePerColumnSecondaryIndex.java
+++ b/src/java/org/apache/cassandra/db/index/AbstractSimplePerColumnSecondaryIndex.java
@@ -18,7 +18,10 @@
 package org.apache.cassandra.db.index;
 
 import java.nio.ByteBuffer;
+import java.util.concurrent.Future;
 
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.concurrent.OpOrder;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.db.*;
@@ -27,6 +30,7 @@ import org.apache.cassandra.db.composites.CellNameType;
 import org.apache.cassandra.db.marshal.*;
 import org.apache.cassandra.dht.*;
 import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.memory.AbstractAllocator;
 
 /**
  * Implements a secondary index for a column family using a second column family
@@ -84,6 +88,21 @@ public abstract class AbstractSimplePerColumnSecondaryIndex extends PerColumnSec
 
     public void delete(ByteBuffer rowKey, Cell cell)
     {
+        throw new IllegalStateException();
+    }
+
+    public void insert(ByteBuffer rowKey, Cell cell)
+    {
+        throw new IllegalStateException();
+    }
+
+    public void update(ByteBuffer rowKey, Cell cell)
+    {
+        throw new IllegalStateException();
+    }
+
+    public void delete(ByteBuffer rowKey, Cell cell, OpOrder.Group opGroup)
+    {
         if (cell.isMarkedForDelete(System.currentTimeMillis()))
             return;
 
@@ -91,12 +110,12 @@ public abstract class AbstractSimplePerColumnSecondaryIndex extends PerColumnSec
         int localDeletionTime = (int) (System.currentTimeMillis() / 1000);
         ColumnFamily cfi = ArrayBackedSortedColumns.factory.create(indexCfs.metadata);
         cfi.addTombstone(makeIndexColumnName(rowKey, cell), localDeletionTime, cell.timestamp());
-        indexCfs.apply(valueKey, cfi, SecondaryIndexManager.nullUpdater);
+        indexCfs.apply(valueKey, cfi, SecondaryIndexManager.nullUpdater, opGroup, null);
         if (logger.isDebugEnabled())
             logger.debug("removed index entry for cleaned-up value {}:{}", valueKey, cfi);
     }
 
-    public void insert(ByteBuffer rowKey, Cell cell)
+    public void insert(ByteBuffer rowKey, Cell cell, OpOrder.Group opGroup)
     {
         DecoratedKey valueKey = getIndexKeyFor(getIndexedValue(rowKey, cell));
         ColumnFamily cfi = ArrayBackedSortedColumns.factory.create(indexCfs.metadata);
@@ -113,12 +132,12 @@ public abstract class AbstractSimplePerColumnSecondaryIndex extends PerColumnSec
         if (logger.isDebugEnabled())
             logger.debug("applying index row {} in {}", indexCfs.metadata.getKeyValidator().getString(valueKey.key), cfi);
 
-        indexCfs.apply(valueKey, cfi, SecondaryIndexManager.nullUpdater);
+        indexCfs.apply(valueKey, cfi, SecondaryIndexManager.nullUpdater, opGroup, null);
     }
 
-    public void update(ByteBuffer rowKey, Cell col)
+    public void update(ByteBuffer rowKey, Cell col, OpOrder.Group opGroup)
     {
-        insert(rowKey, col);
+        insert(rowKey, col, opGroup);
     }
 
     public void removeIndex(ByteBuffer columnName)
@@ -128,7 +147,13 @@ public abstract class AbstractSimplePerColumnSecondaryIndex extends PerColumnSec
 
     public void forceBlockingFlush()
     {
-        indexCfs.forceBlockingFlush();
+        Future<?> wait;
+        // we synchronise on the baseCfs to make sure we are ordered correctly with other flushes to the base CFS
+        synchronized (baseCfs.getDataTracker())
+        {
+            wait = indexCfs.forceFlush();
+        }
+        FBUtilities.waitOnFuture(wait);
     }
 
     public void invalidate()
@@ -151,9 +176,9 @@ public abstract class AbstractSimplePerColumnSecondaryIndex extends PerColumnSec
         return indexCfs.name;
     }
 
-    public long getLiveSize()
+    public AbstractAllocator getOnHeapAllocator()
     {
-        return indexCfs.getMemtableDataSize();
+        return indexCfs.getDataTracker().getView().getCurrentMemtable().getAllocator();
     }
 
     public void reload()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/src/java/org/apache/cassandra/db/index/PerColumnSecondaryIndex.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/index/PerColumnSecondaryIndex.java b/src/java/org/apache/cassandra/db/index/PerColumnSecondaryIndex.java
index 15138ea..e094c4c 100644
--- a/src/java/org/apache/cassandra/db/index/PerColumnSecondaryIndex.java
+++ b/src/java/org/apache/cassandra/db/index/PerColumnSecondaryIndex.java
@@ -19,6 +19,7 @@ package org.apache.cassandra.db.index;
 
 import java.nio.ByteBuffer;
 
+import org.apache.cassandra.utils.concurrent.OpOrder;
 import org.apache.cassandra.db.Cell;
 import org.apache.cassandra.utils.FBUtilities;
 
@@ -29,12 +30,12 @@ import org.apache.cassandra.utils.FBUtilities;
 public abstract class PerColumnSecondaryIndex extends SecondaryIndex
 {
     /**
-     * Delete a column from the index
+     * Delete a column from the index.
      *
      * @param rowKey the underlying row key which is indexed
      * @param col all the column info
      */
-    public abstract void delete(ByteBuffer rowKey, Cell col);
+    public abstract void delete(ByteBuffer rowKey, Cell col, OpOrder.Group opGroup);
 
     /**
      * insert a column to the index
@@ -42,7 +43,7 @@ public abstract class PerColumnSecondaryIndex extends SecondaryIndex
      * @param rowKey the underlying row key which is indexed
      * @param col all the column info
      */
-    public abstract void insert(ByteBuffer rowKey, Cell col);
+    public abstract void insert(ByteBuffer rowKey, Cell col, OpOrder.Group opGroup);
 
     /**
      * update a column from the index
@@ -50,14 +51,13 @@ public abstract class PerColumnSecondaryIndex extends SecondaryIndex
      * @param rowKey the underlying row key which is indexed
      * @param col all the column info
      */
-    public abstract void update(ByteBuffer rowKey, Cell col);
+    public abstract void update(ByteBuffer rowKey, Cell col, OpOrder.Group opGroup);
 
     public String getNameForSystemKeyspace(ByteBuffer column)
     {
         return getIndexName();
     }
 
-    @Override
     public boolean validate(Cell cell)
     {
         return cell.value().remaining() < FBUtilities.MAX_UNSIGNED_SHORT;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/src/java/org/apache/cassandra/db/index/PerRowSecondaryIndex.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/index/PerRowSecondaryIndex.java b/src/java/org/apache/cassandra/db/index/PerRowSecondaryIndex.java
index 7565c29..d73d056 100644
--- a/src/java/org/apache/cassandra/db/index/PerRowSecondaryIndex.java
+++ b/src/java/org/apache/cassandra/db/index/PerRowSecondaryIndex.java
@@ -20,6 +20,7 @@ package org.apache.cassandra.db.index;
 import java.nio.ByteBuffer;
 import java.nio.charset.CharacterCodingException;
 
+import org.apache.cassandra.utils.concurrent.OpOrder;
 import org.apache.cassandra.db.Cell;
 import org.apache.cassandra.db.ColumnFamily;
 import org.apache.cassandra.db.DecoratedKey;
@@ -43,9 +44,8 @@ public abstract class PerRowSecondaryIndex extends SecondaryIndex
      *
      * @param key
      */
-    public abstract void delete(DecoratedKey key);
+    public abstract void delete(DecoratedKey key, OpOrder.Group opGroup);
 
-    @Override
     public String getNameForSystemKeyspace(ByteBuffer columnName)
     {
         try
@@ -58,7 +58,6 @@ public abstract class PerRowSecondaryIndex extends SecondaryIndex
         }
     }
 
-    @Override
     public boolean validate(Cell cell)
     {
         return true;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/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 7edfd94..7d9338a 100644
--- a/src/java/org/apache/cassandra/db/index/SecondaryIndex.java
+++ b/src/java/org/apache/cassandra/db/index/SecondaryIndex.java
@@ -21,6 +21,7 @@ import java.nio.ByteBuffer;
 import java.util.*;
 import java.util.concurrent.*;
 
+import org.apache.cassandra.utils.memory.AbstractAllocator;
 import org.apache.commons.lang3.StringUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -137,14 +138,14 @@ public abstract class SecondaryIndex
     protected abstract SecondaryIndexSearcher createSecondaryIndexSearcher(Set<ByteBuffer> columns);
 
     /**
-     * Forces this indexes in memory data to disk
+     * Forces this indexes' in memory data to disk
      */
     public abstract void forceBlockingFlush();
 
     /**
      * Get current amount of memory this index is consuming (in bytes)
      */
-    public abstract long getLiveSize();
+    public abstract AbstractAllocator getOnHeapAllocator();
 
     /**
      * Allow access to the underlying column family store if there is one


[6/6] git commit: remove Table.switchlock and introduce o.a.c.utils.memory package patch by Benedict Elliott Smith; reviewed by jbellis for CASSANDRA-5549

Posted by jb...@apache.org.
remove Table.switchlock and introduce o.a.c.utils.memory package
patch by Benedict Elliott Smith; reviewed by jbellis for CASSANDRA-5549


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

Branch: refs/heads/trunk
Commit: 4b54b8acd21999ad4394feb93deb7cca1de445c0
Parents: 00fc318
Author: Jonathan Ellis <jb...@apache.org>
Authored: Thu Jan 30 17:08:46 2014 -0600
Committer: Jonathan Ellis <jb...@apache.org>
Committed: Thu Jan 30 17:09:18 2014 -0600

----------------------------------------------------------------------
 build.xml                                       |   6 +-
 conf/cassandra-env.sh                           |   2 +-
 conf/cassandra.yaml                             |   9 +-
 lib/jamm-0.2.5.jar                              | Bin 5792 -> 0 bytes
 lib/jamm-0.2.6.jar                              | Bin 0 -> 13290 bytes
 lib/licenses/jamm-0.2.5.txt                     | 202 --------
 lib/licenses/jamm-0.2.6.txt                     | 202 ++++++++
 .../cache/ConcurrentLinkedHashCache.java        |   2 +-
 .../apache/cassandra/cache/CounterCacheKey.java |  14 +-
 .../cassandra/cache/IMeasurableMemory.java      |   7 +-
 .../org/apache/cassandra/cache/KeyCacheKey.java |  12 +-
 .../org/apache/cassandra/cache/RowCacheKey.java |  10 +-
 .../cassandra/cache/RowCacheSentinel.java       |   6 -
 .../org/apache/cassandra/config/Config.java     |   4 +-
 .../cassandra/config/DatabaseDescriptor.java    |  38 +-
 .../apache/cassandra/cql3/ColumnIdentifier.java |  32 +-
 .../apache/cassandra/cql3/QueryProcessor.java   |  35 +-
 .../cassandra/db/ArrayBackedSortedColumns.java  |  10 +-
 .../apache/cassandra/db/AtomicBTreeColumns.java | 174 ++++---
 src/java/org/apache/cassandra/db/Cell.java      |  21 +-
 .../org/apache/cassandra/db/ClockAndCount.java  |   8 +-
 .../cassandra/db/CollationController.java       |   2 +-
 .../org/apache/cassandra/db/ColumnFamily.java   |  11 +-
 .../apache/cassandra/db/ColumnFamilyStore.java  | 502 +++++++++++-------
 .../cassandra/db/ColumnFamilyStoreMBean.java    |   4 +-
 .../org/apache/cassandra/db/CounterCell.java    |  10 +-
 .../apache/cassandra/db/CounterMutation.java    |   5 +-
 .../apache/cassandra/db/CounterUpdateCell.java  |   8 +-
 .../org/apache/cassandra/db/DataTracker.java    | 182 ++++---
 .../org/apache/cassandra/db/DeletedCell.java    |   7 +-
 .../org/apache/cassandra/db/DeletionInfo.java   |  12 +-
 .../org/apache/cassandra/db/DeletionTime.java   |  10 +-
 .../org/apache/cassandra/db/EmptyColumns.java   |   6 +-
 .../org/apache/cassandra/db/ExpiringCell.java   |   5 +-
 src/java/org/apache/cassandra/db/Keyspace.java  |  42 +-
 src/java/org/apache/cassandra/db/Memtable.java  | 340 ++++++-------
 .../org/apache/cassandra/db/MeteredFlusher.java | 131 -----
 .../apache/cassandra/db/RangeTombstoneList.java |  33 +-
 .../org/apache/cassandra/db/RowIndexEntry.java  |  27 +-
 .../db/TreeMapBackedSortedColumns.java          |   7 +-
 .../apache/cassandra/db/UnsortedColumns.java    |   7 +-
 .../db/commitlog/AbstractCommitLogService.java  |   8 +-
 .../cassandra/db/commitlog/CommitLog.java       |  21 +-
 .../db/commitlog/CommitLogSegment.java          | 166 +++---
 .../db/commitlog/CommitLogSegmentManager.java   |  65 ++-
 .../db/commitlog/PeriodicCommitLogService.java  |   6 +-
 .../cassandra/db/commitlog/ReplayPosition.java  |   5 +
 .../db/compaction/CompactionManager.java        |   9 +-
 .../cassandra/db/compaction/CompactionTask.java |  14 +-
 .../db/compaction/LazilyCompactedRow.java       |   4 +-
 .../db/composites/BoundedComposite.java         |  19 +-
 .../cassandra/db/composites/CellName.java       |   6 +-
 .../cassandra/db/composites/Composite.java      |   6 +-
 .../cassandra/db/composites/Composites.java     |  13 +-
 .../db/composites/CompoundComposite.java        |  28 +-
 .../db/composites/CompoundDenseCellName.java    |  18 +-
 .../db/composites/CompoundSparseCellName.java   |  37 +-
 .../db/composites/SimpleComposite.java          |  17 +-
 .../db/composites/SimpleDenseCellName.java      |  17 +-
 .../db/composites/SimpleSparseCellName.java     |  26 +-
 .../db/composites/SimpleSparseCellNameType.java |   2 +-
 .../SimpleSparseInternedCellName.java           |  58 +++
 .../cassandra/db/context/CounterContext.java    |  12 +-
 .../apache/cassandra/db/context/IContext.java   |  75 +++
 .../apache/cassandra/db/filter/ColumnSlice.java |  19 +-
 .../apache/cassandra/db/filter/QueryFilter.java |   2 +-
 .../AbstractSimplePerColumnSecondaryIndex.java  |  41 +-
 .../db/index/PerColumnSecondaryIndex.java       |  10 +-
 .../db/index/PerRowSecondaryIndex.java          |   5 +-
 .../cassandra/db/index/SecondaryIndex.java      |   5 +-
 .../db/index/SecondaryIndexManager.java         | 117 ++++-
 .../db/index/composites/CompositesIndex.java    |  23 +-
 .../cassandra/db/index/keys/KeysSearcher.java   |  14 +-
 .../cassandra/dht/ByteOrderedPartitioner.java   |  10 +
 .../org/apache/cassandra/dht/IPartitioner.java  |   7 +
 .../apache/cassandra/dht/LocalPartitioner.java  |   8 +
 .../cassandra/dht/Murmur3Partitioner.java       |   8 +
 .../dht/OrderPreservingPartitioner.java         |   8 +
 .../apache/cassandra/dht/RandomPartitioner.java |   8 +
 .../io/sstable/AbstractSSTableSimpleWriter.java |   2 +-
 .../cassandra/io/sstable/IndexHelper.java       |  12 +-
 .../apache/cassandra/io/sstable/SSTable.java    |   4 +-
 .../io/sstable/metadata/MetadataCollector.java  |   1 +
 .../cassandra/io/util/SequentialWriter.java     |  17 +-
 .../cassandra/metrics/ColumnFamilyMetrics.java  |  61 ++-
 .../cassandra/metrics/CommitLogMetrics.java     |   9 +
 .../cassandra/service/CassandraDaemon.java      |   5 -
 .../org/apache/cassandra/utils/Allocator.java   |  41 --
 .../org/apache/cassandra/utils/FBUtilities.java |  12 +-
 .../apache/cassandra/utils/HeapAllocator.java   |  41 --
 .../org/apache/cassandra/utils/ObjectSizes.java | 247 +++------
 .../apache/cassandra/utils/SlabAllocator.java   | 220 --------
 .../org/apache/cassandra/utils/WaitQueue.java   | 264 ----------
 .../org/apache/cassandra/utils/btree/BTree.java |  48 +-
 .../apache/cassandra/utils/btree/BTreeSet.java  |   2 +-
 .../apache/cassandra/utils/btree/Builder.java   |  17 +-
 .../cassandra/utils/btree/NodeBuilder.java      |  55 +-
 .../cassandra/utils/btree/ReplaceFunction.java  |  17 -
 .../cassandra/utils/btree/UpdateFunction.java   |  30 ++
 .../cassandra/utils/concurrent/OpOrder.java     | 411 +++++++++++++++
 .../cassandra/utils/concurrent/WaitQueue.java   | 508 +++++++++++++++++++
 .../utils/memory/AbstractAllocator.java         |  62 +++
 .../utils/memory/ContextAllocator.java          |  59 +++
 .../cassandra/utils/memory/HeapAllocator.java   |  38 ++
 .../apache/cassandra/utils/memory/HeapPool.java |  16 +
 .../utils/memory/HeapPoolAllocator.java         |  47 ++
 .../utils/memory/HeapSlabAllocator.java         | 204 ++++++++
 .../cassandra/utils/memory/HeapSlabPool.java    |  16 +
 .../org/apache/cassandra/utils/memory/Pool.java | 140 +++++
 .../cassandra/utils/memory/PoolAllocator.java   | 175 +++++++
 .../utils/memory/PoolCleanerThread.java         |  55 ++
 .../cassandra/concurrent/LongOpOrderTest.java   | 223 ++++++++
 .../cassandra/db/LongFlushMemtableTest.java     |  72 +++
 .../apache/cassandra/db/MeteredFlusherTest.java |  72 ---
 .../apache/cassandra/utils/LongBTreeTest.java   |   6 +-
 .../cassandra/cache/CacheProviderTest.java      |   2 +-
 .../apache/cassandra/cache/ObjectSizeTest.java  | 105 ----
 .../cassandra/concurrent/WaitQueueTest.java     |   2 +-
 .../db/ArrayBackedSortedColumnsTest.java        |   2 +-
 .../apache/cassandra/db/ColumnFamilyTest.java   |   2 +-
 .../org/apache/cassandra/db/CommitLogTest.java  |   6 +-
 .../apache/cassandra/db/CounterCellTest.java    |  10 +-
 .../apache/cassandra/db/KeyCollisionTest.java   |   6 +
 .../apache/cassandra/db/RangeTombstoneTest.java |  16 +-
 .../db/SecondaryIndexCellSizeTest.java          |  31 +-
 .../db/context/CounterContextTest.java          |  21 +-
 .../db/index/PerRowSecondaryIndexTest.java      |   9 +-
 .../streaming/StreamingTransferTest.java        |   2 +-
 128 files changed, 4112 insertions(+), 2373 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/build.xml
----------------------------------------------------------------------
diff --git a/build.xml b/build.xml
index de477aa..a033874 100644
--- a/build.xml
+++ b/build.xml
@@ -357,7 +357,7 @@
           </dependency>
           <dependency groupId="com.googlecode.json-simple" artifactId="json-simple" version="1.1"/>
           <dependency groupId="com.github.stephenc.high-scale-lib" artifactId="high-scale-lib" version="1.1.2"/>
-          <dependency groupId="com.github.stephenc" artifactId="jamm" version="0.2.5"/>
+          <dependency groupId="com.github.stephenc" artifactId="jamm" version="0.2.6"/>
           <dependency groupId="com.thinkaurelius.thrift" artifactId="thrift-server" version="0.3.3">
 	      	<exclusion groupId="org.slf4j" artifactId="slf4j-log4j12"/>
           </dependency>
@@ -656,7 +656,7 @@
           <pathelement location="${test.conf}"/>
         </classpath>
         <jvmarg value="-Dstorage-config=${test.conf}"/>
-        <jvmarg value="-javaagent:${basedir}/lib/jamm-0.2.5.jar" />
+        <jvmarg value="-javaagent:${basedir}/lib/jamm-0.2.6.jar" />
         <jvmarg value="-ea"/>
       </java>
     </target>
@@ -1051,7 +1051,7 @@
         <formatter type="brief" usefile="false"/>
         <jvmarg value="-Dstorage-config=${test.conf}"/>
         <jvmarg value="-Djava.awt.headless=true"/>
-        <jvmarg value="-javaagent:${basedir}/lib/jamm-0.2.5.jar" />
+        <jvmarg value="-javaagent:${basedir}/lib/jamm-0.2.6.jar" />
         <jvmarg value="-ea"/>
         <jvmarg value="-Xss256k"/>
         <optjvmargs/>

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/conf/cassandra-env.sh
----------------------------------------------------------------------
diff --git a/conf/cassandra-env.sh b/conf/cassandra-env.sh
index 78a39f4..20f26da 100644
--- a/conf/cassandra-env.sh
+++ b/conf/cassandra-env.sh
@@ -165,7 +165,7 @@ JVM_OPTS="$JVM_OPTS -ea"
 if [ "$JVM_VENDOR" != "OpenJDK" -o "$JVM_VERSION" \> "1.6.0" ] \
       || [ "$JVM_VERSION" = "1.6.0" -a "$JVM_PATCH_VERSION" -ge 23 ]
 then
-    JVM_OPTS="$JVM_OPTS -javaagent:$CASSANDRA_HOME/lib/jamm-0.2.5.jar"
+    JVM_OPTS="$JVM_OPTS -javaagent:$CASSANDRA_HOME/lib/jamm-0.2.6.jar"
 fi
 
 # enable thread priorities, primarily so we can give periodic tasks

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/conf/cassandra.yaml
----------------------------------------------------------------------
diff --git a/conf/cassandra.yaml b/conf/cassandra.yaml
index 06cb33f..8ddd5e0 100644
--- a/conf/cassandra.yaml
+++ b/conf/cassandra.yaml
@@ -280,6 +280,10 @@ concurrent_counter_writes: 32
 # If omitted, Cassandra will set it to 1/4 of the heap.
 # memtable_total_space_in_mb: 2048
 
+# Ratio of occupied non-flushing memtable size to total permitted size
+# that will trigger a flush of the largest memtable.
+memtable_cleanup_threshold: 0.4
+
 # Total space to use for commitlogs.  Since commitlog segments are
 # mmapped, and hence use up address space, the default size is 32
 # on 32-bit JVMs, and 1024 on 64-bit JVMs.
@@ -297,11 +301,6 @@ concurrent_counter_writes: 32
 # By default this will be set to the amount of data directories defined.
 #memtable_flush_writers: 1
 
-# the number of full memtables to allow pending flush, that is,
-# waiting for a writer thread.  At a minimum, this should be set to
-# the maximum number of secondary indexes created on a single CF.
-memtable_flush_queue_size: 4
-
 # A fixed memory pool size in MB for for SSTable index summaries. If left
 # empty, this will default to 5% of the heap size. If the memory usage of
 # all index summaries exceeds this limit, SSTables with low read rates will

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/lib/jamm-0.2.5.jar
----------------------------------------------------------------------
diff --git a/lib/jamm-0.2.5.jar b/lib/jamm-0.2.5.jar
deleted file mode 100644
index ef8750d..0000000
Binary files a/lib/jamm-0.2.5.jar and /dev/null differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/lib/jamm-0.2.6.jar
----------------------------------------------------------------------
diff --git a/lib/jamm-0.2.6.jar b/lib/jamm-0.2.6.jar
new file mode 100644
index 0000000..04e5c2e
Binary files /dev/null and b/lib/jamm-0.2.6.jar differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/lib/licenses/jamm-0.2.5.txt
----------------------------------------------------------------------
diff --git a/lib/licenses/jamm-0.2.5.txt b/lib/licenses/jamm-0.2.5.txt
deleted file mode 100644
index d645695..0000000
--- a/lib/licenses/jamm-0.2.5.txt
+++ /dev/null
@@ -1,202 +0,0 @@
-
-                                 Apache License
-                           Version 2.0, January 2004
-                        http://www.apache.org/licenses/
-
-   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
-
-   1. Definitions.
-
-      "License" shall mean the terms and conditions for use, reproduction,
-      and distribution as defined by Sections 1 through 9 of this document.
-
-      "Licensor" shall mean the copyright owner or entity authorized by
-      the copyright owner that is granting the License.
-
-      "Legal Entity" shall mean the union of the acting entity and all
-      other entities that control, are controlled by, or are under common
-      control with that entity. For the purposes of this definition,
-      "control" means (i) the power, direct or indirect, to cause the
-      direction or management of such entity, whether by contract or
-      otherwise, or (ii) ownership of fifty percent (50%) or more of the
-      outstanding shares, or (iii) beneficial ownership of such entity.
-
-      "You" (or "Your") shall mean an individual or Legal Entity
-      exercising permissions granted by this License.
-
-      "Source" form shall mean the preferred form for making modifications,
-      including but not limited to software source code, documentation
-      source, and configuration files.
-
-      "Object" form shall mean any form resulting from mechanical
-      transformation or translation of a Source form, including but
-      not limited to compiled object code, generated documentation,
-      and conversions to other media types.
-
-      "Work" shall mean the work of authorship, whether in Source or
-      Object form, made available under the License, as indicated by a
-      copyright notice that is included in or attached to the work
-      (an example is provided in the Appendix below).
-
-      "Derivative Works" shall mean any work, whether in Source or Object
-      form, that is based on (or derived from) the Work and for which the
-      editorial revisions, annotations, elaborations, or other modifications
-      represent, as a whole, an original work of authorship. For the purposes
-      of this License, Derivative Works shall not include works that remain
-      separable from, or merely link (or bind by name) to the interfaces of,
-      the Work and Derivative Works thereof.
-
-      "Contribution" shall mean any work of authorship, including
-      the original version of the Work and any modifications or additions
-      to that Work or Derivative Works thereof, that is intentionally
-      submitted to Licensor for inclusion in the Work by the copyright owner
-      or by an individual or Legal Entity authorized to submit on behalf of
-      the copyright owner. For the purposes of this definition, "submitted"
-      means any form of electronic, verbal, or written communication sent
-      to the Licensor or its representatives, including but not limited to
-      communication on electronic mailing lists, source code control systems,
-      and issue tracking systems that are managed by, or on behalf of, the
-      Licensor for the purpose of discussing and improving the Work, but
-      excluding communication that is conspicuously marked or otherwise
-      designated in writing by the copyright owner as "Not a Contribution."
-
-      "Contributor" shall mean Licensor and any individual or Legal Entity
-      on behalf of whom a Contribution has been received by Licensor and
-      subsequently incorporated within the Work.
-
-   2. Grant of Copyright License. Subject to the terms and conditions of
-      this License, each Contributor hereby grants to You a perpetual,
-      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
-      copyright license to reproduce, prepare Derivative Works of,
-      publicly display, publicly perform, sublicense, and distribute the
-      Work and such Derivative Works in Source or Object form.
-
-   3. Grant of Patent License. Subject to the terms and conditions of
-      this License, each Contributor hereby grants to You a perpetual,
-      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
-      (except as stated in this section) patent license to make, have made,
-      use, offer to sell, sell, import, and otherwise transfer the Work,
-      where such license applies only to those patent claims licensable
-      by such Contributor that are necessarily infringed by their
-      Contribution(s) alone or by combination of their Contribution(s)
-      with the Work to which such Contribution(s) was submitted. If You
-      institute patent litigation against any entity (including a
-      cross-claim or counterclaim in a lawsuit) alleging that the Work
-      or a Contribution incorporated within the Work constitutes direct
-      or contributory patent infringement, then any patent licenses
-      granted to You under this License for that Work shall terminate
-      as of the date such litigation is filed.
-
-   4. Redistribution. You may reproduce and distribute copies of the
-      Work or Derivative Works thereof in any medium, with or without
-      modifications, and in Source or Object form, provided that You
-      meet the following conditions:
-
-      (a) You must give any other recipients of the Work or
-          Derivative Works a copy of this License; and
-
-      (b) You must cause any modified files to carry prominent notices
-          stating that You changed the files; and
-
-      (c) You must retain, in the Source form of any Derivative Works
-          that You distribute, all copyright, patent, trademark, and
-          attribution notices from the Source form of the Work,
-          excluding those notices that do not pertain to any part of
-          the Derivative Works; and
-
-      (d) If the Work includes a "NOTICE" text file as part of its
-          distribution, then any Derivative Works that You distribute must
-          include a readable copy of the attribution notices contained
-          within such NOTICE file, excluding those notices that do not
-          pertain to any part of the Derivative Works, in at least one
-          of the following places: within a NOTICE text file distributed
-          as part of the Derivative Works; within the Source form or
-          documentation, if provided along with the Derivative Works; or,
-          within a display generated by the Derivative Works, if and
-          wherever such third-party notices normally appear. The contents
-          of the NOTICE file are for informational purposes only and
-          do not modify the License. You may add Your own attribution
-          notices within Derivative Works that You distribute, alongside
-          or as an addendum to the NOTICE text from the Work, provided
-          that such additional attribution notices cannot be construed
-          as modifying the License.
-
-      You may add Your own copyright statement to Your modifications and
-      may provide additional or different license terms and conditions
-      for use, reproduction, or distribution of Your modifications, or
-      for any such Derivative Works as a whole, provided Your use,
-      reproduction, and distribution of the Work otherwise complies with
-      the conditions stated in this License.
-
-   5. Submission of Contributions. Unless You explicitly state otherwise,
-      any Contribution intentionally submitted for inclusion in the Work
-      by You to the Licensor shall be under the terms and conditions of
-      this License, without any additional terms or conditions.
-      Notwithstanding the above, nothing herein shall supersede or modify
-      the terms of any separate license agreement you may have executed
-      with Licensor regarding such Contributions.
-
-   6. Trademarks. This License does not grant permission to use the trade
-      names, trademarks, service marks, or product names of the Licensor,
-      except as required for reasonable and customary use in describing the
-      origin of the Work and reproducing the content of the NOTICE file.
-
-   7. Disclaimer of Warranty. Unless required by applicable law or
-      agreed to in writing, Licensor provides the Work (and each
-      Contributor provides its Contributions) on an "AS IS" BASIS,
-      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
-      implied, including, without limitation, any warranties or conditions
-      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
-      PARTICULAR PURPOSE. You are solely responsible for determining the
-      appropriateness of using or redistributing the Work and assume any
-      risks associated with Your exercise of permissions under this License.
-
-   8. Limitation of Liability. In no event and under no legal theory,
-      whether in tort (including negligence), contract, or otherwise,
-      unless required by applicable law (such as deliberate and grossly
-      negligent acts) or agreed to in writing, shall any Contributor be
-      liable to You for damages, including any direct, indirect, special,
-      incidental, or consequential damages of any character arising as a
-      result of this License or out of the use or inability to use the
-      Work (including but not limited to damages for loss of goodwill,
-      work stoppage, computer failure or malfunction, or any and all
-      other commercial damages or losses), even if such Contributor
-      has been advised of the possibility of such damages.
-
-   9. Accepting Warranty or Additional Liability. While redistributing
-      the Work or Derivative Works thereof, You may choose to offer,
-      and charge a fee for, acceptance of support, warranty, indemnity,
-      or other liability obligations and/or rights consistent with this
-      License. However, in accepting such obligations, You may act only
-      on Your own behalf and on Your sole responsibility, not on behalf
-      of any other Contributor, and only if You agree to indemnify,
-      defend, and hold each Contributor harmless for any liability
-      incurred by, or claims asserted against, such Contributor by reason
-      of your accepting any such warranty or additional liability.
-
-   END OF TERMS AND CONDITIONS
-
-   APPENDIX: How to apply the Apache License to your work.
-
-      To apply the Apache License to your work, attach the following
-      boilerplate notice, with the fields enclosed by brackets "[]"
-      replaced with your own identifying information. (Don't include
-      the brackets!)  The text should be enclosed in the appropriate
-      comment syntax for the file format. We also recommend that a
-      file or class name and description of purpose be included on the
-      same "printed page" as the copyright notice for easier
-      identification within third-party archives.
-
-   Copyright [yyyy] [name of copyright owner]
-
-   Licensed 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.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/lib/licenses/jamm-0.2.6.txt
----------------------------------------------------------------------
diff --git a/lib/licenses/jamm-0.2.6.txt b/lib/licenses/jamm-0.2.6.txt
new file mode 100644
index 0000000..d645695
--- /dev/null
+++ b/lib/licenses/jamm-0.2.6.txt
@@ -0,0 +1,202 @@
+
+                                 Apache License
+                           Version 2.0, January 2004
+                        http://www.apache.org/licenses/
+
+   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+   1. Definitions.
+
+      "License" shall mean the terms and conditions for use, reproduction,
+      and distribution as defined by Sections 1 through 9 of this document.
+
+      "Licensor" shall mean the copyright owner or entity authorized by
+      the copyright owner that is granting the License.
+
+      "Legal Entity" shall mean the union of the acting entity and all
+      other entities that control, are controlled by, or are under common
+      control with that entity. For the purposes of this definition,
+      "control" means (i) the power, direct or indirect, to cause the
+      direction or management of such entity, whether by contract or
+      otherwise, or (ii) ownership of fifty percent (50%) or more of the
+      outstanding shares, or (iii) beneficial ownership of such entity.
+
+      "You" (or "Your") shall mean an individual or Legal Entity
+      exercising permissions granted by this License.
+
+      "Source" form shall mean the preferred form for making modifications,
+      including but not limited to software source code, documentation
+      source, and configuration files.
+
+      "Object" form shall mean any form resulting from mechanical
+      transformation or translation of a Source form, including but
+      not limited to compiled object code, generated documentation,
+      and conversions to other media types.
+
+      "Work" shall mean the work of authorship, whether in Source or
+      Object form, made available under the License, as indicated by a
+      copyright notice that is included in or attached to the work
+      (an example is provided in the Appendix below).
+
+      "Derivative Works" shall mean any work, whether in Source or Object
+      form, that is based on (or derived from) the Work and for which the
+      editorial revisions, annotations, elaborations, or other modifications
+      represent, as a whole, an original work of authorship. For the purposes
+      of this License, Derivative Works shall not include works that remain
+      separable from, or merely link (or bind by name) to the interfaces of,
+      the Work and Derivative Works thereof.
+
+      "Contribution" shall mean any work of authorship, including
+      the original version of the Work and any modifications or additions
+      to that Work or Derivative Works thereof, that is intentionally
+      submitted to Licensor for inclusion in the Work by the copyright owner
+      or by an individual or Legal Entity authorized to submit on behalf of
+      the copyright owner. For the purposes of this definition, "submitted"
+      means any form of electronic, verbal, or written communication sent
+      to the Licensor or its representatives, including but not limited to
+      communication on electronic mailing lists, source code control systems,
+      and issue tracking systems that are managed by, or on behalf of, the
+      Licensor for the purpose of discussing and improving the Work, but
+      excluding communication that is conspicuously marked or otherwise
+      designated in writing by the copyright owner as "Not a Contribution."
+
+      "Contributor" shall mean Licensor and any individual or Legal Entity
+      on behalf of whom a Contribution has been received by Licensor and
+      subsequently incorporated within the Work.
+
+   2. Grant of Copyright License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      copyright license to reproduce, prepare Derivative Works of,
+      publicly display, publicly perform, sublicense, and distribute the
+      Work and such Derivative Works in Source or Object form.
+
+   3. Grant of Patent License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      (except as stated in this section) patent license to make, have made,
+      use, offer to sell, sell, import, and otherwise transfer the Work,
+      where such license applies only to those patent claims licensable
+      by such Contributor that are necessarily infringed by their
+      Contribution(s) alone or by combination of their Contribution(s)
+      with the Work to which such Contribution(s) was submitted. If You
+      institute patent litigation against any entity (including a
+      cross-claim or counterclaim in a lawsuit) alleging that the Work
+      or a Contribution incorporated within the Work constitutes direct
+      or contributory patent infringement, then any patent licenses
+      granted to You under this License for that Work shall terminate
+      as of the date such litigation is filed.
+
+   4. Redistribution. You may reproduce and distribute copies of the
+      Work or Derivative Works thereof in any medium, with or without
+      modifications, and in Source or Object form, provided that You
+      meet the following conditions:
+
+      (a) You must give any other recipients of the Work or
+          Derivative Works a copy of this License; and
+
+      (b) You must cause any modified files to carry prominent notices
+          stating that You changed the files; and
+
+      (c) You must retain, in the Source form of any Derivative Works
+          that You distribute, all copyright, patent, trademark, and
+          attribution notices from the Source form of the Work,
+          excluding those notices that do not pertain to any part of
+          the Derivative Works; and
+
+      (d) If the Work includes a "NOTICE" text file as part of its
+          distribution, then any Derivative Works that You distribute must
+          include a readable copy of the attribution notices contained
+          within such NOTICE file, excluding those notices that do not
+          pertain to any part of the Derivative Works, in at least one
+          of the following places: within a NOTICE text file distributed
+          as part of the Derivative Works; within the Source form or
+          documentation, if provided along with the Derivative Works; or,
+          within a display generated by the Derivative Works, if and
+          wherever such third-party notices normally appear. The contents
+          of the NOTICE file are for informational purposes only and
+          do not modify the License. You may add Your own attribution
+          notices within Derivative Works that You distribute, alongside
+          or as an addendum to the NOTICE text from the Work, provided
+          that such additional attribution notices cannot be construed
+          as modifying the License.
+
+      You may add Your own copyright statement to Your modifications and
+      may provide additional or different license terms and conditions
+      for use, reproduction, or distribution of Your modifications, or
+      for any such Derivative Works as a whole, provided Your use,
+      reproduction, and distribution of the Work otherwise complies with
+      the conditions stated in this License.
+
+   5. Submission of Contributions. Unless You explicitly state otherwise,
+      any Contribution intentionally submitted for inclusion in the Work
+      by You to the Licensor shall be under the terms and conditions of
+      this License, without any additional terms or conditions.
+      Notwithstanding the above, nothing herein shall supersede or modify
+      the terms of any separate license agreement you may have executed
+      with Licensor regarding such Contributions.
+
+   6. Trademarks. This License does not grant permission to use the trade
+      names, trademarks, service marks, or product names of the Licensor,
+      except as required for reasonable and customary use in describing the
+      origin of the Work and reproducing the content of the NOTICE file.
+
+   7. Disclaimer of Warranty. Unless required by applicable law or
+      agreed to in writing, Licensor provides the Work (and each
+      Contributor provides its Contributions) on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+      implied, including, without limitation, any warranties or conditions
+      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+      PARTICULAR PURPOSE. You are solely responsible for determining the
+      appropriateness of using or redistributing the Work and assume any
+      risks associated with Your exercise of permissions under this License.
+
+   8. Limitation of Liability. In no event and under no legal theory,
+      whether in tort (including negligence), contract, or otherwise,
+      unless required by applicable law (such as deliberate and grossly
+      negligent acts) or agreed to in writing, shall any Contributor be
+      liable to You for damages, including any direct, indirect, special,
+      incidental, or consequential damages of any character arising as a
+      result of this License or out of the use or inability to use the
+      Work (including but not limited to damages for loss of goodwill,
+      work stoppage, computer failure or malfunction, or any and all
+      other commercial damages or losses), even if such Contributor
+      has been advised of the possibility of such damages.
+
+   9. Accepting Warranty or Additional Liability. While redistributing
+      the Work or Derivative Works thereof, You may choose to offer,
+      and charge a fee for, acceptance of support, warranty, indemnity,
+      or other liability obligations and/or rights consistent with this
+      License. However, in accepting such obligations, You may act only
+      on Your own behalf and on Your sole responsibility, not on behalf
+      of any other Contributor, and only if You agree to indemnify,
+      defend, and hold each Contributor harmless for any liability
+      incurred by, or claims asserted against, such Contributor by reason
+      of your accepting any such warranty or additional liability.
+
+   END OF TERMS AND CONDITIONS
+
+   APPENDIX: How to apply the Apache License to your work.
+
+      To apply the Apache License to your work, attach the following
+      boilerplate notice, with the fields enclosed by brackets "[]"
+      replaced with your own identifying information. (Don't include
+      the brackets!)  The text should be enclosed in the appropriate
+      comment syntax for the file format. We also recommend that a
+      file or class name and description of purpose be included on the
+      same "printed page" as the copyright notice for easier
+      identification within third-party archives.
+
+   Copyright [yyyy] [name of copyright owner]
+
+   Licensed 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.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/src/java/org/apache/cassandra/cache/ConcurrentLinkedHashCache.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cache/ConcurrentLinkedHashCache.java b/src/java/org/apache/cassandra/cache/ConcurrentLinkedHashCache.java
index f1e0466..8182447 100644
--- a/src/java/org/apache/cassandra/cache/ConcurrentLinkedHashCache.java
+++ b/src/java/org/apache/cassandra/cache/ConcurrentLinkedHashCache.java
@@ -54,7 +54,7 @@ public class ConcurrentLinkedHashCache<K extends IMeasurableMemory, V extends IM
         {
             public int weightOf(K key, V value)
             {
-                long size = key.memorySize() + value.memorySize();
+                long size = key.unsharedHeapSize() + value.unsharedHeapSize();
                 assert size <= Integer.MAX_VALUE : "Serialized size cannot be more than 2GB/Integer.MAX_VALUE";
                 return (int) size;
             }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/src/java/org/apache/cassandra/cache/CounterCacheKey.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cache/CounterCacheKey.java b/src/java/org/apache/cassandra/cache/CounterCacheKey.java
index acbe323..b6920cf 100644
--- a/src/java/org/apache/cassandra/cache/CounterCacheKey.java
+++ b/src/java/org/apache/cassandra/cache/CounterCacheKey.java
@@ -22,11 +22,17 @@ import java.util.Arrays;
 import java.util.UUID;
 
 import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.cql3.ColumnIdentifier;
 import org.apache.cassandra.db.composites.CellName;
+import org.apache.cassandra.db.composites.CellNames;
+import org.apache.cassandra.db.composites.SimpleSparseCellName;
 import org.apache.cassandra.utils.*;
 
 public class CounterCacheKey implements CacheKey
 {
+    private static final long EMPTY_SIZE = ObjectSizes.measure(new CounterCacheKey(null, ByteBufferUtil.EMPTY_BYTE_BUFFER, CellNames.simpleDense(ByteBuffer.allocate(1))))
+                                           + ObjectSizes.measure(new UUID(0, 0));
+
     public final UUID cfId;
     public final byte[] partitionKey;
     public final byte[] cellName;
@@ -49,11 +55,11 @@ public class CounterCacheKey implements CacheKey
         return new PathInfo(cf.left, cf.right, cfId);
     }
 
-    public long memorySize()
+    public long unsharedHeapSize()
     {
-        return ObjectSizes.getFieldSize(3 * ObjectSizes.getReferenceSize())
-             + ObjectSizes.getArraySize(partitionKey)
-             + ObjectSizes.getArraySize(cellName);
+        return EMPTY_SIZE
+               + ObjectSizes.sizeOfArray(partitionKey)
+               + ObjectSizes.sizeOfArray(cellName);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/src/java/org/apache/cassandra/cache/IMeasurableMemory.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cache/IMeasurableMemory.java b/src/java/org/apache/cassandra/cache/IMeasurableMemory.java
index 16ca7c2..149bff6 100644
--- a/src/java/org/apache/cassandra/cache/IMeasurableMemory.java
+++ b/src/java/org/apache/cassandra/cache/IMeasurableMemory.java
@@ -23,5 +23,10 @@ package org.apache.cassandra.cache;
 
 public interface IMeasurableMemory
 {
-    public long memorySize();
+    /**
+     * @return the amount of on-heap memory retained by the object that might be reclaimed if the object were reclaimed,
+     * i.e. it should try to exclude globally cached data where possible, or counting portions of arrays that are
+     * referenced by the object but used by other objects only (e.g. slabbed byte-buffers), etc.
+     */
+    public long unsharedHeapSize();
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/src/java/org/apache/cassandra/cache/KeyCacheKey.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cache/KeyCacheKey.java b/src/java/org/apache/cassandra/cache/KeyCacheKey.java
index 1133d94..5624133 100644
--- a/src/java/org/apache/cassandra/cache/KeyCacheKey.java
+++ b/src/java/org/apache/cassandra/cache/KeyCacheKey.java
@@ -30,6 +30,8 @@ public class KeyCacheKey implements CacheKey
     public final UUID cfId;
     public final Descriptor desc;
 
+    private static final long EMPTY_SIZE = ObjectSizes.measure(new KeyCacheKey(null, null, ByteBufferUtil.EMPTY_BYTE_BUFFER));
+
     // keeping an array instead of a ByteBuffer lowers the overhead of the key cache working set,
     // without extra copies on lookup since client-provided key ByteBuffers will be array-backed already
     public final byte[] key;
@@ -52,15 +54,9 @@ public class KeyCacheKey implements CacheKey
         return String.format("KeyCacheKey(%s, %s)", desc, ByteBufferUtil.bytesToHex(ByteBuffer.wrap(key)));
     }
 
-    public long memorySize()
+    public long unsharedHeapSize()
     {
-        return ObjectSizes.getFieldSize(// cfId
-                                        ObjectSizes.getReferenceSize() +
-                                        // desc
-                                        ObjectSizes.getReferenceSize() +
-                                        // key
-                                        ObjectSizes.getReferenceSize())
-               + ObjectSizes.getArraySize(key);
+        return EMPTY_SIZE + ObjectSizes.sizeOfArray(key);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/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 fc4a6e0..bbd8591 100644
--- a/src/java/org/apache/cassandra/cache/RowCacheKey.java
+++ b/src/java/org/apache/cassandra/cache/RowCacheKey.java
@@ -33,6 +33,8 @@ public class RowCacheKey implements CacheKey, Comparable<RowCacheKey>
     public final UUID cfId;
     public final byte[] key;
 
+    private static final long EMPTY_SIZE = ObjectSizes.measure(new RowCacheKey(null, ByteBufferUtil.EMPTY_BYTE_BUFFER));
+
     public RowCacheKey(UUID cfId, DecoratedKey key)
     {
         this(cfId, key.key);
@@ -51,13 +53,9 @@ public class RowCacheKey implements CacheKey, Comparable<RowCacheKey>
         return new PathInfo(cf.left, cf.right, cfId);
     }
 
-    public long memorySize()
+    public long unsharedHeapSize()
     {
-        return ObjectSizes.getFieldSize(// cfId
-                                        ObjectSizes.getReferenceSize() +
-                                        // key
-                                        ObjectSizes.getReferenceSize())
-               + ObjectSizes.getArraySize(key);
+        return EMPTY_SIZE + ObjectSizes.sizeOfArray(key);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/src/java/org/apache/cassandra/cache/RowCacheSentinel.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cache/RowCacheSentinel.java b/src/java/org/apache/cassandra/cache/RowCacheSentinel.java
index 9a014dc..01e90c0 100644
--- a/src/java/org/apache/cassandra/cache/RowCacheSentinel.java
+++ b/src/java/org/apache/cassandra/cache/RowCacheSentinel.java
@@ -57,10 +57,4 @@ public class RowCacheSentinel implements IRowCacheEntry
     {
         return Objects.hashCode(sentinelId);
     }
-
-    public long memorySize()
-    {
-        // Only long reference.
-        return ObjectSizes.getFieldSize(TypeSizes.NATIVE.sizeof(sentinelId));
-    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/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 5a944a2..2eb9b18 100644
--- a/src/java/org/apache/cassandra/config/Config.java
+++ b/src/java/org/apache/cassandra/config/Config.java
@@ -79,6 +79,7 @@ public class Config
 
     public Integer memtable_flush_writers = null; // will get set to the length of data dirs in DatabaseDescriptor
     public Integer memtable_total_space_in_mb;
+    public float memtable_cleanup_threshold = 0.4f;
 
     public Integer storage_port = 7000;
     public Integer ssl_storage_port = 7001;
@@ -160,7 +161,6 @@ public class Config
     public boolean compaction_preheat_key_cache = true;
 
     public volatile boolean incremental_backups = false;
-    public int memtable_flush_queue_size = 4;
     public boolean trickle_fsync = false;
     public int trickle_fsync_interval_in_kb = 10240;
 
@@ -187,7 +187,7 @@ public class Config
 
     public boolean inter_dc_tcp_nodelay = true;
 
-    public String memtable_allocator = "SlabAllocator";
+    public String memtable_allocator = "HeapSlabPool";
 
     private static boolean outboundBindAny = false;
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/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 eca8881..2793237 100644
--- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
+++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
@@ -49,9 +49,9 @@ import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.scheduler.IRequestScheduler;
 import org.apache.cassandra.scheduler.NoScheduler;
 import org.apache.cassandra.service.CacheService;
-import org.apache.cassandra.utils.Allocator;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.memory.Pool;
 
 public class DatabaseDescriptor
 {
@@ -93,7 +93,7 @@ public class DatabaseDescriptor
     private static String localDC;
     private static Comparator<InetAddress> localComparator;
 
-    private static Class<? extends Allocator> memtableAllocator;
+    private static Class<? extends Pool> memtablePool;
 
     static
     {
@@ -261,8 +261,8 @@ public class DatabaseDescriptor
         if (conf.memtable_total_space_in_mb == null)
             conf.memtable_total_space_in_mb = (int) (Runtime.getRuntime().maxMemory() / (4 * 1048576));
         if (conf.memtable_total_space_in_mb <= 0)
-            throw new ConfigurationException("memtable_total_space_in_mb must be positive");
-        logger.info("Global memtable threshold is enabled at {}MB", conf.memtable_total_space_in_mb);
+            throw new ConfigurationException("memtable_heap_space_in_mb must be positive");
+        logger.info("Global memtable heap threshold is enabled at {}MB", conf.memtable_total_space_in_mb);
 
         /* Memtable flush writer threads */
         if (conf.memtable_flush_writers != null && conf.memtable_flush_writers < 1)
@@ -482,10 +482,10 @@ public class DatabaseDescriptor
             conf.server_encryption_options = conf.encryption_options;
         }
 
-        String allocatorClass = conf.memtable_allocator;
-        if (!allocatorClass.contains("."))
-            allocatorClass = "org.apache.cassandra.utils." + allocatorClass;
-        memtableAllocator = FBUtilities.classForName(allocatorClass, "allocator");
+        String allocatorPoolClass = conf.memtable_allocator;
+        if (!allocatorPoolClass.contains("."))
+            allocatorPoolClass = "org.apache.cassandra.utils.memory." + allocatorPoolClass;
+        memtablePool = FBUtilities.classForName(allocatorPoolClass, "allocator pool");
 
         // Hardcoded system keyspaces
         List<KSMetaData> systemKeyspaces = Arrays.asList(KSMetaData.systemKeyspace());
@@ -1235,23 +1235,11 @@ public class DatabaseDescriptor
         conf.incremental_backups = value;
     }
 
-    public static int getFlushQueueSize()
-    {
-        return conf.memtable_flush_queue_size;
-    }
-
     public static int getFileCacheSizeInMB()
     {
         return conf.file_cache_size_in_mb;
     }
 
-    public static int getTotalMemtableSpaceInMB()
-    {
-        // should only be called if estimatesRealMemtableSize() is true
-        assert conf.memtable_total_space_in_mb > 0;
-        return conf.memtable_total_space_in_mb;
-    }
-
     public static long getTotalCommitlogSpaceInMB()
     {
         return conf.commitlog_total_space_in_mb;
@@ -1382,15 +1370,17 @@ public class DatabaseDescriptor
         return conf.preheat_kernel_page_cache;
     }
 
-    public static Allocator getMemtableAllocator()
+    public static Pool getMemtableAllocatorPool()
     {
         try
         {
-            return memtableAllocator.newInstance();
+            return memtablePool
+                   .getConstructor(long.class, float.class, Runnable.class)
+                   .newInstance(conf.memtable_total_space_in_mb << 20, conf.memtable_cleanup_threshold, new ColumnFamilyStore.FlushLargestColumnFamily());
         }
-        catch (InstantiationException | IllegalAccessException e)
+        catch (Exception e)
         {
-            throw new RuntimeException(e);
+            throw new AssertionError(e);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/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 db203e6..b10f3be 100644
--- a/src/java/org/apache/cassandra/cql3/ColumnIdentifier.java
+++ b/src/java/org/apache/cassandra/cql3/ColumnIdentifier.java
@@ -22,19 +22,22 @@ import java.nio.ByteBuffer;
 
 import org.apache.cassandra.cache.IMeasurableMemory;
 import org.apache.cassandra.cql3.statements.Selectable;
-import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.ObjectSizes;
+import org.apache.cassandra.utils.memory.AbstractAllocator;
 
 /**
  * Represents an identifer for a CQL column definition.
+ * TODO : should support light-weight mode without text representation for when not interned
  */
 public class ColumnIdentifier implements Selectable, Comparable<ColumnIdentifier>, IMeasurableMemory
 {
     public final ByteBuffer bytes;
     private final String text;
 
+    private static final long EMPTY_SIZE = ObjectSizes.measure(new ColumnIdentifier("", true));
+
     public ColumnIdentifier(String rawText, boolean keepCase)
     {
         this.text = keepCase ? rawText : rawText.toLowerCase(Locale.US);
@@ -47,6 +50,12 @@ public class ColumnIdentifier implements Selectable, Comparable<ColumnIdentifier
         this.text = type.getString(bytes);
     }
 
+    private ColumnIdentifier(ByteBuffer bytes, String text)
+    {
+        this.bytes = bytes;
+        this.text = text;
+    }
+
     @Override
     public final int hashCode()
     {
@@ -73,11 +82,18 @@ public class ColumnIdentifier implements Selectable, Comparable<ColumnIdentifier
         return text;
     }
 
-    public long memorySize()
+    public long unsharedHeapSize()
+    {
+        return EMPTY_SIZE
+             + ObjectSizes.sizeOnHeapOf(bytes)
+             + ObjectSizes.sizeOf(text);
+    }
+
+    public long excessHeapSizeExcludingData()
     {
-        return ObjectSizes.getFieldSize(2 * ObjectSizes.getReferenceSize())
-             + ObjectSizes.getSize(bytes)
-             + TypeSizes.NATIVE.sizeof(text);
+        return EMPTY_SIZE
+             + ObjectSizes.sizeOnHeapExcludingData(bytes)
+             + ObjectSizes.sizeOf(text);
     }
 
     public int compareTo(ColumnIdentifier other)
@@ -87,4 +103,10 @@ public class ColumnIdentifier implements Selectable, Comparable<ColumnIdentifier
 
         return bytes.compareTo(other.bytes);
     }
+
+    public ColumnIdentifier clone(AbstractAllocator allocator)
+    {
+        return new ColumnIdentifier(allocator.clone(bytes), text);
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/src/java/org/apache/cassandra/cql3/QueryProcessor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/QueryProcessor.java b/src/java/org/apache/cassandra/cql3/QueryProcessor.java
index 0c0167e..7035f63 100644
--- a/src/java/org/apache/cassandra/cql3/QueryProcessor.java
+++ b/src/java/org/apache/cassandra/cql3/QueryProcessor.java
@@ -49,7 +49,7 @@ public class QueryProcessor
     public static final SemanticVersion CQL_VERSION = new SemanticVersion("3.1.4");
 
     private static final Logger logger = LoggerFactory.getLogger(QueryProcessor.class);
-    private static final MemoryMeter meter = new MemoryMeter();
+    private static final MemoryMeter meter = new MemoryMeter().withGuessing(MemoryMeter.Guess.FALLBACK_BEST);
     private static final long MAX_CACHE_PREPARED_MEMORY = Runtime.getRuntime().maxMemory() / 256;
     private static final int MAX_CACHE_PREPARED_COUNT = 10000;
 
@@ -76,28 +76,14 @@ public class QueryProcessor
 
     static
     {
-        if (MemoryMeter.isInitialized())
-        {
-            preparedStatements = new ConcurrentLinkedHashMap.Builder<MD5Digest, CQLStatement>()
-                                 .maximumWeightedCapacity(MAX_CACHE_PREPARED_MEMORY)
-                                 .weigher(cqlMemoryUsageWeigher)
-                                 .build();
-            thriftPreparedStatements = new ConcurrentLinkedHashMap.Builder<Integer, CQLStatement>()
-                                       .maximumWeightedCapacity(MAX_CACHE_PREPARED_MEMORY)
-                                       .weigher(thriftMemoryUsageWeigher)
-                                       .build();
-        }
-        else
-        {
-            logger.error("Unable to initialize MemoryMeter (jamm not specified as javaagent).  This means "
-                         + "Cassandra will be unable to measure object sizes accurately and may consequently OOM.");
-            preparedStatements = new ConcurrentLinkedHashMap.Builder<MD5Digest, CQLStatement>()
-                                 .maximumWeightedCapacity(MAX_CACHE_PREPARED_COUNT)
-                                 .build();
-            thriftPreparedStatements = new ConcurrentLinkedHashMap.Builder<Integer, CQLStatement>()
-                                       .maximumWeightedCapacity(MAX_CACHE_PREPARED_COUNT)
-                                       .build();
-        }
+        preparedStatements = new ConcurrentLinkedHashMap.Builder<MD5Digest, CQLStatement>()
+                             .maximumWeightedCapacity(MAX_CACHE_PREPARED_MEMORY)
+                             .weigher(cqlMemoryUsageWeigher)
+                             .build();
+        thriftPreparedStatements = new ConcurrentLinkedHashMap.Builder<Integer, CQLStatement>()
+                                   .maximumWeightedCapacity(MAX_CACHE_PREPARED_MEMORY)
+                                   .weigher(thriftMemoryUsageWeigher)
+                                   .build();
     }
 
     private static final List<PreExecutionHook> preExecutionHooks = new CopyOnWriteArrayList<>();
@@ -449,9 +435,6 @@ public class QueryProcessor
 
     private static long measure(Object key)
     {
-        if (!MemoryMeter.isInitialized())
-            return 1;
-
         return key instanceof MeasurableForPreparedCache
              ? ((MeasurableForPreparedCache)key).measureForPreparedCache(meter)
              : meter.measureDeep(key);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/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 dbebf5e..7bcbe25 100644
--- a/src/java/org/apache/cassandra/db/ArrayBackedSortedColumns.java
+++ b/src/java/org/apache/cassandra/db/ArrayBackedSortedColumns.java
@@ -29,7 +29,7 @@ import org.apache.cassandra.db.composites.CellName;
 import org.apache.cassandra.db.composites.CellNameType;
 import org.apache.cassandra.db.composites.Composite;
 import org.apache.cassandra.db.filter.ColumnSlice;
-import org.apache.cassandra.utils.Allocator;
+import org.apache.cassandra.utils.memory.AbstractAllocator;
 
 /**
  * A ColumnFamily backed by an ArrayList.
@@ -101,7 +101,7 @@ public class ArrayBackedSortedColumns extends AbstractThreadUnsafeSortedColumns
      * without knowing about (we can revisit that decision later if we have
      * use cases where most insert are in sorted order but a few are not).
      */
-    public void addColumn(Cell cell, Allocator allocator)
+    public void addColumn(Cell cell, AbstractAllocator allocator)
     {
         if (cells.isEmpty())
         {
@@ -131,7 +131,7 @@ public class ArrayBackedSortedColumns extends AbstractThreadUnsafeSortedColumns
             if (pos >= 0)
                 resolveAgainst(pos, cell, allocator);
             else
-                cells.add(-pos-1, cell);
+                cells.add(-pos - 1, cell);
         }
     }
 
@@ -139,7 +139,7 @@ public class ArrayBackedSortedColumns extends AbstractThreadUnsafeSortedColumns
      * Resolve against element at position i.
      * Assume that i is a valid position.
      */
-    private void resolveAgainst(int i, Cell cell, Allocator allocator)
+    private void resolveAgainst(int i, Cell cell, AbstractAllocator allocator)
     {
         Cell oldCell = cells.get(i);
 
@@ -184,7 +184,7 @@ public class ArrayBackedSortedColumns extends AbstractThreadUnsafeSortedColumns
         return -mid - (result < 0 ? 1 : 2);
     }
 
-    public void addAll(ColumnFamily cm, Allocator allocator, Function<Cell, Cell> transformation)
+    public void addAll(ColumnFamily cm, AbstractAllocator allocator, Function<Cell, Cell> transformation)
     {
         delete(cm.deletionInfo());
         if (cm.getColumnCount() == 0)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/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 c475a0e..238bb7c 100644
--- a/src/java/org/apache/cassandra/db/AtomicBTreeColumns.java
+++ b/src/java/org/apache/cassandra/db/AtomicBTreeColumns.java
@@ -18,25 +18,28 @@
 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;
+import java.util.List;
 import java.util.concurrent.atomic.AtomicReferenceFieldUpdater;
 
 import com.google.common.base.Function;
 import com.google.common.base.Functions;
-import com.google.common.collect.Iterators;
+import com.google.common.collect.*;
 
 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.filter.ColumnSlice;
 import org.apache.cassandra.db.index.SecondaryIndexManager;
-import org.apache.cassandra.utils.Allocator;
+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.ReplaceFunction;
+import org.apache.cassandra.utils.btree.UpdateFunction;
+import org.apache.cassandra.utils.memory.AbstractAllocator;
 
 import static org.apache.cassandra.db.index.SecondaryIndexManager.Updater;
 
@@ -51,6 +54,9 @@ 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))
+            + ObjectSizes.measure(new Holder(null, null));
+
     private static final Function<Cell, CellName> NAME = new Function<Cell, CellName>()
     {
         public CellName apply(Cell column)
@@ -152,52 +158,58 @@ public class AtomicBTreeColumns extends ColumnFamily
         }
     }
 
-    public void addColumn(Cell column, Allocator allocator)
-    {
-        while (true)
-        {
-            Holder current = ref;
-            Holder update = ref.update(this, current.deletionInfo, metadata.comparator.columnComparator(), Arrays.asList(column), null);
-            if (refUpdater.compareAndSet(this, current, update))
-                return;
-        }
-    }
-
-    public void addAll(ColumnFamily cm, Allocator allocator, Function<Cell, Cell> transformation)
+    public void addAll(ColumnFamily cm, AbstractAllocator allocator, Function<Cell, Cell> transformation)
     {
-        addAllWithSizeDelta(cm, allocator, transformation, SecondaryIndexManager.nullUpdater);
+        addAllWithSizeDelta(cm, allocator, transformation, SecondaryIndexManager.nullUpdater, new Delta());
     }
 
     // the function we provide to the btree utilities to perform any column replacements
-    private static final class ColumnUpdater implements ReplaceFunction<Cell>
+    private static final class ColumnUpdater implements UpdateFunction<Cell>
     {
-        final Allocator allocator;
+        final AtomicBTreeColumns updating;
+        final Holder ref;
+        final AbstractAllocator allocator;
         final Function<Cell, Cell> transform;
         final Updater indexer;
-        long delta;
+        final Delta delta;
 
-        private ColumnUpdater(Allocator allocator, Function<Cell, Cell> transform, Updater indexer)
+        private ColumnUpdater(AtomicBTreeColumns updating, Holder ref, AbstractAllocator allocator, Function<Cell, Cell> transform, Updater indexer, Delta delta)
         {
+            this.updating = updating;
+            this.ref = ref;
             this.allocator = allocator;
             this.transform = transform;
             this.indexer = indexer;
+            this.delta = delta;
         }
 
         public Cell apply(Cell inserted)
         {
             indexer.insert(inserted);
-            delta += inserted.dataSize();
+            delta.insert(inserted);
             return transform.apply(inserted);
         }
 
-        public Cell apply(Cell replaced, Cell update)
+        public Cell apply(Cell existing, Cell update)
         {
-            Cell reconciled = update.reconcile(replaced, allocator);
-            indexer.update(replaced, reconciled);
-            delta += reconciled.dataSize() - replaced.dataSize();
-
+            Cell reconciled = update.reconcile(existing, allocator);
+            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)
@@ -215,11 +227,11 @@ public class AtomicBTreeColumns extends ColumnFamily
     }
 
     /**
-     * This is only called by Memtable.resolve, so only AtomicSortedColumns needs to implement it.
+     * 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 long addAllWithSizeDelta(final ColumnFamily cm, Allocator allocator, Function<Cell, Cell> transformation, Updater indexer)
+    public Delta addAllWithSizeDelta(final ColumnFamily cm, AbstractAllocator allocator, Function<Cell, Cell> transformation, Updater indexer, Delta delta)
     {
         boolean transformed = false;
         Collection<Cell> insert;
@@ -248,11 +260,14 @@ public class AtomicBTreeColumns extends ColumnFamily
                     }
                 }
             }
+
+            delta.reset();
             deletionInfo = current.deletionInfo.copy().add(deletionInfo);
+            delta.addHeapSize(deletionInfo.unsharedHeapSize() - current.deletionInfo.unsharedHeapSize());
+            ColumnUpdater updater = new ColumnUpdater(this, current, allocator, transformation, indexer, delta);
+            Object[] tree = BTree.update(current.tree, metadata.comparator.columnComparator(), insert, true, updater);
 
-            ColumnUpdater updater = new ColumnUpdater(allocator, transformation, indexer);
-            Holder h = current.update(this, deletionInfo, metadata.comparator.columnComparator(), insert, updater);
-            if (h != null && refUpdater.compareAndSet(this, current, h))
+            if (tree != null && refUpdater.compareAndSet(this, current, new Holder(tree, deletionInfo)))
             {
                 indexer.updateRowLevelIndexes();
                 return updater.delta;
@@ -268,25 +283,20 @@ public class AtomicBTreeColumns extends ColumnFamily
 
     }
 
-    public boolean replace(Cell oldColumn, Cell newColumn)
+    // no particular reason not to implement these next methods, we just haven't needed them yet
+
+    public void addColumn(Cell column, AbstractAllocator allocator)
     {
-        if (!oldColumn.name().equals(newColumn.name()))
-            throw new IllegalArgumentException();
+        throw new UnsupportedOperationException();
+    }
 
-        while (true)
-        {
-            Holder current = ref;
-            Holder modified = current.update(this, current.deletionInfo, metadata.comparator.columnComparator(), Arrays.asList(newColumn), null);
-            if (modified == current)
-                return false;
-            if (refUpdater.compareAndSet(this, current, modified))
-                return true;
-        }
+    public boolean replace(Cell oldColumn, Cell newColumn)
+    {
+        throw new UnsupportedOperationException();
     }
 
     public void clear()
     {
-        // no particular reason not to implement this, we just haven't needed it yet
         throw new UnsupportedOperationException();
     }
 
@@ -377,35 +387,71 @@ public class AtomicBTreeColumns extends ColumnFamily
         {
             return new Holder(this.tree, info);
         }
+    }
+
+    // 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
+    {
+        private long dataSize;
+        private long heapSize;
+
+        // 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;
 
-        Holder update(AtomicBTreeColumns container, DeletionInfo deletionInfo, Comparator<Cell> cmp, Collection<Cell> update, ReplaceFunction<Cell> replaceF)
+        protected void reset()
         {
-            Object[] r = BTree.update(tree, cmp, update, true, replaceF, new TerminateEarly(container, this));
-            // result can be null if terminate early kicks in, in which case we need to propagate the early failure so we can retry
-            if (r == null)
-                return null;
-            return new Holder(r, deletionInfo);
+            this.dataSize = 0;
+            this.heapSize = 0;
+            discarded.clear();
         }
-    }
 
-    // a function provided to the btree functions that aborts the modification
-    // if we already know the final cas will fail
-    private static final class TerminateEarly implements Function<Object, Boolean>
-    {
-        final AtomicBTreeColumns columns;
-        final Holder ref;
+        protected void addHeapSize(long heapSize)
+        {
+            this.heapSize += heapSize;
+        }
 
-        private TerminateEarly(AtomicBTreeColumns columns, Holder ref)
+        protected void swap(Cell old, Cell updated)
         {
-            this.columns = columns;
-            this.ref = ref;
+            dataSize += updated.dataSize() - old.dataSize();
+            heapSize += updated.excessHeapSizeExcludingData() - old.excessHeapSizeExcludingData();
+            discarded.add(old);
+        }
+
+        protected void insert(Cell insert)
+        {
+            this.dataSize += insert.dataSize();
+            this.heapSize += insert.excessHeapSizeExcludingData();
+        }
+
+        private void abort(Cell neverUsed)
+        {
+            if (aborted == null)
+                aborted = new ArrayList<>();
+            aborted.add(neverUsed);
+        }
+
+        public long dataSize()
+        {
+            return dataSize;
+        }
+
+        public long excessHeapSize()
+        {
+            return heapSize;
         }
 
-        public Boolean apply(Object o)
+        public Iterable<Cell> reclaimed()
         {
-            if (ref != columns.ref)
-                return Boolean.TRUE;
-            return Boolean.FALSE;
+            if (aborted == null)
+                return discarded;
+            return Iterables.concat(discarded, aborted);
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/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 d3cf085..44656ea 100644
--- a/src/java/org/apache/cassandra/db/Cell.java
+++ b/src/java/org/apache/cassandra/db/Cell.java
@@ -29,14 +29,16 @@ 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.Allocator;
+import org.apache.cassandra.utils.ObjectSizes;
+import org.apache.cassandra.utils.memory.AbstractAllocator;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
-import org.apache.cassandra.utils.HeapAllocator;
+import org.apache.cassandra.utils.memory.HeapAllocator;
 
 /**
  * Cell is immutable, which prevents all kinds of confusion in a multithreaded environment.
@@ -45,6 +47,8 @@ public class Cell implements 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))));
+
     /**
      * For 2.0-formatted sstables (where column count is not stored), @param count should be Integer.MAX_VALUE,
      * and we will look for the end-of-row column name marker instead of relying on that.
@@ -158,7 +162,14 @@ public class Cell implements OnDiskAtom
 
     public int dataSize()
     {
-        return name().dataSize() + value.remaining() + TypeSizes.NATIVE.sizeof(timestamp);
+        return name.dataSize() + value.remaining() + TypeSizes.NATIVE.sizeof(timestamp);
+    }
+
+    // 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)
@@ -215,7 +226,7 @@ public class Cell implements OnDiskAtom
         return reconcile(cell, HeapAllocator.instance);
     }
 
-    public Cell reconcile(Cell cell, Allocator allocator)
+    public Cell reconcile(Cell cell, AbstractAllocator allocator)
     {
         // tombstones take precedence.  (if both are tombstones, then it doesn't matter which one we use.)
         if (isMarkedForDelete(System.currentTimeMillis()))
@@ -252,7 +263,7 @@ public class Cell implements OnDiskAtom
         return result;
     }
 
-    public Cell localCopy(ColumnFamilyStore cfs, Allocator allocator)
+    public Cell localCopy(ColumnFamilyStore cfs, AbstractAllocator allocator)
     {
         return new Cell(name.copy(allocator), allocator.clone(value), timestamp);
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/src/java/org/apache/cassandra/db/ClockAndCount.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ClockAndCount.java b/src/java/org/apache/cassandra/db/ClockAndCount.java
index 1678c8c..db6c705 100644
--- a/src/java/org/apache/cassandra/db/ClockAndCount.java
+++ b/src/java/org/apache/cassandra/db/ClockAndCount.java
@@ -24,6 +24,9 @@ import org.apache.cassandra.utils.ObjectSizes;
 
 public class ClockAndCount implements IMeasurableMemory
 {
+
+    private static final long EMPTY_SIZE = ObjectSizes.measure(new ClockAndCount(0, 0));
+
     public static ClockAndCount BLANK = ClockAndCount.create(0L, 0L);
 
     public final long clock;
@@ -40,10 +43,9 @@ public class ClockAndCount implements IMeasurableMemory
         return new ClockAndCount(clock, count);
     }
 
-    public long memorySize()
+    public long unsharedHeapSize()
     {
-        return ObjectSizes.getFieldSize(TypeSizes.NATIVE.sizeof(clock))
-             + ObjectSizes.getFieldSize(TypeSizes.NATIVE.sizeof(count));
+        return EMPTY_SIZE;
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/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 1f164a9..0ce2654 100644
--- a/src/java/org/apache/cassandra/db/CollationController.java
+++ b/src/java/org/apache/cassandra/db/CollationController.java
@@ -30,7 +30,7 @@ import org.apache.cassandra.db.marshal.CounterColumnType;
 import org.apache.cassandra.io.sstable.SSTableReader;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.tracing.Tracing;
-import org.apache.cassandra.utils.HeapAllocator;
+import org.apache.cassandra.utils.memory.HeapAllocator;
 
 public class CollationController
 {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/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 9ce6b0c..9d2856d 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamily.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamily.java
@@ -31,6 +31,9 @@ import java.util.UUID;
 import com.google.common.base.Function;
 import com.google.common.base.Functions;
 import com.google.common.collect.ImmutableMap;
+
+import org.apache.cassandra.utils.memory.AbstractAllocator;
+import org.apache.cassandra.utils.memory.HeapAllocator;
 import org.apache.commons.lang3.builder.HashCodeBuilder;
 
 import org.apache.cassandra.cache.IRowCacheEntry;
@@ -198,7 +201,7 @@ public abstract class ColumnFamily implements Iterable<Cell>, IRowCacheEntry
      * If a cell with the same name is already present in the map, it will
      * be replaced by the newly added cell.
      */
-    public abstract void addColumn(Cell cell, Allocator allocator);
+    public abstract void addColumn(Cell cell, AbstractAllocator allocator);
 
     /**
      * Adds all the columns of a given column map to this column map.
@@ -209,7 +212,7 @@ public abstract class ColumnFamily implements Iterable<Cell>, IRowCacheEntry
      *   </code>
      *  but is potentially faster.
      */
-    public abstract void addAll(ColumnFamily cm, Allocator allocator, Function<Cell, Cell> transformation);
+    public abstract void addAll(ColumnFamily cm, AbstractAllocator allocator, Function<Cell, Cell> transformation);
 
     /**
      * Replace oldCell if present by newCell.
@@ -282,7 +285,7 @@ public abstract class ColumnFamily implements Iterable<Cell>, IRowCacheEntry
         delete(columns.deletionInfo());
     }
 
-    public void addAll(ColumnFamily cf, Allocator allocator)
+    public void addAll(ColumnFamily cf, AbstractAllocator allocator)
     {
         addAll(cf, allocator, Functions.<Cell>identity());
     }
@@ -404,7 +407,7 @@ public abstract class ColumnFamily implements Iterable<Cell>, IRowCacheEntry
         resolve(cf, HeapAllocator.instance);
     }
 
-    public void resolve(ColumnFamily cf, Allocator allocator)
+    public void resolve(ColumnFamily cf, AbstractAllocator allocator)
     {
         // Row _does_ allow null CF objects :(  seems a necessary evil for efficiency
         if (cf == null)


[2/6] remove Table.switchlock and introduce o.a.c.utils.memory package patch by Benedict Elliott Smith; reviewed by jbellis for CASSANDRA-5549

Posted by jb...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/src/java/org/apache/cassandra/utils/btree/ReplaceFunction.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/btree/ReplaceFunction.java b/src/java/org/apache/cassandra/utils/btree/ReplaceFunction.java
deleted file mode 100644
index a193c31..0000000
--- a/src/java/org/apache/cassandra/utils/btree/ReplaceFunction.java
+++ /dev/null
@@ -1,17 +0,0 @@
-package org.apache.cassandra.utils.btree;
-
-import com.google.common.base.Function;
-
-/**
- * An interface defining a function to be applied to both the object we are replacing in a BTree and
- * the object that is intended to replace it, returning the object to actually replace it.
- *
- * If this is a new insertion, that is there is no object to replace, the one argument variant of
- * the function will be called.
- *
- * @param <V>
- */
-public interface ReplaceFunction<V> extends Function<V, V>
-{
-    V apply(V replaced, V update);
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/src/java/org/apache/cassandra/utils/btree/UpdateFunction.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/btree/UpdateFunction.java b/src/java/org/apache/cassandra/utils/btree/UpdateFunction.java
new file mode 100644
index 0000000..cd30492
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/btree/UpdateFunction.java
@@ -0,0 +1,30 @@
+package org.apache.cassandra.utils.btree;
+
+import com.google.common.base.Function;
+
+/**
+ * An interface defining a function to be applied to both the object we are replacing in a BTree and
+ * the object that is intended to replace it, returning the object to actually replace it.
+ *
+ * @param <V>
+ */
+public interface UpdateFunction<V> extends Function<V, V>
+{
+    /**
+     * @param replacing the value in the original tree we have matched
+     * @param update the value in the updating collection that matched
+     * @return the value to insert into the new tree
+     */
+    V apply(V replacing, V update);
+
+    /**
+     * @return true if we should fail the update
+     */
+    boolean abortEarly();
+
+    /**
+     * @param heapSize extra heap space allocated (over previous tree)
+     */
+    void allocated(long heapSize);
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/src/java/org/apache/cassandra/utils/concurrent/OpOrder.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/concurrent/OpOrder.java b/src/java/org/apache/cassandra/utils/concurrent/OpOrder.java
new file mode 100644
index 0000000..44330a0
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/concurrent/OpOrder.java
@@ -0,0 +1,411 @@
+package org.apache.cassandra.utils.concurrent;
+
+import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
+
+/**
+ * <p>A class for providing synchronization between producers and consumers that do not
+ * communicate directly with each other, but where the consumers need to process their
+ * work in contiguous batches. In particular this is useful for both CommitLog and Memtable
+ * where the producers (writing threads) are modifying a structure that the consumer
+ * (flush executor) only batch syncs, but needs to know what 'position' the work is at
+ * for co-ordination with other processes,
+ *
+ * <p>The typical usage is something like:
+ * <pre>
+     public final class ExampleShared
+     {
+        final OpOrder order = new OpOrder();
+        volatile SharedState state;
+
+        static class SharedState
+        {
+            volatile Barrier barrier;
+
+            // ...
+        }
+
+        public void consume()
+        {
+            SharedState state = this.state;
+            state.setReplacement(new State())
+            state.doSomethingToPrepareForBarrier();
+
+            state.barrier = order.newBarrier();
+            // seal() MUST be called after newBarrier() else barrier.isAfter()
+            // will always return true, and barrier.await() will fail
+            state.barrier.issue();
+
+            // wait for all producer work started prior to the barrier to complete
+            state.barrier.await();
+
+            // change the shared state to its replacement, as the current state will no longer be used by producers
+            this.state = state.getReplacement();
+
+            state.doSomethingWithExclusiveAccess();
+        }
+
+        public void produce()
+        {
+            Group opGroup = order.start();
+            try
+            {
+                SharedState s = state;
+                while (s.barrier != null && !s.barrier.isAfter(opGroup))
+                    s = s.getReplacement();
+                s.doProduceWork();
+            }
+            finally
+            {
+                opGroup.finishOne();
+            }
+        }
+    }
+ * </pre>
+ */
+public class OpOrder
+{
+    /**
+     * Constant that when an Ordered.running is equal to, indicates the Ordered is complete
+     */
+    private static final int FINISHED = -1;
+
+    /**
+     * A linked list starting with the most recent Ordered object, i.e. the one we should start new operations from,
+     * with (prev) links to any incomplete Ordered instances, and (next) links to any potential future Ordered instances.
+     * Once all operations started against an Ordered instance and its ancestors have been finished the next instance
+     * will unlink this one
+     */
+    private volatile Group current = new Group();
+
+    /**
+     * Start an operation against this OpOrder.
+     * Once the operation is completed Ordered.finishOne() MUST be called EXACTLY once for this operation.
+     *
+     * @return the Ordered instance that manages this OpOrder
+     */
+    public Group start()
+    {
+        while (true)
+        {
+            Group current = this.current;
+            if (current.register())
+                return current;
+        }
+    }
+
+    /**
+     * Creates a new barrier. The barrier is only a placeholder until barrier.issue() is called on it,
+     * after which all new operations will start against a new Group that will not be accepted
+     * by barrier.isAfter(), and barrier.await() will return only once all operations started prior to the issue
+     * have completed.
+     *
+     * @return
+     */
+    public Barrier newBarrier()
+    {
+        return new Barrier();
+    }
+
+    public Group getCurrent()
+    {
+        return current;
+    }
+
+    /**
+     * Represents a group of identically ordered operations, i.e. all operations started in the interval between
+     * two barrier issuances. For each register() call this is returned, finishOne() must be called exactly once.
+     * It should be treated like taking a lock().
+     */
+    public static final class Group implements Comparable<Group>
+    {
+        /**
+         * In general this class goes through the following stages:
+         * 1) LIVE:      many calls to register() and finishOne()
+         * 2) FINISHING: a call to expire() (after a barrier issue), means calls to register() will now fail,
+         *               and we are now 'in the past' (new operations will be started against a new Ordered)
+         * 3) FINISHED:  once the last finishOne() is called, this Ordered is done. We call unlink().
+         * 4) ZOMBIE:    all our operations are finished, but some operations against an earlier Ordered are still
+         *               running, or tidying up, so unlink() fails to remove us
+         * 5) COMPLETE:  all operations started on or before us are FINISHED (and COMPLETE), so we are unlinked
+         * <p/>
+         * Another parallel states is ISBLOCKING:
+         * <p/>
+         * isBlocking => a barrier that is waiting on us (either directly, or via a future Ordered) is blocking general
+         * progress. This state is entered by calling Barrier.markBlocking(). If the running operations are blocked
+         * on a Signal that is also registered with the isBlockingSignal (probably through isSafeBlockingSignal)
+         * then they will be notified that they are blocking forward progress, and may take action to avoid that.
+         */
+
+        private volatile Group prev, next;
+        private final long id; // monotonically increasing id for compareTo()
+        private volatile int running = 0; // number of operations currently running.  < 0 means we're expired, and the count of tasks still running is -(running + 1)
+        private volatile boolean isBlocking; // indicates running operations are blocking future barriers
+        private final WaitQueue isBlockingSignal = new WaitQueue(); // signal to wait on to indicate isBlocking is true
+        private final WaitQueue waiting = new WaitQueue(); // signal to wait on for completion
+
+        static final AtomicIntegerFieldUpdater<Group> runningUpdater = AtomicIntegerFieldUpdater.newUpdater(Group.class, "running");
+
+        // constructs first instance only
+        private Group()
+        {
+            this.id = 0;
+        }
+
+        private Group(Group prev)
+        {
+            this.id = prev.id + 1;
+            this.prev = prev;
+        }
+
+        // prevents any further operations starting against this Ordered instance
+        // if there are no running operations, calls unlink; otherwise, we let the last op to finishOne call it.
+        // this means issue() won't have to block for ops to finish.
+        private void expire()
+        {
+            while (true)
+            {
+                int current = running;
+                if (current < 0)
+                    throw new IllegalStateException();
+                if (runningUpdater.compareAndSet(this, current, -1 - current))
+                {
+                    // if we're already finished (no running ops), unlink ourselves
+                    if (current == 0)
+                        unlink();
+                    return;
+                }
+            }
+        }
+
+        // attempts to start an operation against this Ordered instance, and returns true if successful.
+        private boolean register()
+        {
+            while (true)
+            {
+                int current = running;
+                if (current < 0)
+                    return false;
+                if (runningUpdater.compareAndSet(this, current, current + 1))
+                    return true;
+            }
+        }
+
+        /**
+         * To be called exactly once for each register() call this object is returned for, indicating the operation
+         * is complete
+         */
+        public void finishOne()
+        {
+            while (true)
+            {
+                int current = running;
+                if (current < 0)
+                {
+                    if (runningUpdater.compareAndSet(this, current, current + 1))
+                    {
+                        if (current + 1 == FINISHED)
+                        {
+                            // if we're now finished, unlink ourselves
+                            unlink();
+                        }
+                        return;
+                    }
+                }
+                else if (runningUpdater.compareAndSet(this, current, current - 1))
+                {
+                    return;
+                }
+            }
+        }
+
+        /**
+         * called once we know all operations started against this Ordered have completed,
+         * however we do not know if operations against its ancestors have completed, or
+         * if its descendants have completed ahead of it, so we attempt to create the longest
+         * chain from the oldest still linked Ordered. If we can't reach the oldest through
+         * an unbroken chain of completed Ordered, we abort, and leave the still completing
+         * ancestor to tidy up.
+         */
+        private void unlink()
+        {
+            // walk back in time to find the start of the list
+            Group start = this;
+            while (true)
+            {
+                Group prev = start.prev;
+                if (prev == null)
+                    break;
+                // if we haven't finished this Ordered yet abort and let it clean up when it's done
+                if (prev.running != FINISHED)
+                    return;
+                start = prev;
+            }
+
+            // now walk forwards in time, in case we finished up late
+            Group end = this.next;
+            while (end.running == FINISHED)
+                end = end.next;
+
+            // now walk from first to last, unlinking the prev pointer and waking up any blocking threads
+            while (start != end)
+            {
+                Group next = start.next;
+                next.prev = null;
+                start.waiting.signalAll();
+                start = next;
+            }
+        }
+
+        /**
+         * @return true if a barrier we are behind is, or may be, blocking general progress,
+         * so we should try more aggressively to progress
+         */
+        public boolean isBlocking()
+        {
+            return isBlocking;
+        }
+
+        /**
+         * register to be signalled when a barrier waiting on us is, or maybe, blocking general progress,
+         * so we should try more aggressively to progress
+         */
+        public WaitQueue.Signal isBlockingSignal()
+        {
+            return isBlockingSignal.register();
+        }
+
+        /**
+         * wrap the provided signal to also be signalled if the operation gets marked blocking
+         */
+        public WaitQueue.Signal isBlockingSignal(WaitQueue.Signal signal)
+        {
+            return WaitQueue.any(signal, isBlockingSignal());
+        }
+
+        public int compareTo(Group that)
+        {
+            // we deliberately use subtraction, as opposed to Long.compareTo() as we care about ordering
+            // not which is the smaller value, so this permits wrapping in the unlikely event we exhaust the long space
+            long c = this.id - that.id;
+            if (c > 0)
+                return 1;
+            else if (c < 0)
+                return -1;
+            else
+                return 0;
+        }
+    }
+
+    /**
+     * This class represents a synchronisation point providing ordering guarantees on operations started
+     * against the enclosing OpOrder.  When issue() is called upon it (may only happen once per Barrier), the
+     * Barrier atomically partitions new operations from those already running (by expiring the current Group),
+     * and activates its isAfter() method
+     * which indicates if an operation was started before or after this partition. It offers methods to
+     * determine, or block until, all prior operations have finished, and a means to indicate to those operations
+     * that they are blocking forward progress. See {@link OpOrder} for idiomatic usage.
+     */
+    public final class Barrier
+    {
+        // this Barrier was issued after all Group operations started against orderOnOrBefore
+        private volatile Group orderOnOrBefore;
+
+        /**
+         * @return true if @param group was started prior to the issuing of the barrier.
+         *
+         * (Until issue is called, always returns true, but if you rely on this behavior you are probably
+         * Doing It Wrong.)
+         */
+        public boolean isAfter(Group group)
+        {
+            if (orderOnOrBefore == null)
+                return true;
+            // we subtract to permit wrapping round the full range of Long - so we only need to ensure
+            // there are never Long.MAX_VALUE * 2 total Group objects in existence at any one timem which will
+            // take care of itself
+            return orderOnOrBefore.id - group.id >= 0;
+        }
+
+        /**
+         * Issues (seals) the barrier, meaning no new operations may be issued against it, and expires the current
+         * Group.  Must be called before await() for isAfter() to be properly synchronised.
+         */
+        public void issue()
+        {
+            if (orderOnOrBefore != null)
+                throw new IllegalStateException("Can only call issue() once on each Barrier");
+
+            final Group current;
+            synchronized (OpOrder.this)
+            {
+                current = OpOrder.this.current;
+                orderOnOrBefore = current;
+                OpOrder.this.current = current.next = new Group(current);
+            }
+            current.expire();
+        }
+
+        /**
+         * Mark all prior operations as blocking, potentially signalling them to more aggressively make progress
+         */
+        public void markBlocking()
+        {
+            Group current = orderOnOrBefore;
+            while (current != null)
+            {
+                current.isBlocking = true;
+                current.isBlockingSignal.signalAll();
+                current = current.prev;
+            }
+        }
+
+        /**
+         * Register to be signalled once allPriorOpsAreFinished() or allPriorOpsAreFinishedOrSafe() may return true
+         */
+        public WaitQueue.Signal register()
+        {
+            return orderOnOrBefore.waiting.register();
+        }
+
+        /**
+         * @return true if all operations started prior to barrier.issue() have completed
+         */
+        public boolean allPriorOpsAreFinished()
+        {
+            Group current = orderOnOrBefore;
+            if (current == null)
+                throw new IllegalStateException("This barrier needs to have issue() called on it before prior operations can complete");
+            if (current.next.prev == null)
+                return true;
+            return false;
+        }
+
+        /**
+         * wait for all operations started prior to issuing the barrier to complete
+         */
+        public void await()
+        {
+            while (!allPriorOpsAreFinished())
+            {
+                WaitQueue.Signal signal = register();
+                if (allPriorOpsAreFinished())
+                {
+                    signal.cancel();
+                    return;
+                }
+                else
+                    signal.awaitUninterruptibly();
+            }
+            assert orderOnOrBefore.running == FINISHED;
+        }
+
+        /**
+         * returns the Group we are waiting on - any Group with .compareTo(getSyncPoint()) <= 0
+         * must complete before await() returns
+         */
+        public Group getSyncPoint()
+        {
+            return orderOnOrBefore;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/src/java/org/apache/cassandra/utils/concurrent/WaitQueue.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/concurrent/WaitQueue.java b/src/java/org/apache/cassandra/utils/concurrent/WaitQueue.java
new file mode 100644
index 0000000..9bef8c3
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/concurrent/WaitQueue.java
@@ -0,0 +1,508 @@
+package org.apache.cassandra.utils.concurrent;
+
+import com.yammer.metrics.core.TimerContext;
+import org.slf4j.*;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.ConcurrentLinkedDeque;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
+import java.util.concurrent.locks.LockSupport;
+
+/**
+ * <p>A relatively easy to use utility for general purpose thread signalling.</p>
+ * <p>Usage on a thread awaiting a state change using a WaitQueue q is:</p>
+ * <pre>
+ * {@code
+ *      while (!conditionMet())
+ *          Signal s = q.register();
+ *              if (!conditionMet())    // or, perhaps more correctly, !conditionChanged()
+ *                  s.await();
+ *              else
+ *                  s.cancel();
+ * }
+ * </pre>
+ * A signalling thread, AFTER changing the state, then calls q.signal() to wake up one, or q.signalAll()
+ * to wake up all, waiting threads.
+ * <p>To understand intuitively how this class works, the idea is simply that a thread, once it considers itself
+ * incapable of making progress, registers to be awoken once that changes. Since this could have changed between
+ * checking and registering (in which case the thread that made this change would have been unable to signal it),
+ * it checks the condition again, sleeping only if it hasn't changed/still is not met.</p>
+ * <p>This thread synchronisation scheme has some advantages over Condition objects and Object.wait/notify in that no monitor
+ * acquisition is necessary and, in fact, besides the actual waiting on a signal, all operations are non-blocking.
+ * As a result consumers can never block producers, nor each other, or vice versa, from making progress.
+ * Threads that are signalled are also put into a RUNNABLE state almost simultaneously, so they can all immediately make
+ * progress without having to serially acquire the monitor/lock, reducing scheduler delay incurred.</p>
+ *
+ * <p>A few notes on utilisation:</p>
+ * <p>1. A thread will only exit await() when it has been signalled, but this does not guarantee the condition has not
+ * been altered since it was signalled, and depending on your design it is likely the outer condition will need to be
+ * checked in a loop, though this is not always the case.</p>
+ * <p>2. Each signal is single use, so must be re-registered after each await(). This is true even if it times out.</p>
+ * <p>3. If you choose not to wait on the signal (because the condition has been met before you waited on it)
+ * you must cancel() the signal if the signalling thread uses signal() to awake waiters; otherwise signals will be
+ * lost. If signalAll() is used but infrequent, and register() is frequent, cancel() should still be used to prevent the
+ * queue growing unboundedly. Similarly, if you provide a TimerContext, cancel should be used to ensure it is not erroneously
+ * counted towards wait time.</p>
+ * <p>4. Care must be taken when selecting conditionMet() to ensure we are waiting on the condition that actually
+ * indicates progress is possible. In some complex cases it may be tempting to wait on a condition that is only indicative
+ * of local progress, not progress on the task we are aiming to complete, and a race may leave us waiting for a condition
+ * to be met that we no longer need.
+ * <p>5. This scheme is not fair</p>
+ * <p>6. Only the thread that calls register() may call await()</p>
+ */
+public final class WaitQueue
+{
+
+    private static final Logger logger = LoggerFactory.getLogger(WaitQueue.class);
+
+    private static final int CANCELLED = -1;
+    private static final int SIGNALLED = 1;
+    private static final int NOT_SET = 0;
+
+    private static final AtomicIntegerFieldUpdater signalledUpdater = AtomicIntegerFieldUpdater.newUpdater(RegisteredSignal.class, "state");
+
+    // the waiting signals
+    private final ConcurrentLinkedDeque<RegisteredSignal> queue = new ConcurrentLinkedDeque<>();
+
+    /**
+     * The calling thread MUST be the thread that uses the signal
+     * @return
+     */
+    public Signal register()
+    {
+        RegisteredSignal signal = new RegisteredSignal();
+        queue.add(signal);
+        return signal;
+    }
+
+    /**
+     * The calling thread MUST be the thread that uses the signal.
+     * If the Signal is waited on, context.stop() will be called when the wait times out, the Signal is signalled,
+     * or the waiting thread is interrupted.
+     * @return
+     */
+    public Signal register(TimerContext context)
+    {
+        assert context != null;
+        RegisteredSignal signal = new TimedSignal(context);
+        queue.add(signal);
+        return signal;
+    }
+
+    /**
+     * Signal one waiting thread
+     */
+    public boolean signal()
+    {
+        if (!hasWaiters())
+            return false;
+        while (true)
+        {
+            RegisteredSignal s = queue.poll();
+            if (s == null || s.signal())
+                return s != null;
+        }
+    }
+
+    /**
+     * Signal all waiting threads
+     */
+    public void signalAll()
+    {
+        if (!hasWaiters())
+            return;
+        List<Thread> woke = null;
+        if (logger.isTraceEnabled())
+            woke = new ArrayList<>();
+        long start = System.nanoTime();
+        // we wake up only a snapshot of the queue, to avoid a race where the condition is not met and the woken thread
+        // immediately waits on the queue again
+        RegisteredSignal last = queue.getLast();
+        Iterator<RegisteredSignal> iter = queue.iterator();
+        while (iter.hasNext())
+        {
+            RegisteredSignal signal = iter.next();
+            if (logger.isTraceEnabled())
+            {
+                Thread thread = signal.thread;
+                if (signal.signal())
+                    woke.add(thread);
+            }
+            else
+                signal.signal();
+
+            iter.remove();
+
+            if (signal == last)
+                break;
+        }
+        long end = System.nanoTime();
+        if (woke != null)
+            logger.trace("Woke up {} in {}ms from {}", woke, (end - start) * 0.000001d, Thread.currentThread().getStackTrace()[2]);
+    }
+
+    private void cleanUpCancelled()
+    {
+        // attempt to remove the cancelled from the beginning only, but if we fail to remove any proceed to cover
+        // the whole list
+        Iterator<RegisteredSignal> iter = queue.iterator();
+        while (iter.hasNext())
+        {
+            RegisteredSignal s = iter.next();
+            if (s.isCancelled())
+                iter.remove();
+        }
+    }
+
+    public boolean hasWaiters()
+    {
+        return !queue.isEmpty();
+    }
+
+    /**
+     * Return how many threads are waiting
+     * @return
+     */
+    public int getWaiting()
+    {
+        if (queue.isEmpty())
+            return 0;
+        Iterator<RegisteredSignal> iter = queue.iterator();
+        int count = 0;
+        while (iter.hasNext())
+        {
+            Signal next = iter.next();
+            if (!next.isCancelled())
+                count++;
+        }
+        return count;
+    }
+
+    /**
+     * A Signal is a one-time-use mechanism for a thread to wait for notification that some condition
+     * state has transitioned that it may be interested in (and hence should check if it is).
+     * It is potentially transient, i.e. the state can change in the meantime, it only indicates
+     * that it should be checked, not necessarily anything about what the expected state should be.
+     *
+     * Signal implementations should never wake up spuriously, they are always woken up by a
+     * signal() or signalAll().
+     *
+     * This abstract definition of Signal does not need to be tied to a WaitQueue.
+     * Whilst RegisteredSignal is the main building block of Signals, this abstract
+     * definition allows us to compose Signals in useful ways. The Signal is 'owned' by the
+     * thread that registered itself with WaitQueue(s) to obtain the underlying RegisteredSignal(s);
+     * only the owning thread should use a Signal.
+     */
+    public static interface Signal
+    {
+
+        /**
+         * @return true if signalled; once true, must be discarded by the owning thread.
+         */
+        public boolean isSignalled();
+
+        /**
+         * @return true if cancelled; once cancelled, must be discarded by the owning thread.
+         */
+        public boolean isCancelled();
+
+        /**
+         * @return isSignalled() || isCancelled(). Once true, the state is fixed and the Signal should be discarded
+         * by the owning thread.
+         */
+        public boolean isSet();
+
+        /**
+         * atomically: cancels the Signal if !isSet(), or returns true if isSignalled()
+         *
+         * @return true if isSignalled()
+         */
+        public boolean checkAndClear();
+
+        /**
+         * Should only be called by the owning thread. Indicates the signal can be retired,
+         * and if signalled propagates the signal to another waiting thread
+         */
+        public abstract void cancel();
+
+        /**
+         * Wait, without throwing InterruptedException, until signalled. On exit isSignalled() must be true.
+         * If the thread is interrupted in the meantime, the interrupted flag will be set.
+         */
+        public void awaitUninterruptibly();
+
+        /**
+         * Wait until signalled, or throw an InterruptedException if interrupted before this happens.
+         * On normal exit isSignalled() must be true; however if InterruptedException is thrown isCancelled()
+         * will be true.
+         * @throws InterruptedException
+         */
+        public void await() throws InterruptedException;
+
+        /**
+         * Wait until signalled, or the provided time is reached, or the thread is interrupted. If signalled,
+         * isSignalled() will be true on exit, and the method will return true; if timedout, the method will return
+         * false and isCancelled() will be true; if interrupted an InterruptedException will be thrown and isCancelled()
+         * will be true.
+         * @param until System.currentTimeMillis() to wait until
+         * @return true if signalled, false if timed out
+         * @throws InterruptedException
+         */
+        public boolean awaitUntil(long until) throws InterruptedException;
+    }
+
+    /**
+     * An abstract signal implementation
+     */
+    public static abstract class AbstractSignal implements Signal
+    {
+        public void awaitUninterruptibly()
+        {
+            boolean interrupted = false;
+            while (!isSignalled())
+            {
+                if (Thread.currentThread().interrupted())
+                    interrupted = true;
+                LockSupport.park();
+            }
+            if (interrupted)
+                Thread.currentThread().interrupt();
+            checkAndClear();
+        }
+
+        public void await() throws InterruptedException
+        {
+            while (!isSignalled())
+            {
+                checkInterrupted();
+                LockSupport.park();
+            }
+            checkAndClear();
+        }
+
+        public boolean awaitUntil(long until) throws InterruptedException
+        {
+            while (until < System.currentTimeMillis() && !isSignalled())
+            {
+                checkInterrupted();
+                LockSupport.parkUntil(until);
+            }
+            return checkAndClear();
+        }
+
+        private void checkInterrupted() throws InterruptedException
+        {
+            if (Thread.interrupted())
+            {
+                cancel();
+                throw new InterruptedException();
+            }
+        }
+    }
+
+    /**
+     * A signal registered with this WaitQueue
+     */
+    private class RegisteredSignal extends AbstractSignal
+    {
+        private volatile Thread thread = Thread.currentThread();
+        volatile int state;
+
+        public boolean isSignalled()
+        {
+            return state == SIGNALLED;
+        }
+
+        public boolean isCancelled()
+        {
+            return state == CANCELLED;
+        }
+
+        public boolean isSet()
+        {
+            return state != NOT_SET;
+        }
+
+        private boolean signal()
+        {
+            if (!isSet() && signalledUpdater.compareAndSet(this, NOT_SET, SIGNALLED))
+            {
+                LockSupport.unpark(thread);
+                thread = null;
+                return true;
+            }
+            return false;
+        }
+
+        public boolean checkAndClear()
+        {
+            if (!isSet() && signalledUpdater.compareAndSet(this, NOT_SET, CANCELLED))
+            {
+                thread = null;
+                cleanUpCancelled();
+                return false;
+            }
+            // must now be signalled assuming correct API usage
+            return true;
+        }
+
+        /**
+         * Should only be called by the registered thread. Indicates the signal can be retired,
+         * and if signalled propagates the signal to another waiting thread
+         */
+        public void cancel()
+        {
+            if (isCancelled())
+                return;
+            if (!signalledUpdater.compareAndSet(this, NOT_SET, CANCELLED))
+            {
+                // must already be signalled - switch to cancelled and
+                state = CANCELLED;
+                // propagate the signal
+                WaitQueue.this.signal();
+            }
+            thread = null;
+            cleanUpCancelled();
+        }
+    }
+
+    /**
+     * A RegisteredSignal that stores a TimerContext, and stops the timer when either cancelled or
+     * finished waiting. i.e. if the timer is started when the signal is registered it tracks the
+     * time in between registering and invalidating the signal.
+     */
+    private final class TimedSignal extends RegisteredSignal
+    {
+        private final TimerContext context;
+
+        private TimedSignal(TimerContext context)
+        {
+            this.context = context;
+        }
+
+        @Override
+        public boolean checkAndClear()
+        {
+            context.stop();
+            return super.checkAndClear();
+        }
+
+        @Override
+        public void cancel()
+        {
+            if (!isCancelled())
+            {
+                context.stop();
+                super.cancel();
+            }
+        }
+    }
+
+    /**
+     * An abstract signal wrapping multiple delegate signals
+     */
+    private abstract static class MultiSignal extends AbstractSignal
+    {
+        final Signal[] signals;
+        protected MultiSignal(Signal[] signals)
+        {
+            this.signals = signals;
+        }
+
+        public boolean isCancelled()
+        {
+            for (Signal signal : signals)
+                if (!signal.isCancelled())
+                    return false;
+            return true;
+        }
+
+        public boolean checkAndClear()
+        {
+            for (Signal signal : signals)
+                signal.checkAndClear();
+            return isSignalled();
+        }
+
+        public void cancel()
+        {
+            for (Signal signal : signals)
+                signal.cancel();
+        }
+    }
+
+    /**
+     * A Signal that wraps multiple Signals and returns when any single one of them would have returned
+     */
+    private static class AnySignal extends MultiSignal
+    {
+        protected AnySignal(Signal ... signals)
+        {
+            super(signals);
+        }
+
+        public boolean isSignalled()
+        {
+            for (Signal signal : signals)
+                if (signal.isSignalled())
+                    return true;
+            return false;
+        }
+
+        public boolean isSet()
+        {
+            for (Signal signal : signals)
+                if (signal.isSet())
+                    return true;
+            return false;
+        }
+    }
+
+    /**
+     * A Signal that wraps multiple Signals and returns when all of them would have finished returning
+     */
+    private static class AllSignal extends MultiSignal
+    {
+        protected AllSignal(Signal ... signals)
+        {
+            super(signals);
+        }
+
+        public boolean isSignalled()
+        {
+            for (Signal signal : signals)
+                if (!signal.isSignalled())
+                    return false;
+            return true;
+        }
+
+        public boolean isSet()
+        {
+            for (Signal signal : signals)
+                if (!signal.isSet())
+                    return false;
+            return true;
+        }
+    }
+
+    /**
+     * @param signals
+     * @return a signal that returns only when any of the provided signals would have returned
+     */
+    public static Signal any(Signal ... signals)
+    {
+        return new AnySignal(signals);
+    }
+
+    /**
+     * @param signals
+     * @return a signal that returns only when all provided signals would have returned
+     */
+    public static Signal all(Signal ... signals)
+    {
+        return new AllSignal(signals);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/src/java/org/apache/cassandra/utils/memory/AbstractAllocator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/memory/AbstractAllocator.java b/src/java/org/apache/cassandra/utils/memory/AbstractAllocator.java
new file mode 100644
index 0000000..8ebdf30
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/memory/AbstractAllocator.java
@@ -0,0 +1,62 @@
+/*
+ * 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.ByteBufferUtil;
+
+import java.nio.ByteBuffer;
+
+public abstract class AbstractAllocator
+{
+    /**
+     * Allocate a slice of the given length.
+     */
+    public ByteBuffer clone(ByteBuffer buffer)
+    {
+        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 abstract ByteBuffer allocate(int size);
+
+    //
+    // only really applicable to Pooled subclasses, but we provide default implementations here
+    //
+
+    public long owns()
+    {
+        return 0;
+    }
+
+    public float ownershipRatio()
+    {
+        return 0;
+    }
+
+    public long reclaiming()
+    {
+        return 0;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/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
new file mode 100644
index 0000000..c58340e
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/memory/ContextAllocator.java
@@ -0,0 +1,59 @@
+package org.apache.cassandra.utils.memory;
+
+import com.google.common.base.*;
+import org.apache.cassandra.db.Cell;
+import org.apache.cassandra.utils.concurrent.OpOrder;
+import org.apache.cassandra.db.ColumnFamilyStore;
+
+import java.nio.ByteBuffer;
+
+/**
+ * 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>
+{
+    private final OpOrder.Group opGroup;
+    private final PoolAllocator allocator;
+    private final ColumnFamilyStore cfs;
+
+    public ContextAllocator(OpOrder.Group opGroup, PoolAllocator allocator, ColumnFamilyStore cfs)
+    {
+        this.opGroup = opGroup;
+        this.allocator = allocator;
+        this.cfs = cfs;
+    }
+
+    @Override
+    public ByteBuffer clone(ByteBuffer buffer)
+    {
+        return allocator.clone(buffer, opGroup);
+    }
+
+    public ByteBuffer allocate(int size)
+    {
+        return allocator.allocate(size, opGroup);
+    }
+
+    public Cell apply(Cell column)
+    {
+        return column.localCopy(cfs, this);
+    }
+
+    public long owns()
+    {
+        return allocator.owns();
+    }
+
+    @Override
+    public float ownershipRatio()
+    {
+        return allocator.ownershipRatio();
+    }
+
+    @Override
+    public long reclaiming()
+    {
+        return allocator.reclaiming();
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/src/java/org/apache/cassandra/utils/memory/HeapAllocator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/memory/HeapAllocator.java b/src/java/org/apache/cassandra/utils/memory/HeapAllocator.java
new file mode 100644
index 0000000..86cea64
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/memory/HeapAllocator.java
@@ -0,0 +1,38 @@
+/*
+ * 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 sun.reflect.generics.reflectiveObjects.NotImplementedException;
+
+public final class HeapAllocator extends AbstractAllocator
+{
+    public static final HeapAllocator instance = new HeapAllocator();
+
+    /**
+     * Normally you should use HeapAllocator.instance, since there is no per-Allocator state.
+     * This is exposed so that the reflection done by Memtable works when SlabAllocator is disabled.
+     */
+    private HeapAllocator() {}
+
+    public ByteBuffer allocate(int size)
+    {
+        return ByteBuffer.allocate(size);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/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
new file mode 100644
index 0000000..bc293cb
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/memory/HeapPool.java
@@ -0,0 +1,16 @@
+package org.apache.cassandra.utils.memory;
+
+import org.apache.cassandra.utils.concurrent.OpOrder;
+
+public class HeapPool extends Pool
+{
+    public HeapPool(long maxOnHeapMemory, float cleanupThreshold, Runnable cleaner)
+    {
+        super(maxOnHeapMemory, cleanupThreshold, cleaner);
+    }
+
+    public HeapPoolAllocator newAllocator(OpOrder writes)
+    {
+        return new HeapPoolAllocator(this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/src/java/org/apache/cassandra/utils/memory/HeapPoolAllocator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/memory/HeapPoolAllocator.java b/src/java/org/apache/cassandra/utils/memory/HeapPoolAllocator.java
new file mode 100644
index 0000000..cf798d0
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/memory/HeapPoolAllocator.java
@@ -0,0 +1,47 @@
+/*
+ * 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.OpOrder;
+
+import java.nio.ByteBuffer;
+
+public final class HeapPoolAllocator extends PoolAllocator
+{
+    HeapPoolAllocator(HeapPool pool)
+    {
+        super(pool);
+    }
+
+    public ByteBuffer allocate(int size)
+    {
+        return allocate(size, null);
+    }
+
+    public ByteBuffer allocate(int size, OpOrder.Group opGroup)
+    {
+        markAllocated(size, opGroup);
+        // must loop trying to acquire
+        return ByteBuffer.allocate(size);
+    }
+
+    public void free(ByteBuffer name)
+    {
+        release(name.remaining());
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/src/java/org/apache/cassandra/utils/memory/HeapSlabAllocator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/memory/HeapSlabAllocator.java b/src/java/org/apache/cassandra/utils/memory/HeapSlabAllocator.java
new file mode 100644
index 0000000..4396caf
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/memory/HeapSlabAllocator.java
@@ -0,0 +1,204 @@
+/*
+ * 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 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.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.concurrent.OpOrder;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The SlabAllocator is a bump-the-pointer allocator that allocates
+ * large (2MB by default) regions and then doles them out to threads that request
+ * slices into the array.
+ * <p/>
+ * The purpose of this class is to combat heap fragmentation in long lived
+ * objects: by ensuring that all allocations with similar lifetimes
+ * only to large regions of contiguous memory, we ensure that large blocks
+ * get freed up at the same time.
+ * <p/>
+ * Otherwise, variable length byte arrays allocated end up
+ * interleaved throughout the heap, and the old generation gets progressively
+ * more fragmented until a stop-the-world compacting collection occurs.
+ */
+public class HeapSlabAllocator extends PoolAllocator
+{
+    private static final Logger logger = LoggerFactory.getLogger(HeapSlabAllocator.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<Region>();
+    private final AtomicInteger regionCount = new AtomicInteger(0);
+    private AtomicLong unslabbed = new AtomicLong(0);
+
+    HeapSlabAllocator(Pool pool)
+    {
+        super(pool);
+    }
+
+    public ByteBuffer allocate(int size)
+    {
+        return allocate(size, null);
+    }
+
+    public ByteBuffer allocate(int size, OpOrder.Group opGroup)
+    {
+        assert size >= 0;
+        if (size == 0)
+            return ByteBufferUtil.EMPTY_BYTE_BUFFER;
+
+        markAllocated(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);
+            return ByteBuffer.allocate(size);
+        }
+
+        while (true)
+        {
+            Region region = getRegion();
+
+            // Try to allocate from this region
+            ByteBuffer cloned = region.allocate(size);
+            if (cloned != null)
+                return cloned;
+
+            // not enough space!
+            currentRegion.compareAndSet(region, null);
+        }
+    }
+
+    public void free(ByteBuffer name)
+    {
+        // have to assume we cannot free the memory here, and just reclaim it all when we flush
+    }
+
+    /**
+     * 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(REGION_SIZE);
+            if (currentRegion.compareAndSet(null, 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 ByteBuffer data;
+
+        /**
+         * 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 size in bytes
+         */
+        private Region(int size)
+        {
+            data = ByteBuffer.allocate(size);
+        }
+
+        /**
+         * Try to allocate <code>size</code> bytes from the region.
+         *
+         * @return the successful allocation, or null to indicate not-enough-space
+         */
+        public ByteBuffer allocate(int size)
+        {
+            while (true)
+            {
+                int oldOffset = nextFreeOffset.get();
+
+                if (oldOffset + size > data.capacity()) // capacity == remaining
+                    return null;
+
+                // Try to atomically claim this region
+                if (nextFreeOffset.compareAndSet(oldOffset, oldOffset + size))
+                {
+                    // we got the alloc
+                    allocCount.incrementAndGet();
+                    return (ByteBuffer) data.duplicate().position(oldOffset).limit(oldOffset + size);
+                }
+                // we raced and lost alloc, try again
+            }
+        }
+
+        @Override
+        public String toString()
+        {
+            return "Region@" + System.identityHashCode(this) +
+                   " allocs=" + allocCount.get() + "waste=" +
+                   (data.capacity() - nextFreeOffset.get());
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/src/java/org/apache/cassandra/utils/memory/HeapSlabPool.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/memory/HeapSlabPool.java b/src/java/org/apache/cassandra/utils/memory/HeapSlabPool.java
new file mode 100644
index 0000000..bd1ab98
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/memory/HeapSlabPool.java
@@ -0,0 +1,16 @@
+package org.apache.cassandra.utils.memory;
+
+import org.apache.cassandra.utils.concurrent.OpOrder;
+
+public class HeapSlabPool extends Pool
+{
+    public HeapSlabPool(long maxOnHeapMemory, float cleanupThreshold, Runnable cleaner)
+    {
+        super(maxOnHeapMemory, cleanupThreshold, cleaner);
+    }
+
+    public HeapSlabAllocator newAllocator(OpOrder writes)
+    {
+        return new HeapSlabAllocator(this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/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
new file mode 100644
index 0000000..4e59de8
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/memory/Pool.java
@@ -0,0 +1,140 @@
+package org.apache.cassandra.utils.memory;
+
+import org.apache.cassandra.utils.concurrent.OpOrder;
+import org.apache.cassandra.utils.concurrent.WaitQueue;
+
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicLongFieldUpdater;
+
+
+/**
+ * Represents an amount of memory used for a given purpose, that can be allocated to specific tasks through
+ * child AbstractAllocator objects. AbstractAllocator and MemoryTracker correspond approximately to PoolAllocator and Pool,
+ * respectively, with the MemoryTracker bookkeeping the total shared use of resources, and the AbstractAllocator the amount
+ * checked out and in use by a specific PoolAllocator.
+ *
+ * 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 abstract class Pool
+{
+    // 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
+    private AtomicLong allocated = new AtomicLong();
+    private AtomicLong reclaiming = new AtomicLong();
+
+    final WaitQueue hasRoom = new WaitQueue();
+
+    // a cache of the calculation determining at what allocation threshold we should next clean, and the cleaner we trigger
+    private volatile long nextClean;
+    private final PoolCleanerThread<?> cleanerThread;
+
+    public Pool(long limit, float cleanThreshold, Runnable cleaner)
+    {
+        this.limit = limit;
+        this.cleanThreshold = cleanThreshold;
+        updateNextClean();
+        cleanerThread = cleaner == null ? null : new PoolCleanerThread<>(this, cleaner);
+        if (cleanerThread != null)
+            cleanerThread.start();
+    }
+
+    /** Methods for tracking and triggering a clean **/
+
+    boolean needsCleaning()
+    {
+        return used() >= nextClean && updateNextClean() && cleanerThread != null;
+    }
+
+    void maybeClean()
+    {
+        if (needsCleaning())
+            cleanerThread.trigger();
+    }
+
+    private boolean updateNextClean()
+    {
+        long reclaiming = this.reclaiming.get();
+        return used() >= (nextClean = reclaiming
+                + (long) (this.limit * cleanThreshold));
+    }
+
+    /** Methods to allocate space **/
+
+    boolean tryAllocate(int size)
+    {
+        while (true)
+        {
+            long cur;
+            if ((cur = allocated.get()) + size > limit)
+                return false;
+            if (allocated.compareAndSet(cur, cur + size))
+            {
+                maybeClean();
+                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.get();
+            if (allocated.compareAndSet(cur, cur + size))
+            {
+                if (size > 0)
+                {
+                    maybeClean();
+                }
+                return;
+            }
+        }
+    }
+
+    void release(long size)
+    {
+        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;
+        this.reclaiming.addAndGet(reclaiming);
+        if (reclaiming < 0 && updateNextClean() && cleanerThread != null)
+            cleanerThread.trigger();
+    }
+
+    public long allocated()
+    {
+        return allocated.get();
+    }
+
+    public long used()
+    {
+        return allocated.get();
+    }
+
+    public long reclaiming()
+    {
+        return reclaiming.get();
+    }
+
+    public abstract PoolAllocator newAllocator(OpOrder writes);
+}
+

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/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
new file mode 100644
index 0000000..b30c484
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/memory/PoolAllocator.java
@@ -0,0 +1,175 @@
+/*
+ * 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.OpOrder;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.concurrent.WaitQueue;
+
+import java.nio.ByteBuffer;
+import java.util.concurrent.atomic.AtomicLong;
+
+public abstract class PoolAllocator<P extends Pool> extends AbstractAllocator
+{
+    public final P pool;
+    volatile LifeCycle state = LifeCycle.LIVE;
+
+    static enum LifeCycle
+    {
+        LIVE, DISCARDING, DISCARDED;
+        LifeCycle transition(LifeCycle target)
+        {
+            assert target.ordinal() == ordinal() + 1;
+            return target;
+        }
+    }
+
+    // the amount of memory/resource owned by this object
+    private AtomicLong owns = new AtomicLong();
+    // 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 AtomicLong reclaiming = new AtomicLong();
+
+    PoolAllocator(P pool)
+    {
+        this.pool = pool;
+    }
+
+    /**
+     * Mark this allocator as reclaiming; this will mark the memory it owns as reclaiming, so remove it from
+     * any calculation deciding if further cleaning/reclamation is necessary.
+     */
+    public void setDiscarding()
+    {
+        state = state.transition(LifeCycle.DISCARDING);
+        // mark the memory owned by this allocator as reclaiming
+        long prev = reclaiming.get();
+        long cur = owns.get();
+        reclaiming.set(cur);
+        pool.adjustReclaiming(cur - prev);
+    }
+
+    /**
+     * 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
+        pool.release(owns.getAndSet(0));
+        pool.adjustReclaiming(-reclaiming.get());
+    }
+
+    public abstract ByteBuffer allocate(int size, OpOrder.Group opGroup);
+
+    /** Mark the BB as unused, permitting it to be reclaimed */
+    public abstract void free(ByteBuffer name);
+
+    // mark ourselves as owning memory from the tracker.  meant to be called by subclass
+    // allocate method that actually allocates and returns a ByteBuffer
+    protected void markAllocated(int size, OpOrder.Group opGroup)
+    {
+        while (true)
+        {
+            if (pool.tryAllocate(size))
+            {
+                acquired(size);
+                return;
+            }
+            WaitQueue.Signal signal = opGroup.isBlockingSignal(pool.hasRoom.register());
+            boolean allocated = pool.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 (by-passes any constraints) an amount allocated in the tracker, and owned by us.
+    private void allocated(int size)
+    {
+        pool.adjustAllocated(size);
+        owns.addAndGet(size);
+    }
+
+    // retroactively mark (by-passes any constraints) an amount owned by us
+    private void acquired(int size)
+    {
+        owns.addAndGet(size);
+    }
+
+    // release an amount of memory from our ownership, and deallocate it in the tracker
+    void release(int size)
+    {
+        pool.release(size);
+        owns.addAndGet(-size);
+    }
+
+    public boolean isLive()
+    {
+        return state == LifeCycle.LIVE;
+    }
+
+    /**
+     * 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, ColumnFamilyStore cfs)
+    {
+        return new ContextAllocator(opGroup, this, cfs);
+    }
+
+    @Override
+    public long owns()
+    {
+        return owns.get();
+    }
+
+    @Override
+    public float ownershipRatio()
+    {
+        return owns.get() / (float) pool.limit;
+    }
+
+    @Override
+    public long reclaiming()
+    {
+        return reclaiming.get();
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/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
new file mode 100644
index 0000000..bfae475
--- /dev/null
+++ b/src/java/org/apache/cassandra/utils/memory/PoolCleanerThread.java
@@ -0,0 +1,55 @@
+package org.apache.cassandra.utils.memory;
+
+import org.apache.cassandra.utils.concurrent.WaitQueue;
+
+/**
+ * A thread that reclaims memor 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.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/4b54b8ac/test/long/org/apache/cassandra/concurrent/LongOpOrderTest.java
----------------------------------------------------------------------
diff --git a/test/long/org/apache/cassandra/concurrent/LongOpOrderTest.java b/test/long/org/apache/cassandra/concurrent/LongOpOrderTest.java
new file mode 100644
index 0000000..f13c1b2
--- /dev/null
+++ b/test/long/org/apache/cassandra/concurrent/LongOpOrderTest.java
@@ -0,0 +1,223 @@
+package org.apache.cassandra.concurrent;
+
+import org.apache.cassandra.utils.concurrent.OpOrder;
+
+import org.cliffc.high_scale_lib.NonBlockingHashMap;
+import org.junit.*;
+import org.slf4j.*;
+
+import static org.junit.Assert.*;
+
+import java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+// TODO: we don't currently test SAFE functionality at all!
+// TODO: should also test markBlocking and SyncOrdered
+public class LongOpOrderTest
+{
+
+    private static final Logger logger = LoggerFactory.getLogger(LongOpOrderTest.class);
+
+    static final int CONSUMERS = 4;
+    static final int PRODUCERS = 32;
+
+    static final long RUNTIME = TimeUnit.MINUTES.toMillis(5);
+    static final long REPORT_INTERVAL = TimeUnit.MINUTES.toMillis(1);
+
+    static final Thread.UncaughtExceptionHandler handler = new Thread.UncaughtExceptionHandler()
+    {
+        @Override
+        public void uncaughtException(Thread t, Throwable e)
+        {
+            System.err.println(t.getName() + ": " + e.getMessage());
+            e.printStackTrace();
+        }
+    };
+
+    final OpOrder order = new OpOrder();
+    final AtomicInteger errors = new AtomicInteger();
+
+    class TestOrdering implements Runnable
+    {
+
+        final int[] waitNanos = new int[1 << 16];
+        volatile State state = new State();
+        final ScheduledExecutorService sched;
+
+        TestOrdering(ExecutorService exec, ScheduledExecutorService sched)
+        {
+            this.sched = sched;
+            final ThreadLocalRandom rnd = ThreadLocalRandom.current();
+            for (int i = 0 ; i < waitNanos.length ; i++)
+                waitNanos[i] = rnd.nextInt(5000);
+            for (int i = 0 ; i < PRODUCERS / CONSUMERS ; i++)
+                exec.execute(new Producer());
+            exec.execute(this);
+        }
+
+        @Override
+        public void run()
+        {
+            final long until = System.currentTimeMillis() + RUNTIME;
+            long lastReport = System.currentTimeMillis();
+            long count = 0;
+            long opCount = 0;
+            while (true)
+            {
+                long now = System.currentTimeMillis();
+                if (now > until)
+                    break;
+                if (now > lastReport + REPORT_INTERVAL)
+                {
+                    lastReport = now;
+                    logger.info(String.format("%s: Executed %d barriers with %d operations. %.0f%% complete.",
+                            Thread.currentThread().getName(), count, opCount, 100 * (1 - ((until - now) / (double) RUNTIME))));
+                }
+                try
+                {
+                    Thread.sleep(0, waitNanos[((int) (count & (waitNanos.length - 1)))]);
+                } catch (InterruptedException e)
+                {
+                    e.printStackTrace();
+                }
+
+                final State s = state;
+                s.barrier = order.newBarrier();
+                s.replacement = new State();
+                s.barrier.issue();
+                s.barrier.await();
+                s.check();
+                opCount += s.totalCount();
+                state = s.replacement;
+                sched.schedule(new Runnable()
+                {
+                    @Override
+                    public void run()
+                    {
+                        s.check();
+                    }
+                }, 1, TimeUnit.SECONDS);
+                count++;
+            }
+        }
+
+        class State
+        {
+
+            volatile OpOrder.Barrier barrier;
+            volatile State replacement;
+            final NonBlockingHashMap<OpOrder.Group, AtomicInteger> count = new NonBlockingHashMap<>();
+            int checkCount = -1;
+
+            boolean accept(OpOrder.Group opGroup)
+            {
+                if (barrier != null && !barrier.isAfter(opGroup))
+                    return false;
+                AtomicInteger c;
+                if (null == (c = count.get(opGroup)))
+                {
+                    count.putIfAbsent(opGroup, new AtomicInteger());
+                    c = count.get(opGroup);
+                }
+                c.incrementAndGet();
+                return true;
+            }
+
+            int totalCount()
+            {
+                int c = 0;
+                for (AtomicInteger v : count.values())
+                    c += v.intValue();
+                return c;
+            }
+
+            void check()
+            {
+                boolean delete;
+                if (checkCount >= 0)
+                {
+                    if (checkCount != totalCount())
+                    {
+                        errors.incrementAndGet();
+                        logger.error("Received size changed after barrier finished: {} vs {}", checkCount, totalCount());
+                    }
+                    delete = true;
+                }
+                else
+                {
+                    checkCount = totalCount();
+                    delete = false;
+                }
+                for (Map.Entry<OpOrder.Group, AtomicInteger> e : count.entrySet())
+                {
+                    if (e.getKey().compareTo(barrier.getSyncPoint()) > 0)
+                    {
+                        errors.incrementAndGet();
+                        logger.error("Received an operation that was created after the barrier was issued.");
+                    }
+                    if (TestOrdering.this.count.get(e.getKey()).intValue() != e.getValue().intValue())
+                    {
+                        errors.incrementAndGet();
+                        logger.error("Missing registered operations. {} vs {}", TestOrdering.this.count.get(e.getKey()).intValue(), e.getValue().intValue());
+                    }
+                    if (delete)
+                        TestOrdering.this.count.remove(e.getKey());
+                }
+            }
+
+        }
+
+        final NonBlockingHashMap<OpOrder.Group, AtomicInteger> count = new NonBlockingHashMap<>();
+
+        class Producer implements Runnable
+        {
+            public void run()
+            {
+                while (true)
+                {
+                    AtomicInteger c;
+                    OpOrder.Group opGroup = order.start();
+                    try
+                    {
+                        if (null == (c = count.get(opGroup)))
+                        {
+                            count.putIfAbsent(opGroup, new AtomicInteger());
+                            c = count.get(opGroup);
+                        }
+                        c.incrementAndGet();
+                        State s = state;
+                        while (!s.accept(opGroup))
+                            s = s.replacement;
+                    }
+                    finally
+                    {
+                        opGroup.finishOne();
+                    }
+                }
+            }
+        }
+
+    }
+
+    @Test
+    public void testOrdering() throws InterruptedException
+    {
+        errors.set(0);
+        Thread.setDefaultUncaughtExceptionHandler(handler);
+        final ExecutorService exec = Executors.newCachedThreadPool(new NamedThreadFactory("checker"));
+        final ScheduledExecutorService checker = Executors.newScheduledThreadPool(1, new NamedThreadFactory("checker"));
+        for (int i = 0 ; i < CONSUMERS ; i++)
+            new TestOrdering(exec, checker);
+        exec.shutdown();
+        exec.awaitTermination((long) (RUNTIME * 1.1), TimeUnit.MILLISECONDS);
+        assertTrue(exec.isShutdown());
+        assertTrue(errors.get() == 0);
+    }
+
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/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
new file mode 100644
index 0000000..e7bfe30
--- /dev/null
+++ b/test/long/org/apache/cassandra/db/LongFlushMemtableTest.java
@@ -0,0 +1,72 @@
+package org.apache.cassandra.db;
+/*
+ *
+ * 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.
+ *
+ */
+
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.junit.Test;
+
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.Util;
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.service.MigrationManager;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+public class LongFlushMemtableTest extends SchemaLoader
+{
+    @Test
+    public void testFlushMemtables() throws IOException, ConfigurationException
+    {
+        Keyspace keyspace = Keyspace.open("Keyspace1");
+        for (int i = 0; i < 100; i++)
+        {
+            CFMetaData metadata = CFMetaData.denseCFMetaData(keyspace.getName(), "_CF" + i, UTF8Type.instance);
+            MigrationManager.announceNewColumnFamily(metadata);
+        }
+
+        for (int j = 0; j < 200; j++)
+        {
+            for (int i = 0; i < 100; i++)
+            {
+                Mutation rm = new Mutation("Keyspace1", ByteBufferUtil.bytes("key" + j));
+                ColumnFamily cf = TreeMapBackedSortedColumns.factory.create("Keyspace1", "_CF" + i);
+                // don't cheat by allocating this outside of the loop; that defeats the purpose of deliberately using lots of memory
+                ByteBuffer value = ByteBuffer.allocate(100000);
+                cf.addColumn(new Cell(Util.cellname("c"), value));
+                rm.add(cf);
+                rm.applyUnsafe();
+            }
+        }
+
+        int flushes = 0;
+        for (ColumnFamilyStore cfs : ColumnFamilyStore.all())
+        {
+            if (cfs.name.startsWith("_CF"))
+                flushes += cfs.getMemtableSwitchCount();
+        }
+        assert flushes > 0;
+    }
+}
+

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/test/long/org/apache/cassandra/db/MeteredFlusherTest.java
----------------------------------------------------------------------
diff --git a/test/long/org/apache/cassandra/db/MeteredFlusherTest.java b/test/long/org/apache/cassandra/db/MeteredFlusherTest.java
deleted file mode 100644
index 4bab277..0000000
--- a/test/long/org/apache/cassandra/db/MeteredFlusherTest.java
+++ /dev/null
@@ -1,72 +0,0 @@
-package org.apache.cassandra.db;
-/*
- *
- * 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.
- *
- */
-
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-import org.junit.Test;
-
-import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.Util;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.db.marshal.UTF8Type;
-import org.apache.cassandra.service.MigrationManager;
-import org.apache.cassandra.utils.ByteBufferUtil;
-
-public class MeteredFlusherTest extends SchemaLoader
-{
-    @Test
-    public void testManyMemtables() throws IOException, ConfigurationException
-    {
-        Keyspace keyspace = Keyspace.open("Keyspace1");
-        for (int i = 0; i < 100; i++)
-        {
-            CFMetaData metadata = CFMetaData.denseCFMetaData(keyspace.getName(), "_CF" + i, UTF8Type.instance);
-            MigrationManager.announceNewColumnFamily(metadata);
-        }
-
-        for (int j = 0; j < 200; j++)
-        {
-            for (int i = 0; i < 100; i++)
-            {
-                Mutation rm = new Mutation("Keyspace1", ByteBufferUtil.bytes("key" + j));
-                ColumnFamily cf = TreeMapBackedSortedColumns.factory.create("Keyspace1", "_CF" + i);
-                // don't cheat by allocating this outside of the loop; that defeats the purpose of deliberately using lots of memory
-                ByteBuffer value = ByteBuffer.allocate(100000);
-                cf.addColumn(new Cell(Util.cellname("c"), value));
-                rm.add(cf);
-                rm.applyUnsafe();
-            }
-        }
-
-        int flushes = 0;
-        for (ColumnFamilyStore cfs : ColumnFamilyStore.all())
-        {
-            if (cfs.name.startsWith("_CF"))
-                flushes += cfs.getMemtableSwitchCount();
-        }
-        assert flushes > 0;
-    }
-}
-

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/test/long/org/apache/cassandra/utils/LongBTreeTest.java
----------------------------------------------------------------------
diff --git a/test/long/org/apache/cassandra/utils/LongBTreeTest.java b/test/long/org/apache/cassandra/utils/LongBTreeTest.java
index 9f31743..b8e60de 100644
--- a/test/long/org/apache/cassandra/utils/LongBTreeTest.java
+++ b/test/long/org/apache/cassandra/utils/LongBTreeTest.java
@@ -50,7 +50,7 @@ public class LongBTreeTest
         TreeSet<Integer> canon = new TreeSet<>();
         for (int i = 0 ; i < 10000000 ; i++)
             canon.add(i);
-        Object[] btree = BTree.build(Arrays.asList(Integer.MIN_VALUE, Integer.MAX_VALUE), ICMP, true);
+        Object[] btree = BTree.build(Arrays.asList(Integer.MIN_VALUE, Integer.MAX_VALUE), ICMP, true, null);
         btree = BTree.update(btree, ICMP, canon, true);
         canon.add(Integer.MIN_VALUE);
         canon.add(Integer.MAX_VALUE);
@@ -171,7 +171,7 @@ public class LongBTreeTest
                     canon.putAll(buffer);
                     ctxt.stop();
                     ctxt = BTREE_TIMER.time();
-                    btree = BTree.update(btree, ICMP, buffer.keySet(), true, null, null);
+                    btree = BTree.update(btree, ICMP, buffer.keySet(), true, null);
                     ctxt.stop();
 
                     if (quickEquality)
@@ -200,7 +200,7 @@ public class LongBTreeTest
             String id = String.format("[0..%d)", canon.size());
             System.out.println("Testing " + id);
             Futures.allAsList(testAllSlices(id, cur, canon)).get();
-            cur = BTree.update(cur, ICMP, Arrays.asList(i), true, null, null);
+            cur = BTree.update(cur, ICMP, Arrays.asList(i), true, null);
             canon.add(i);
         }
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/test/unit/org/apache/cassandra/cache/CacheProviderTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cache/CacheProviderTest.java b/test/unit/org/apache/cassandra/cache/CacheProviderTest.java
index aa6d3dd..849c30c 100644
--- a/test/unit/org/apache/cassandra/cache/CacheProviderTest.java
+++ b/test/unit/org/apache/cassandra/cache/CacheProviderTest.java
@@ -142,7 +142,7 @@ public class CacheProviderTest extends SchemaLoader
             this.string = input;
         }
 
-        public long memorySize()
+        public long unsharedHeapSize()
         {
             return string.length();
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/test/unit/org/apache/cassandra/cache/ObjectSizeTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cache/ObjectSizeTest.java b/test/unit/org/apache/cassandra/cache/ObjectSizeTest.java
deleted file mode 100644
index da34711..0000000
--- a/test/unit/org/apache/cassandra/cache/ObjectSizeTest.java
+++ /dev/null
@@ -1,105 +0,0 @@
-package org.apache.cassandra.cache;
-/*
- * 
- * 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.
- * 
- */
-
-
-import java.nio.ByteBuffer;
-import java.util.UUID;
-
-import org.junit.Assert;
-
-import org.apache.cassandra.db.ColumnIndex;
-import org.apache.cassandra.db.DeletionTime;
-import org.apache.cassandra.db.RowIndexEntry;
-import org.apache.cassandra.utils.ObjectSizes;
-import org.github.jamm.MemoryMeter;
-import org.junit.Test;
-
-public class ObjectSizeTest
-{
-    public static final MemoryMeter meter = new MemoryMeter().omitSharedBufferOverhead();
-
-    @Test
-    public void testArraySizes()
-    {
-        long size = ObjectSizes.getArraySize(0, 1);
-        long size2 = meter.measureDeep(new byte[0]);
-        Assert.assertEquals(size, size2);
-    }
-
-    @Test
-    public void testBiggerArraySizes()
-    {
-        long size = ObjectSizes.getArraySize(0, 1);
-        long size2 = meter.measureDeep(new byte[0]);
-        Assert.assertEquals(size, size2);
-
-        size = ObjectSizes.getArraySize(8, 1);
-        size2 = meter.measureDeep(new byte[8]);
-        Assert.assertEquals(size, size2);
-    }
-
-    @Test
-    public void testKeyCacheKey()
-    {
-        KeyCacheKey key = new KeyCacheKey(null, null, ByteBuffer.wrap(new byte[0]));
-        long size = key.memorySize();
-        long size2 = meter.measureDeep(key);
-        Assert.assertEquals(size, size2);
-    }
-
-    @Test
-    public void testKeyCacheValue()
-    {
-        RowIndexEntry entry = new RowIndexEntry(123);
-        long size = entry.memorySize();
-        long size2 = meter.measureDeep(entry);
-        Assert.assertEquals(size, size2);
-    }
-
-    @Test
-    public void testKeyCacheValueWithDelInfo()
-    {
-        RowIndexEntry entry = RowIndexEntry.create(123, new DeletionTime(123, 123), ColumnIndex.nothing());
-        long size = entry.memorySize();
-        long size2 = meter.measureDeep(entry);
-        Assert.assertEquals(size, size2);
-    }
-
-    @Test
-    public void testRowCacheKey()
-    {
-        UUID id = UUID.randomUUID();
-        RowCacheKey key = new RowCacheKey(id, ByteBuffer.wrap(new byte[11]));
-        long size = key.memorySize();
-        long size2 = meter.measureDeep(key) - meter.measureDeep(id);
-        Assert.assertEquals(size, size2);
-    }
-
-    @Test
-    public void testRowCacheSentinel()
-    {
-        RowCacheSentinel sentinel = new RowCacheSentinel(123);
-        long size = sentinel.memorySize();
-        long size2 = meter.measureDeep(sentinel);
-        Assert.assertEquals(size, size2);
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4b54b8ac/test/unit/org/apache/cassandra/concurrent/WaitQueueTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/concurrent/WaitQueueTest.java b/test/unit/org/apache/cassandra/concurrent/WaitQueueTest.java
index c2888bc..15d40dc 100644
--- a/test/unit/org/apache/cassandra/concurrent/WaitQueueTest.java
+++ b/test/unit/org/apache/cassandra/concurrent/WaitQueueTest.java
@@ -1,6 +1,6 @@
 package org.apache.cassandra.concurrent;
 
-import org.apache.cassandra.utils.WaitQueue;
+import org.apache.cassandra.utils.concurrent.WaitQueue;
 import org.junit.*;
 
 import java.util.concurrent.atomic.AtomicBoolean;