Fix RecoveryManagerTruncateTest, includes a backport of CASSANDRA-11743

patch by Stefania Alborghetti and Benjamin Lerer; reviewed by Branimir Lambov 
for CASSANDRA-12802


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

Branch: refs/heads/cassandra-2.2
Commit: 079029a44290f78121e3577da60cec93a8ca9f10
Parents: f7ded1c
Author: Stefania Alborghetti <stefania.alborghe...@datastax.com>
Authored: Fri May 27 10:55:23 2016 +0200
Committer: Stefania Alborghetti <stefania.alborghe...@datastax.com>
Committed: Wed Oct 26 08:48:11 2016 +0800

----------------------------------------------------------------------
 CHANGES.txt                                         |  1 +
 .../apache/cassandra/db/commitlog/CommitLog.java    | 15 +++++++++++++--
 .../cassandra/db/commitlog/CommitLogSegment.java    | 16 +++++++++++++++-
 3 files changed, 29 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/079029a4/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index e922635..792d6f8 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.1.17
+ * Fix RecoveryManagerTruncateTest (CASSANDRA-12802)
  * Don't skip sstables based on maxLocalDeletionTime (CASSANDRA-12765)
 
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/079029a4/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 502e704..2484e9b 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLog.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLog.java
@@ -35,6 +35,7 @@ import org.apache.cassandra.config.Config;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.io.FSWriteError;
+import org.apache.cassandra.io.sstable.SSTableDeletingTask;
 import org.apache.cassandra.io.util.DataOutputByteBuffer;
 import org.apache.cassandra.metrics.CommitLogMetrics;
 import org.apache.cassandra.net.MessagingService;
@@ -101,7 +102,7 @@ public class CommitLog implements CommitLogMBean
                 // we used to try to avoid instantiating commitlog (thus 
creating an empty segment ready for writes)
                 // until after recover was finished.  this turns out to be 
fragile; it is less error-prone to go
                 // ahead and allow writes before recover(), and just skip 
active segments when we do.
-                return CommitLogDescriptor.isValid(name) && 
!instance.allocator.manages(name);
+                return CommitLogDescriptor.isValid(name) && 
CommitLogSegment.shouldReplay(name);
             }
         };
 
@@ -367,11 +368,21 @@ public class CommitLog implements CommitLogMBean
 
     /**
      * FOR TESTING PURPOSES. See CommitLogAllocator.
+     *
+     * There is a race at the moment, even if this method
+     * is synchronized we can still create an allocation
+     * on a segment that will be closed in this method,
+     * therefore causing {@link Allocation#awaitDiskSync()} to hang
+     * forever. This typically happens because of the mutations created
+     * by {@link org.apache.cassandra.io.sstable.SSTableReader.GlobalTidy},
+     * that's why we wait for all deletions to complete firtst.
      */
-    public void resetUnsafe()
+    public synchronized void resetUnsafe()
     {
+        SSTableDeletingTask.waitForDeletions();
         sync(true);
         allocator.resetUnsafe();
+        CommitLogSegment.resetReplayLimit();
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/079029a4/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 2ea8701..1093f87 100644
--- a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java
+++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java
@@ -61,6 +61,7 @@ public class CommitLogSegment
 
     private final static long idBase;
     private final static AtomicInteger nextId = new AtomicInteger(1);
+    private static long replayLimitId;
     static
     {
         long maxId = Long.MIN_VALUE;
@@ -69,7 +70,7 @@ public class CommitLogSegment
             if (CommitLogDescriptor.isValid(file.getName()))
                 maxId = 
Math.max(CommitLogDescriptor.fromFileName(file.getName()).id, maxId);
         }
-        idBase = Math.max(System.currentTimeMillis(), maxId + 1);
+        replayLimitId = idBase = Math.max(System.currentTimeMillis(), maxId + 
1);
     }
 
     // The commit log entry overhead in bytes (int: length + int: head 
checksum + int: tail checksum)
@@ -203,6 +204,19 @@ public class CommitLogSegment
         }
     }
 
+    static boolean shouldReplay(String name)
+    {
+        return CommitLogDescriptor.fromFileName(name).id < replayLimitId;
+    }
+
+    /**
+     * FOR TESTING PURPOSES.
+     */
+    static void resetReplayLimit()
+    {
+        replayLimitId = getNextId();
+    }
+
     // allocate bytes in the segment, or return -1 if not enough space
     private int allocate(int size)
     {

Reply via email to