You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by al...@apache.org on 2016/08/09 16:00:00 UTC
cassandra git commit: Delay releasing Memtable memory on flush until
PostFlush has finished running
Repository: cassandra
Updated Branches:
refs/heads/trunk 7c60840e9 -> 4878852fe
Delay releasing Memtable memory on flush until PostFlush has finished running
patch by Ariel Weisberg; reviewed by Branimir Lambov for CASSANDRA-12358
Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/4878852f
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/4878852f
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/4878852f
Branch: refs/heads/trunk
Commit: 4878852fe4aae3516c21fdeafac5c5746a93c31f
Parents: 7c60840
Author: Ariel Weisberg <ar...@datastax.com>
Authored: Mon Aug 1 20:48:49 2016 -0400
Committer: Aleksey Yeschenko <al...@apache.org>
Committed: Tue Aug 9 16:58:48 2016 +0100
----------------------------------------------------------------------
CHANGES.txt | 1 +
.../org/apache/cassandra/db/ColumnFamilyStore.java | 14 ++++++++------
2 files changed, 9 insertions(+), 6 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/cassandra/blob/4878852f/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 5161045..65e8aad 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
3.10
+ * Delay releasing Memtable memory on flush until PostFlush has finished running (CASSANDRA-12358)
* Cassandra stress should dump all setting on startup (CASSANDRA-11914)
* Make it possible to compact a given token range (CASSANDRA-10643)
* Allow updating DynamicEndpointSnitch properties via JMX (CASSANDRA-12179)
http://git-wip-us.apache.org/repos/asf/cassandra/blob/4878852f/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 8439111..36f54e7 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@ -866,9 +866,8 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
logFlush();
Flush flush = new Flush(false);
flushExecutor.execute(flush);
- ListenableFutureTask<CommitLogPosition> task = ListenableFutureTask.create(flush.postFlush);
- postFlushExecutor.submit(task);
- return task;
+ postFlushExecutor.execute(flush.postFlushTask);
+ return flush.postFlushTask;
}
}
@@ -1036,6 +1035,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
{
final OpOrder.Barrier writeBarrier;
final List<Memtable> memtables = new ArrayList<>();
+ final ListenableFutureTask<CommitLogPosition> postFlushTask;
final PostFlush postFlush;
final boolean truncate;
@@ -1078,6 +1078,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
// commit log segment position have also completed, i.e. the memtables are done and ready to flush
writeBarrier.issue();
postFlush = new PostFlush(!truncate, writeBarrier, memtables);
+ postFlushTask = ListenableFutureTask.create(postFlush);
}
public void run()
@@ -1214,14 +1215,14 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
// issue a read barrier for reclaiming the memory, and offload the wait to another thread
final OpOrder.Barrier readBarrier = readOrdering.newBarrier();
readBarrier.issue();
- reclaimExecutor.execute(new WrappedRunnable()
+ postFlushTask.addListener(new WrappedRunnable()
{
public void runMayThrow()
{
readBarrier.await();
memtable.setDiscarded();
}
- });
+ }, reclaimExecutor);
}
}
@@ -2211,7 +2212,8 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
{
final Flush flush = new Flush(true);
flushExecutor.execute(flush);
- return postFlushExecutor.submit(flush.postFlush);
+ postFlushExecutor.execute(flush.postFlushTask);
+ return flush.postFlushTask;
}
}