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;