bookkeeper-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From zhai...@apache.org
Subject [bookkeeper] branch master updated: ISSUE #479: CompactionTest#testCompactionWithEntryLogRollover failed
Date Tue, 29 Aug 2017 08:05:26 GMT
This is an automated email from the ASF dual-hosted git repository.

zhaijia 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 ae58e92  ISSUE #479: CompactionTest#testCompactionWithEntryLogRollover failed
ae58e92 is described below

commit ae58e92bd3f0c17ed242544a097dd19656d39c42
Author: Sijie Guo <sijie@apache.org>
AuthorDate: Tue Aug 29 16:05:18 2017 +0800

    ISSUE #479: CompactionTest#testCompactionWithEntryLogRollover failed
    
    Descriptions of the changes in this PR:
    
    - restart bookie with disabling gc for #testCompactionWithEntryLogRollover
    - improve other tests to remove `Thread.sleep` and use a more deterministic way for doing
gc
    
    Author: Sijie Guo <sijie@apache.org>
    
    Reviewers: Jia Zhai <None>
    
    This closes #480 from sijie/fix_compaction_test, closes #479
---
 .../bookkeeper/bookie/GarbageCollectorThread.java  |  7 +-
 .../apache/bookkeeper/bookie/CompactionTest.java   | 99 +++++++++++++++++++---
 2 files changed, 89 insertions(+), 17 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 700b2f6..ebebca2 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
@@ -57,7 +57,7 @@ public class GarbageCollectorThread extends SafeRunnable {
     // Maps entry log files to the set of ledgers that comprise the file and the size usage
per ledger
     private Map<Long, EntryLogMetadata> entryLogMetaMap = new ConcurrentHashMap<Long,
EntryLogMetadata>();
 
-    ScheduledExecutorService gcExecutor;
+    private final ScheduledExecutorService gcExecutor;
     Future<?> scheduledFuture = null;
 
     // This is how often we want to run the Garbage Collector Thread (in milliseconds).
@@ -67,16 +67,15 @@ public class GarbageCollectorThread extends SafeRunnable {
     boolean enableMinorCompaction = false;
     final double minorCompactionThreshold;
     final long minorCompactionInterval;
+    long lastMinorCompactionTime;
 
     boolean enableMajorCompaction = false;
     final double majorCompactionThreshold;
     final long majorCompactionInterval;
+    long lastMajorCompactionTime;
 
     final boolean isForceGCAllowWhenNoSpace;
 
-    long lastMinorCompactionTime;
-    long lastMajorCompactionTime;
-
     final boolean isThrottleByBytes;
     final int maxOutstandingRequests;
     final int compactionRateByEntries;
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 602764b..87bbf35 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
@@ -124,6 +124,12 @@ public abstract class CompactionTest extends BookKeeperClusterTestCase
{
         super.setUp();
     }
 
+    private GarbageCollectorThread getGCThread() {
+        assertEquals(1, bs.size());
+        BookieServer server = bs.get(0);
+        return ((InterleavedLedgerStorage) server.getBookie().ledgerStorage).gcThread;
+    }
+
     LedgerHandle[] prepareData(int numEntryLogs, boolean changeNum)
         throws Exception {
         // since an entry log file can hold at most 100 entries
@@ -180,13 +186,22 @@ public abstract class CompactionTest extends BookKeeperClusterTestCase
{
         // restart bookies
         restartBookies(baseConf);
 
+        long lastMinorCompactionTime = getGCThread().lastMinorCompactionTime;
+        long lastMajorCompactionTime = getGCThread().lastMajorCompactionTime;
+
         // remove ledger2 and ledger3
         // so entry log 1 and 2 would have ledger1 entries left
         bkc.deleteLedger(lhs[1].getId());
         bkc.deleteLedger(lhs[2].getId());
         LOG.info("Finished deleting the ledgers contains most entries.");
-        Thread.sleep(baseConf.getMajorCompactionInterval() * 1000
-                   + baseConf.getGcWaitTime());
+
+        assertFalse(getGCThread().enableMajorCompaction);
+        assertFalse(getGCThread().enableMinorCompaction);
+        getGCThread().triggerGC().get();
+
+        // after garbage collection, compaction should not be executed
+        assertEquals(lastMinorCompactionTime, getGCThread().lastMinorCompactionTime);
+        assertEquals(lastMajorCompactionTime, getGCThread().lastMajorCompactionTime);
 
         // entry logs ([0,1].log) should not be compacted.
         for (File ledgerDirectory : tmpDirs) {
@@ -251,17 +266,29 @@ public abstract class CompactionTest extends BookKeeperClusterTestCase
{
 
         // disable major compaction
         baseConf.setMajorCompactionThreshold(0.0f);
+        baseConf.setGcWaitTime(60000);
+        baseConf.setMinorCompactionInterval(120000);
+        baseConf.setMajorCompactionInterval(240000);
 
         // restart bookies
         restartBookies(baseConf);
 
+        long lastMinorCompactionTime = getGCThread().lastMinorCompactionTime;
+        long lastMajorCompactionTime = getGCThread().lastMajorCompactionTime;
+        assertFalse(getGCThread().enableMajorCompaction);
+        assertTrue(getGCThread().enableMinorCompaction);
+
         // remove ledger2 and ledger3
         bkc.deleteLedger(lhs[1].getId());
         bkc.deleteLedger(lhs[2].getId());
 
         LOG.info("Finished deleting the ledgers contains most entries.");
-        Thread.sleep(baseConf.getMinorCompactionInterval() * 1000
-                   + baseConf.getGcWaitTime());
+        getGCThread().enableForceGC();
+        getGCThread().triggerGC().get();
+
+        // after garbage collection, major compaction should not be executed
+        assertEquals(lastMajorCompactionTime, getGCThread().lastMajorCompactionTime);
+        assertTrue(getGCThread().lastMinorCompactionTime > lastMinorCompactionTime);
 
         // entry logs ([0,1,2].log) should be compacted.
         for (File ledgerDirectory : tmpDirs) {
@@ -285,10 +312,18 @@ public abstract class CompactionTest extends BookKeeperClusterTestCase
{
 
         // disable major compaction
         baseConf.setMajorCompactionThreshold(0.0f);
+        baseConf.setGcWaitTime(60000);
+        baseConf.setMinorCompactionInterval(120000);
+        baseConf.setMajorCompactionInterval(240000);
 
         // restart bookies
         restartBookies(baseConf);
 
+        long lastMinorCompactionTime = getGCThread().lastMinorCompactionTime;
+        long lastMajorCompactionTime = getGCThread().lastMajorCompactionTime;
+        assertFalse(getGCThread().enableMajorCompaction);
+        assertTrue(getGCThread().enableMinorCompaction);
+
         for (BookieServer bookieServer : bs) {
             Bookie bookie = bookieServer.getBookie();
             LedgerDirsManager ledgerDirsManager = bookie.getLedgerDirsManager();
@@ -305,7 +340,11 @@ public abstract class CompactionTest extends BookKeeperClusterTestCase
{
         bkc.deleteLedger(lhs[2].getId());
 
         LOG.info("Finished deleting the ledgers contains most entries.");
-        Thread.sleep(baseConf.getMinorCompactionInterval() * 1000 + baseConf.getGcWaitTime());
+        getGCThread().triggerGC().get();
+
+        // after garbage collection, major compaction should not be executed
+        assertEquals(lastMajorCompactionTime, getGCThread().lastMajorCompactionTime);
+        assertEquals(lastMinorCompactionTime, getGCThread().lastMinorCompactionTime);
 
         // entry logs ([0,1,2].log) should still remain, because both major and Minor compaction
are disabled.
         for (File ledgerDirectory : tmpDirs) {
@@ -330,10 +369,18 @@ public abstract class CompactionTest extends BookKeeperClusterTestCase
{
         // here we are setting isForceGCAllowWhenNoSpace to true, so Major and Minor compaction
wont be disabled in case
         // when discs are full
         baseConf.setIsForceGCAllowWhenNoSpace(true);
+        baseConf.setGcWaitTime(60000);
+        baseConf.setMinorCompactionInterval(120000);
+        baseConf.setMajorCompactionInterval(240000);
 
         // restart bookies
         restartBookies(baseConf);
 
+        long lastMinorCompactionTime = getGCThread().lastMinorCompactionTime;
+        long lastMajorCompactionTime = getGCThread().lastMajorCompactionTime;
+        assertFalse(getGCThread().enableMajorCompaction);
+        assertTrue(getGCThread().enableMinorCompaction);
+
         for (BookieServer bookieServer : bs) {
             Bookie bookie = bookieServer.getBookie();
             LedgerDirsManager ledgerDirsManager = bookie.getLedgerDirsManager();
@@ -350,7 +397,12 @@ public abstract class CompactionTest extends BookKeeperClusterTestCase
{
         bkc.deleteLedger(lhs[2].getId());
 
         LOG.info("Finished deleting the ledgers contains most entries.");
-        Thread.sleep(baseConf.getMinorCompactionInterval() * 1000 + baseConf.getGcWaitTime()
+ 500);
+        getGCThread().enableForceGC();
+        getGCThread().triggerGC().get();
+
+        // after garbage collection, major compaction should not be executed
+        assertEquals(lastMajorCompactionTime, getGCThread().lastMajorCompactionTime);
+        assertTrue(getGCThread().lastMinorCompactionTime > lastMinorCompactionTime);
 
         // though all discs are added to filled dirs list, compaction would succeed, because
in EntryLogger for
         // allocating newlog
@@ -391,17 +443,28 @@ public abstract class CompactionTest extends BookKeeperClusterTestCase
{
 
         // disable minor compaction
         baseConf.setMinorCompactionThreshold(0.0f);
+        baseConf.setGcWaitTime(60000);
+        baseConf.setMinorCompactionInterval(120000);
+        baseConf.setMajorCompactionInterval(240000);
 
         // restart bookies
         restartBookies(baseConf);
 
+        long lastMinorCompactionTime = getGCThread().lastMinorCompactionTime;
+        long lastMajorCompactionTime = getGCThread().lastMajorCompactionTime;
+        assertTrue(getGCThread().enableMajorCompaction);
+        assertFalse(getGCThread().enableMinorCompaction);
+
         // remove ledger1 and ledger3
         bkc.deleteLedger(lhs[0].getId());
         bkc.deleteLedger(lhs[2].getId());
         LOG.info("Finished deleting the ledgers contains most entries.");
+        getGCThread().enableForceGC();
+        getGCThread().triggerGC().get();
 
-        Thread.sleep(baseConf.getMajorCompactionInterval() * 1000
-                   + baseConf.getGcWaitTime());
+        // after garbage collection, minor compaction should not be executed
+        assertTrue(getGCThread().lastMinorCompactionTime > lastMinorCompactionTime);
+        assertTrue(getGCThread().lastMajorCompactionTime > lastMajorCompactionTime);
 
         // entry logs ([0,1,2].log) should be compacted
         for (File ledgerDirectory : tmpDirs) {
@@ -423,12 +486,21 @@ public abstract class CompactionTest extends BookKeeperClusterTestCase
{
             lh.close();
         }
 
+        long lastMinorCompactionTime = getGCThread().lastMinorCompactionTime;
+        long lastMajorCompactionTime = getGCThread().lastMajorCompactionTime;
+        assertTrue(getGCThread().enableMajorCompaction);
+        assertTrue(getGCThread().enableMinorCompaction);
+
         // remove ledger1 and ledger2
         bkc.deleteLedger(lhs[0].getId());
         bkc.deleteLedger(lhs[1].getId());
         LOG.info("Finished deleting the ledgers contains less entries.");
-        Thread.sleep(baseConf.getMajorCompactionInterval() * 1000
-                   + baseConf.getGcWaitTime());
+        getGCThread().enableForceGC();
+        getGCThread().triggerGC().get();
+
+        // after garbage collection, minor compaction should not be executed
+        assertTrue(getGCThread().lastMinorCompactionTime > lastMinorCompactionTime);
+        assertTrue(getGCThread().lastMajorCompactionTime > lastMajorCompactionTime);
 
         // entry logs ([0,1,2].log) should not be compacted
         for (File ledgerDirectory : tmpDirs) {
@@ -463,8 +535,9 @@ public abstract class CompactionTest extends BookKeeperClusterTestCase
{
         LOG.info("Finished deleting the ledgers contains most entries.");
         // restart bookies again to roll entry log files.
         restartBookies();
-        Thread.sleep(baseConf.getMajorCompactionInterval() * 1000
-                   + baseConf.getGcWaitTime());
+
+        getGCThread().enableForceGC();
+        getGCThread().triggerGC().get();
 
         // entry logs (0.log) should not be compacted
         // entry logs ([1,2,3].log) should be compacted.
@@ -750,7 +823,7 @@ public abstract class CompactionTest extends BookKeeperClusterTestCase
{
         baseConf.setGcWaitTime(60000);
         baseConf.setMinorCompactionInterval(0);
         baseConf.setMajorCompactionInterval(0);
-        restartBookies();
+        restartBookies(baseConf);
 
         // prepare data
         LedgerHandle[] lhs = prepareData(3, false);

-- 
To stop receiving notification emails like this one, please contact
['"commits@bookkeeper.apache.org" <commits@bookkeeper.apache.org>'].

Mime
View raw message