This is an automated email from the ASF dual-hosted git repository.

eolivelli pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/bookkeeper.git


The following commit(s) were added to refs/heads/master by this push:
     new 3ce2fb7  ISSUE #501: Fix race in CompactionTest
3ce2fb7 is described below

commit 3ce2fb7ed510976413df07b00c81c3383f5494d1
Author: Ivan Kelly <[email protected]>
AuthorDate: Fri Nov 3 17:13:33 2017 +0100

    ISSUE #501: Fix race in CompactionTest
    
    testMinorCompactionWithNoWritableLedgerDirsButIsForceGCAllowWhenNoSpaceIsSet
    was failing regularly on jenkins. The issue was that it was trying to
    kick off a forced garbage collection by setting the force flag and
    triggering. However, in some cases another GC would be running and
    would clear the flag after it had been set, but before the trigger was
    called.
    
    This patch adds a new form of triggering which allows the caller to
    explicitly specify which flags will be active for the GC run.
    
    Author: Ivan Kelly <[email protected]>
    
    Reviewers: Enrico Olivelli <[email protected]>
    
    This closes #690 from ivankelly/make-compat-fail, closes #501
---
 .../bookkeeper/bookie/GarbageCollectorThread.java  | 39 +++++++++++++++++-----
 .../apache/bookkeeper/bookie/CompactionTest.java   |  3 +-
 2 files changed, 31 insertions(+), 11 deletions(-)

diff --git 
a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/GarbageCollectorThread.java
 
b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/GarbageCollectorThread.java
index ebebca2..5dcd4a0 100644
--- 
a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/GarbageCollectorThread.java
+++ 
b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/GarbageCollectorThread.java
@@ -278,7 +278,8 @@ public class GarbageCollectorThread extends SafeRunnable {
     public void enableForceGC() {
         if (forceGarbageCollection.compareAndSet(false, true)) {
             LOG.info("Forced garbage collection triggered by thread: {}", 
Thread.currentThread().getName());
-            triggerGC();
+            triggerGC(true, suspendMajorCompaction.get(),
+                      suspendMinorCompaction.get());
         }
     }
 
@@ -289,11 +290,22 @@ public class GarbageCollectorThread extends SafeRunnable {
         }
     }
 
-    /**
-     * Manually trigger GC (for testing).
-     */
+    Future<?> triggerGC(final boolean force,
+                        final boolean suspendMajor,
+                        final boolean suspendMinor) {
+        return gcExecutor.submit(() -> {
+                runWithFlags(force, suspendMajor, suspendMinor);
+            });
+    }
+
     Future<?> triggerGC() {
-        return gcExecutor.submit(this);
+        final boolean force = forceGarbageCollection.get();
+        final boolean suspendMajor = suspendMajorCompaction.get();
+        final boolean suspendMinor = suspendMinorCompaction.get();
+
+        return gcExecutor.submit(() -> {
+                runWithFlags(force, suspendMajor, suspendMinor);
+            });
     }
 
     public void suspendMajorGC() {
@@ -332,6 +344,19 @@ public class GarbageCollectorThread extends SafeRunnable {
     @Override
     public void safeRun() {
         boolean force = forceGarbageCollection.get();
+        boolean suspendMajor = suspendMajorCompaction.get();
+        boolean suspendMinor = suspendMinorCompaction.get();
+
+        runWithFlags(force, suspendMajor, suspendMinor);
+
+        if (force) {
+            // only set force to false if it had been true when the garbage
+            // collection cycle started
+            forceGarbageCollection.set(false);
+        }
+    }
+
+    public void runWithFlags(boolean force, boolean suspendMajor, boolean 
suspendMinor) {
         if (force) {
             LOG.info("Garbage collector thread forced to perform GC before 
expiry of wait time.");
         }
@@ -346,8 +371,6 @@ public class GarbageCollectorThread extends SafeRunnable {
         // gc entry logs
         doGcEntryLogs();
 
-        boolean suspendMajor = suspendMajorCompaction.get();
-        boolean suspendMinor = suspendMinorCompaction.get();
         if (suspendMajor) {
             LOG.info("Disk almost full, suspend major compaction to slow down 
filling disk.");
         }
@@ -364,7 +387,6 @@ public class GarbageCollectorThread extends SafeRunnable {
             lastMajorCompactionTime = MathUtils.now();
             // and also move minor compaction time
             lastMinorCompactionTime = lastMajorCompactionTime;
-            forceGarbageCollection.set(false);
             return;
         }
 
@@ -375,7 +397,6 @@ public class GarbageCollectorThread extends SafeRunnable {
             doCompactEntryLogs(minorCompactionThreshold);
             lastMinorCompactionTime = MathUtils.now();
         }
-        forceGarbageCollection.set(false);
     }
 
     /**
diff --git 
a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CompactionTest.java
 
b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CompactionTest.java
index 9fcba8f..57b9404 100644
--- 
a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CompactionTest.java
+++ 
b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/CompactionTest.java
@@ -397,8 +397,7 @@ public abstract class CompactionTest extends 
BookKeeperClusterTestCase {
         bkc.deleteLedger(lhs[2].getId());
 
         LOG.info("Finished deleting the ledgers contains most entries.");
-        getGCThread().enableForceGC();
-        getGCThread().triggerGC().get();
+        getGCThread().triggerGC(true, false, false).get();
 
         // after garbage collection, major compaction should not be executed
         assertEquals(lastMajorCompactionTime, 
getGCThread().lastMajorCompactionTime);

-- 
To stop receiving notification emails like this one, please contact
['"[email protected]" <[email protected]>'].

Reply via email to