You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by be...@apache.org on 2015/02/19 12:55:04 UTC

[6/9] cassandra git commit: Make CFS.markReferenced() resistant to bad refcounting

Make CFS.markReferenced() resistant to bad refcounting

patch by richard low; reviewed by benedict for CASSANDRA-8829


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

Branch: refs/heads/trunk
Commit: 6db2a2e8a25751fceaf1fc6e48b2095fa93e991f
Parents: a762e5a
Author: Benedict Elliott Smith <be...@apache.org>
Authored: Thu Feb 19 11:51:16 2015 +0000
Committer: Benedict Elliott Smith <be...@apache.org>
Committed: Thu Feb 19 11:51:16 2015 +0000

----------------------------------------------------------------------
 CHANGES.txt                                             |  1 +
 src/java/org/apache/cassandra/db/ColumnFamilyStore.java | 12 +++++++++++-
 2 files changed, 12 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/6db2a2e8/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 076bb65..f68c987 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.0.13:
+ * Make CFS.markReferenced() resistant to bad refcounting (CASSANDRA-8829)
  * Fix StreamTransferTask abort/complete bad refcounting (CASSANDRA-8815)
  * Fix AssertionError when querying a DESC clustering ordered
    table with ASC ordering and paging (CASSANDRA-8767)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6db2a2e8/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 34d3f1d..74cf286 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@ -1452,7 +1452,8 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
 
     private ViewFragment markReferenced(Function<DataTracker.View, List<SSTableReader>> filter)
     {
-        List<SSTableReader> sstables;
+        List<SSTableReader> sstables = null;
+        List<SSTableReader> prevSstables = null;
         DataTracker.View view;
 
         while (true)
@@ -1466,9 +1467,18 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
             }
 
             sstables = filter.apply(view);
+
+            if (null != prevSstables)
+            {
+                // we're trying again so verify we're acquiring something different
+                assert (!new HashSet<>(sstables).containsAll(prevSstables)) : "Next attempt at acquiring " +
+                        "references is trying the same files. There is probably a reference counting bug somewhere.";
+            }
+
             if (SSTableReader.acquireReferences(sstables))
                 break;
             // retry w/ new view
+            prevSstables = sstables;
         }
 
         return new ViewFragment(sstables, Iterables.concat(Collections.singleton(view.memtable), view.memtablesPendingFlush));