You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@bookkeeper.apache.org by si...@apache.org on 2017/06/28 21:24:12 UTC

[bookkeeper] branch master updated: BOOKKEEPER-1034: Bookie start in RO when diskfull

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

sijie 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 9ddd9e6  BOOKKEEPER-1034: Bookie start in RO when diskfull
9ddd9e6 is described below

commit 9ddd9e6f9e48b03a57a2c78ec2630303abd49782
Author: Charan Reddy Guttapalem <cg...@salesforce.com>
AuthorDate: Wed Jun 28 14:24:05 2017 -0700

    BOOKKEEPER-1034: Bookie start in RO when diskfull
    
    When the disk is above threshold, Bookie goes to RO. If we have to restart the
    bookie, on the way back, bookie tries to create new entrylog and other files,
    which will fail because disk usage is above threshold,
    hence bookie refuses to come up. So with this fix we will try to start in RO
    mode if RO is enabled.
    
    Also, if bookie has died abruptly then it may missed flushing EntryMemtable and
    IndexInMemoryPageManager. So next time when it starts when disc is full
    it is failing to create index file and it is shutting down, though we expect it
    to start in readonlymode. So Bookie should be able to create index file
    though it has reached the diskusagethreshold, while starting the Bookie in
    Readonly Mode. But ofcourse there should be some config to safeguard when
    disk usable space is so low.
    
    Minor fixes in shutdown logic of Bookie and Bookieserver.
    
    Author: Charan Reddy Guttapalem <cg...@salesforce.com>
    
    Reviewers: Jia Zhai <None>, Sijie Guo <si...@apache.org>
    
    This closes #190 from reddycharan/bookiestartinreadonlywhendiskfull
---
 bookkeeper-server/conf/bk_server.conf              |   4 +
 .../java/org/apache/bookkeeper/bookie/Bookie.java  |  80 ++++++----
 .../bookkeeper/bookie/IndexPersistenceMgr.java     |   2 +-
 .../bookkeeper/bookie/LedgerDirsManager.java       |  70 ++++++---
 .../bookkeeper/conf/ServerConfiguration.java       |  24 +++
 .../org/apache/bookkeeper/proto/BookieServer.java  |   4 +-
 .../bookie/BookieInitializationTest.java           | 167 ++++++++++++++++++++-
 7 files changed, 293 insertions(+), 58 deletions(-)

diff --git a/bookkeeper-server/conf/bk_server.conf b/bookkeeper-server/conf/bk_server.conf
index 7f52902..65883da 100644
--- a/bookkeeper-server/conf/bk_server.conf
+++ b/bookkeeper-server/conf/bk_server.conf
@@ -317,3 +317,7 @@ zkTimeout=10000
 
 # Stats Provider Class
 #statsProviderClass=org.apache.bookkeeper.stats.CodahaleMetricsProvider
+
+# Minimum safe Usable size to be available in index directory for Bookie to create Index File while replaying 
+# journal at the time of Bookie Start in Readonly Mode (in bytes)
+# minUsableSizeForIndexFileCreation=1073741824
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Bookie.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Bookie.java
index c63333f..789cf33 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Bookie.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Bookie.java
@@ -22,10 +22,17 @@
 package org.apache.bookkeeper.bookie;
 
 import static com.google.common.base.Charsets.UTF_8;
-
-import com.google.common.util.concurrent.SettableFuture;
-import io.netty.buffer.ByteBuf;
-import io.netty.buffer.Unpooled;
+import static org.apache.bookkeeper.bookie.BookKeeperServerStats.BOOKIE_ADD_ENTRY;
+import static org.apache.bookkeeper.bookie.BookKeeperServerStats.BOOKIE_ADD_ENTRY_BYTES;
+import static org.apache.bookkeeper.bookie.BookKeeperServerStats.BOOKIE_READ_ENTRY;
+import static org.apache.bookkeeper.bookie.BookKeeperServerStats.BOOKIE_READ_ENTRY_BYTES;
+import static org.apache.bookkeeper.bookie.BookKeeperServerStats.BOOKIE_RECOVERY_ADD_ENTRY;
+import static org.apache.bookkeeper.bookie.BookKeeperServerStats.JOURNAL_SCOPE;
+import static org.apache.bookkeeper.bookie.BookKeeperServerStats.LD_INDEX_SCOPE;
+import static org.apache.bookkeeper.bookie.BookKeeperServerStats.LD_LEDGER_SCOPE;
+import static org.apache.bookkeeper.bookie.BookKeeperServerStats.READ_BYTES;
+import static org.apache.bookkeeper.bookie.BookKeeperServerStats.SERVER_STATUS;
+import static org.apache.bookkeeper.bookie.BookKeeperServerStats.WRITE_BYTES;
 
 import java.io.File;
 import java.io.FileNotFoundException;
@@ -34,21 +41,20 @@ import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.UnknownHostException;
 import java.nio.ByteBuffer;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
 import java.util.concurrent.Callable;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.stream.Collectors;
 
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Sets;
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import org.apache.bookkeeper.bookie.Journal.JournalScanner;
 import org.apache.bookkeeper.bookie.LedgerDirsManager.LedgerDirsListener;
 import org.apache.bookkeeper.bookie.LedgerDirsManager.NoWritableLedgerDirException;
@@ -67,6 +73,7 @@ import org.apache.bookkeeper.util.BookKeeperConstants;
 import org.apache.bookkeeper.util.DiskChecker;
 import org.apache.bookkeeper.util.IOUtils;
 import org.apache.bookkeeper.util.MathUtils;
+import org.apache.bookkeeper.util.ZkUtils;
 import org.apache.bookkeeper.util.collections.ConcurrentLongHashMap;
 import org.apache.bookkeeper.versioning.Version;
 import org.apache.bookkeeper.versioning.Versioned;
@@ -81,23 +88,19 @@ import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.Watcher.Event.EventType;
 import org.apache.zookeeper.Watcher.Event.KeeperState;
 import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.data.ACL;
 import org.apache.zookeeper.data.Stat;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import static org.apache.bookkeeper.bookie.BookKeeperServerStats.BOOKIE_ADD_ENTRY;
-import static org.apache.bookkeeper.bookie.BookKeeperServerStats.BOOKIE_ADD_ENTRY_BYTES;
-import static org.apache.bookkeeper.bookie.BookKeeperServerStats.BOOKIE_READ_ENTRY;
-import static org.apache.bookkeeper.bookie.BookKeeperServerStats.BOOKIE_READ_ENTRY_BYTES;
-import static org.apache.bookkeeper.bookie.BookKeeperServerStats.BOOKIE_RECOVERY_ADD_ENTRY;
-import static org.apache.bookkeeper.bookie.BookKeeperServerStats.LD_LEDGER_SCOPE;
-import static org.apache.bookkeeper.bookie.BookKeeperServerStats.LD_INDEX_SCOPE;
-import static org.apache.bookkeeper.bookie.BookKeeperServerStats.READ_BYTES;
-import static org.apache.bookkeeper.bookie.BookKeeperServerStats.SERVER_STATUS;
-import static org.apache.bookkeeper.bookie.BookKeeperServerStats.WRITE_BYTES;
-import static org.apache.bookkeeper.bookie.BookKeeperServerStats.JOURNAL_SCOPE;
-import org.apache.bookkeeper.util.ZkUtils;
-import org.apache.zookeeper.data.ACL;
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import com.google.common.util.concurrent.SettableFuture;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
 
 /**
  * Implements a bookie.
@@ -613,15 +616,33 @@ public class Bookie extends BookieCriticalThread {
         this.ledgerMonitor = new LedgerDirsMonitor(conf, 
                                     new DiskChecker(conf.getDiskUsageThreshold(), conf.getDiskUsageWarnThreshold()), 
                                     ledgerDirsManager);
-        this.ledgerMonitor.init();
-        
+        try {
+            this.ledgerMonitor.init();
+        } catch (NoWritableLedgerDirException nle) {
+            // start in read-only mode if no writable dirs and read-only allowed
+            if(!conf.isReadOnlyModeEnabled()) {
+                throw nle;
+            } else {
+                this.transitionToReadOnlyMode();
+            }
+        }
+
         if (null == idxDirs) {
             this.idxMonitor = this.ledgerMonitor;
         } else {
             this.idxMonitor = new LedgerDirsMonitor(conf, 
                                         new DiskChecker(conf.getDiskUsageThreshold(), conf.getDiskUsageWarnThreshold()), 
                                         indexDirsManager);
-            this.idxMonitor.init();
+            try {
+                this.idxMonitor.init();
+            } catch (NoWritableLedgerDirException nle) {
+                // start in read-only mode if no writable dirs and read-only allowed
+                if(!conf.isReadOnlyModeEnabled()) {
+                    throw nle;
+                } else {
+                    this.transitionToReadOnlyMode();
+                }
+            }
         }
 
         // ZK ephemeral node for this Bookie.
@@ -1225,9 +1246,10 @@ public class Bookie extends BookieCriticalThread {
                 if (indexDirsManager != ledgerDirsManager) {
                     idxMonitor.shutdown();
                 }
-
-                // Shutdown the ZK client
-                if(zk != null) zk.close();
+            }
+            // Shutdown the ZK client
+            if (zk != null) {
+                zk.close();
             }
         } catch (InterruptedException ie) {
             LOG.error("Interrupted during shutting down bookie : ", ie);
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/IndexPersistenceMgr.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/IndexPersistenceMgr.java
index a1d128f..81c37fd 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/IndexPersistenceMgr.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/IndexPersistenceMgr.java
@@ -177,7 +177,7 @@ public class IndexPersistenceMgr {
      */
     private File getNewLedgerIndexFile(Long ledger, File excludedDir)
                     throws NoWritableLedgerDirException {
-        File dir = ledgerDirsManager.pickRandomWritableDir(excludedDir);
+        File dir = ledgerDirsManager.pickRandomWritableDirForNewIndexFile(excludedDir);
         String ledgerName = getLedgerName(ledger);
         return new File(dir, ledgerName);
     }
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDirsManager.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDirsManager.java
index da56950..66782ac 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDirsManager.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDirsManager.java
@@ -56,6 +56,7 @@ public class LedgerDirsManager {
             new ConcurrentHashMap<File, Float>();
     private final long entryLogSize;
     private boolean forceGCAllowWhenNoSpace;
+    private long minUsableSizeForIndexFileCreation;
 
     public LedgerDirsManager(ServerConfiguration conf, File[] dirs) {
         this(conf, dirs, NullStatsLogger.INSTANCE);
@@ -70,6 +71,7 @@ public class LedgerDirsManager {
         this.listeners = new ArrayList<LedgerDirsListener>();
         this.forceGCAllowWhenNoSpace = conf.getIsForceGCAllowWhenNoSpace();
         this.entryLogSize = conf.getEntryLogSizeLimit();
+        this.minUsableSizeForIndexFileCreation = conf.getMinUsableSizeForIndexFileCreation();
         for (File dir : dirs) {
             diskUsages.put(dir, 0f);
             String statName = "dir_" + dir.getPath().replace('/', '_') + "_usage";
@@ -172,9 +174,7 @@ public class LedgerDirsManager {
         return !writableLedgerDirectories.isEmpty();
     }
 
-    public List<File> getWritableLedgerDirsForNewLog()
-        throws NoWritableLedgerDirException {
-
+    public List<File> getWritableLedgerDirsForNewLog() throws NoWritableLedgerDirException {
         if (!writableLedgerDirectories.isEmpty()) {
             return writableLedgerDirectories;
         }
@@ -191,27 +191,32 @@ public class LedgerDirsManager {
         // That means we must have turned readonly but the compaction
         // must have started running and it needs to allocate
         // a new log file to move forward with the compaction.
-        List<File> fullLedgerDirsToAccomodateNewEntryLog = new ArrayList<File>();
+        return getDirsAboveUsableThresholdSize((long) (this.entryLogSize * 1.2));
+    }
+
+    List<File> getDirsAboveUsableThresholdSize(long thresholdSize) throws NoWritableLedgerDirException {
+        List<File> fullLedgerDirsToAccomodate = new ArrayList<File>();
         for (File dir: this.ledgerDirectories) {
-            // Pick dirs which can accommodate little more than an entry log.
-            if (dir.getUsableSpace() > (this.entryLogSize * 1.2) ) {
-                fullLedgerDirsToAccomodateNewEntryLog.add(dir);
+            // Pick dirs which can accommodate little more than thresholdSize
+            if (dir.getUsableSpace() > (thresholdSize) ) {
+                fullLedgerDirsToAccomodate.add(dir);
             }
         }
 
-        if (!fullLedgerDirsToAccomodateNewEntryLog.isEmpty()) {
-            LOG.info("No writable ledger dirs. Trying to go beyond to accomodate compaction."
-                    + "Dirs that can accomodate new entryLog are: {}", fullLedgerDirsToAccomodateNewEntryLog);
-            return fullLedgerDirsToAccomodateNewEntryLog;
+        if (!fullLedgerDirsToAccomodate.isEmpty()) {
+            LOG.info("No writable ledger dirs below diskUsageThreshold. "
+                    + "But Dirs that can accomodate {} are: {}", thresholdSize, fullLedgerDirsToAccomodate);
+            return fullLedgerDirsToAccomodate;
         }
 
-        // We will reach here when we have no option of creating a new log file for compaction
-        String errMsg = "All ledger directories are non writable and no reserved space left for creating entry log file.";
+        // We will reach here when we find no ledgerDir which has atleast
+        // thresholdSize usable space
+        String errMsg = "All ledger directories are non writable and no reserved space (" + thresholdSize + ") left.";
         NoWritableLedgerDirException e = new NoWritableLedgerDirException(errMsg);
         LOG.error(errMsg, e);
         throw e;
     }
-
+    
     /**
      * @return full-filled ledger dirs.
      */
@@ -294,20 +299,47 @@ public class LedgerDirsManager {
      */
     File pickRandomWritableDir(File excludedDir) throws NoWritableLedgerDirException {
         List<File> writableDirs = getWritableLedgerDirs();
+        return pickRandomDir(writableDirs, excludedDir);
+    }
 
-        final int start = rand.nextInt(writableDirs.size());
+    /**
+     * Pick up a dir randomly from writableLedgerDirectories. If writableLedgerDirectories is empty
+     * then pick up a dir randomly from the ledger/indexdirs which have usable space more than
+     * minUsableSizeForIndexFileCreation.
+     * 
+     * @param excludedDir
+     *          The directory to exclude during pickup.
+     * @return
+     * @throws NoWritableLedgerDirException if there is no dir available.
+     */
+    File pickRandomWritableDirForNewIndexFile(File excludedDir) throws NoWritableLedgerDirException {
+        final List<File> writableDirsForNewIndexFile;
+        if (!writableLedgerDirectories.isEmpty()) {
+            writableDirsForNewIndexFile = writableLedgerDirectories;
+        } else {
+            // We don't have writable Index Dirs.
+            // That means we must have turned readonly. But 
+            // during the Bookie restart, while replaying the journal there might be a need 
+            // to create new Index file and it should proceed.
+            writableDirsForNewIndexFile = getDirsAboveUsableThresholdSize(minUsableSizeForIndexFileCreation);
+        }
+        return pickRandomDir(writableDirsForNewIndexFile, excludedDir);
+    }
+    
+    File pickRandomDir(List<File> dirs, File excludedDir) throws NoWritableLedgerDirException{
+        final int start = rand.nextInt(dirs.size());
         int idx = start;
-        File candidate = writableDirs.get(idx);
+        File candidate = dirs.get(idx);
         while (null != excludedDir && excludedDir.equals(candidate)) {
-            idx = (idx + 1) % writableDirs.size();
+            idx = (idx + 1) % dirs.size();
             if (idx == start) {
                 // after searching all available dirs,
                 // no writable dir is found
                 throw new NoWritableLedgerDirException("No writable directories found from "
-                        + " available writable dirs (" + writableDirs + ") : exclude dir "
+                        + " available writable dirs (" + dirs + ") : exclude dir "
                         + excludedDir);
             }
-            candidate = writableDirs.get(idx);
+            candidate = dirs.get(idx);
         }
         return candidate;
     }
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/ServerConfiguration.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/ServerConfiguration.java
index a3844d1..28952f2 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/ServerConfiguration.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/ServerConfiguration.java
@@ -142,6 +142,8 @@ public class ServerConfiguration extends AbstractConfiguration {
 
     // Bookie auth provider factory class name
     protected final static String BOOKIE_AUTH_PROVIDER_FACTORY_CLASS = "bookieAuthProviderFactoryClass";
+    
+    protected final static String MIN_USABLESIZE_FOR_INDEXFILE_CREATION = "minUsableSizeForIndexFileCreation";
 
     /**
      * Construct a default configuration object
@@ -1939,4 +1941,26 @@ public class ServerConfiguration extends AbstractConfiguration {
         super.setNettyMaxFrameSizeBytes(maxSize);
         return this;
     }
+    
+    /**
+     * Gets the minimum safe Usable size to be available in index directory for Bookie to create Index File while replaying 
+     * journal at the time of Bookie Start in Readonly Mode (in bytes)
+     * 
+     * @return
+     */
+    public long getMinUsableSizeForIndexFileCreation() {
+        return this.getLong(MIN_USABLESIZE_FOR_INDEXFILE_CREATION, 100 * 1024 * 1024L);
+    }
+
+    /**
+     * Sets the minimum safe Usable size to be available in index directory for Bookie to create Index File while replaying 
+     * journal at the time of Bookie Start in Readonly Mode (in bytes)
+     * 
+     * @param minUsableSizeForIndexFileCreation
+     * @return
+     */
+    public ServerConfiguration setMinUsableSizeForIndexFileCreation(long minUsableSizeForIndexFileCreation) {
+        this.setProperty(MIN_USABLESIZE_FOR_INDEXFILE_CREATION, Long.toString(minUsableSizeForIndexFileCreation));
+        return this;
+    }
 }
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieServer.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieServer.java
index 87bc60e..42a2bb1 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieServer.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieServer.java
@@ -161,11 +161,11 @@ public class BookieServer {
     }
 
     public synchronized void shutdown() {
+        LOG.info("Shutting down BookieServer");
+        this.nettyServer.shutdown();
         if (!running) {
             return;
         }
-        LOG.info("Shutting down BookieServer");
-        this.nettyServer.shutdown();
         exitCode = bookie.shutdown();
         if (isAutoRecoveryDaemonEnabled && this.autoRecoveryMain != null) {
             this.autoRecoveryMain.shutdown();
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieInitializationTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieInitializationTest.java
index 35a9c02..b3e6ca9 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieInitializationTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/BookieInitializationTest.java
@@ -20,6 +20,7 @@
  */
 package org.apache.bookkeeper.bookie;
 
+import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 import java.io.File;
@@ -27,11 +28,19 @@ import java.io.IOException;
 import java.net.BindException;
 import java.net.InetAddress;
 
+import org.apache.bookkeeper.bookie.LedgerDirsManager.NoWritableLedgerDirException;
+import org.apache.bookkeeper.client.BookKeeper;
+import org.apache.bookkeeper.client.BookKeeper.DigestType;
 import org.apache.bookkeeper.client.BookKeeperAdmin;
+import org.apache.bookkeeper.client.LedgerHandle;
 import org.apache.bookkeeper.conf.ClientConfiguration;
 import org.apache.bookkeeper.conf.ServerConfiguration;
 import org.apache.bookkeeper.conf.TestBKConfiguration;
 import org.apache.bookkeeper.proto.BookieServer;
+import org.apache.bookkeeper.replication.ReplicationException.CompatibilityException;
+import org.apache.bookkeeper.replication.ReplicationException.UnavailableException;
+import org.apache.bookkeeper.stats.NullStatsLogger;
+import org.apache.bookkeeper.stats.StatsLogger;
 import org.apache.bookkeeper.test.BookKeeperClusterTestCase;
 import org.apache.bookkeeper.util.DiskChecker;
 import org.apache.bookkeeper.zookeeper.ZooKeeperClient;
@@ -327,25 +336,169 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
     }
 
     /**
-     * Check disk full. Expected to throw NoWritableLedgerDirException
-     * during bookie initialisation.
+     * Check disk full. Expected to fail on start.
      */
     @Test(timeout = 30000)
-    public void testWithDiskFull() throws Exception {
+    public void testWithDiskFullReadOnlyDisabledOrForceGCAllowDisabled() throws Exception {
         File tmpDir = createTempDir("DiskCheck", "test");
         long usableSpace = tmpDir.getUsableSpace();
         long totalSpace = tmpDir.getTotalSpace();
         final ServerConfiguration conf = TestBKConfiguration.newServerConfiguration()
                 .setZkServers(zkUtil.getZooKeeperConnectString())
                 .setZkTimeout(5000).setJournalDirName(tmpDir.getPath())
-                .setLedgerDirNames(new String[] { tmpDir.getPath() });
-        conf.setDiskUsageThreshold((1f - ((float) usableSpace / (float) totalSpace)) - 0.05f);
-        conf.setDiskUsageWarnThreshold((1f - ((float) usableSpace / (float) totalSpace)) - 0.25f);
+                .setLedgerDirNames(new String[] { tmpDir.getPath() })
+                .setDiskCheckInterval(1000)
+                .setDiskUsageThreshold((1.0f - ((float) usableSpace / (float) totalSpace)) * 0.999f)
+                .setDiskUsageWarnThreshold(0.0f);
+        
+        // if isForceGCAllowWhenNoSpace or readOnlyModeEnabled is not set and Bookie is 
+        // started when Disk is full, then it will fail to start with NoWritableLedgerDirException
+        
+        conf.setIsForceGCAllowWhenNoSpace(false)
+            .setReadOnlyModeEnabled(false);
         try {
             new Bookie(conf);
-        } catch (Exception e) {
+            fail("NoWritableLedgerDirException expected");
+        } catch(NoWritableLedgerDirException e) {
+            // expected
+        }
+        
+        conf.setIsForceGCAllowWhenNoSpace(true)
+            .setReadOnlyModeEnabled(false);
+        try {
+            new Bookie(conf);
+            fail("NoWritableLedgerDirException expected");
+        } catch(NoWritableLedgerDirException e) {
             // expected
         }
+        
+        conf.setIsForceGCAllowWhenNoSpace(false)
+            .setReadOnlyModeEnabled(true);
+        try {
+            new Bookie(conf);
+            fail("NoWritableLedgerDirException expected");
+        } catch(NoWritableLedgerDirException e) {
+            // expected
+        }
+    }
+    
+    /**
+     * Check disk full. Expected to start as read-only.
+     */
+    @Test(timeout = 30000)
+    public void testWithDiskFullReadOnlyEnabledAndForceGCAllowAllowed() throws Exception {
+        File tmpDir = createTempDir("DiskCheck", "test");
+        long usableSpace = tmpDir.getUsableSpace();
+        long totalSpace = tmpDir.getTotalSpace();
+        final ServerConfiguration conf = TestBKConfiguration.newServerConfiguration()
+                .setZkServers(zkUtil.getZooKeeperConnectString())
+                .setZkTimeout(5000).setJournalDirName(tmpDir.getPath())
+                .setLedgerDirNames(new String[] { tmpDir.getPath() })
+                .setDiskCheckInterval(1000)
+                .setDiskUsageThreshold((1.0f - ((float) usableSpace / (float) totalSpace)) * 0.999f)
+                .setDiskUsageWarnThreshold(0.0f);
+        
+        // if isForceGCAllowWhenNoSpace and readOnlyModeEnabled are set, then Bookie should
+        // start with readonlymode when Disk is full (assuming there is no need for creation of index file
+        // while replaying the journal)
+        conf.setReadOnlyModeEnabled(true)
+            .setIsForceGCAllowWhenNoSpace(true);
+        final Bookie bk = new Bookie(conf);
+        bk.start();
+        Thread.sleep((conf.getDiskCheckInterval() * 2) + 100);
+        
+        assertTrue(bk.isReadOnly());
+        bk.shutdown();
+    }
+
+    class MockBookieServer extends BookieServer {
+        ServerConfiguration conf;
+
+        public MockBookieServer(ServerConfiguration conf) throws IOException, KeeperException, InterruptedException,
+                BookieException, UnavailableException, CompatibilityException {
+            super(conf);
+            this.conf = conf;
+        }
+
+        @Override
+        protected Bookie newBookie(ServerConfiguration conf)
+                throws IOException, KeeperException, InterruptedException, BookieException {
+            return new MockBookieWithNoopShutdown(conf, NullStatsLogger.INSTANCE);
+        }
+    }
+
+    class MockBookieWithNoopShutdown extends Bookie {
+        public MockBookieWithNoopShutdown(ServerConfiguration conf, StatsLogger statsLogger)
+                throws IOException, KeeperException, InterruptedException, BookieException {
+            super(conf, statsLogger);
+        }
+
+        // making Bookie Shutdown no-op. Ideally for this testcase we need to
+        // kill bookie abruptly to simulate the scenario where bookie is killed
+        // without execution of shutdownhook (and corresponding shutdown logic).
+        // Since there is no easy way to simulate abrupt kill of Bookie we are
+        // injecting noop Bookie Shutdown
+        @Override
+        synchronized int shutdown(int exitCode) {
+            return exitCode;
+        }
+    }
+    
+    @Test(timeout = 30000)
+    public void testWithDiskFullAndAbilityToCreateNewIndexFile() throws Exception {
+        File tmpDir = createTempDir("DiskCheck", "test");
+
+        final ServerConfiguration conf = TestBKConfiguration.newServerConfiguration()
+                .setZkServers(zkUtil.getZooKeeperConnectString()).setZkTimeout(5000).setJournalDirName(tmpDir.getPath())
+                .setLedgerDirNames(new String[] { tmpDir.getPath() }).setDiskCheckInterval(1000)
+                .setLedgerStorageClass(SortedLedgerStorage.class.getName()).setAutoRecoveryDaemonEnabled(false);
+
+        BookieServer server = new MockBookieServer(conf);
+        server.start();
+        ClientConfiguration clientConf = new ClientConfiguration();
+        clientConf.setZkServers(zkUtil.getZooKeeperConnectString());
+        BookKeeper bkClient = new BookKeeper(clientConf);
+        LedgerHandle lh = bkClient.createLedger(1, 1, 1, DigestType.CRC32, "passwd".getBytes());
+        long entryId = -1;
+        long numOfEntries = 5;
+        for (int i = 0; i < numOfEntries; i++) {
+            entryId = lh.addEntry("data".getBytes());
+        }
+        Assert.assertTrue("EntryId of the recently added entry should be 0", entryId == (numOfEntries - 1));
+        // We want to simulate the scenario where Bookie is killed abruptly, so
+        // SortedLedgerStorage's EntryMemTable and IndexInMemoryPageManager are
+        // not flushed and hence when bookie is restarted it will replay the
+        // journal. Since there is no easy way to kill the Bookie abruptly, we
+        // are injecting no-op shutdown.
+        server.shutdown();
+
+        long usableSpace = tmpDir.getUsableSpace();
+        long totalSpace = tmpDir.getTotalSpace();
+        conf.setDiskUsageThreshold((1.0f - ((float) usableSpace / (float) totalSpace)) * 0.999f)
+                .setDiskUsageWarnThreshold(0.0f).setReadOnlyModeEnabled(true).setIsForceGCAllowWhenNoSpace(true)
+                .setMinUsableSizeForIndexFileCreation(Long.MAX_VALUE);
+        server = new BookieServer(conf);
+        // Now we are trying to start the Bookie, which tries to replay the
+        // Journal. While replaying the Journal it tries to create the IndexFile
+        // for the ledger (whose entries are not flushed). but since we set
+        // minUsableSizeForIndexFileCreation to very high value, it wouldn't. be
+        // able to find any index dir when all discs are full
+        server.start();
+        Assert.assertFalse("Bookie should be Shutdown", server.getBookie().isRunning());
+        server.shutdown();
+
+        // Here we are setting MinUsableSizeForIndexFileCreation to very low
+        // value. So if index dirs are full then it will consider the dirs which
+        // have atleast MinUsableSizeForIndexFileCreation usable space for the
+        // creation of new Index file.
+        conf.setMinUsableSizeForIndexFileCreation(5 * 1024);
+        server = new BookieServer(conf);
+        server.start();
+        Thread.sleep((conf.getDiskCheckInterval() * 2) + 100);
+        Assert.assertTrue("Bookie should be up and running", server.getBookie().isRunning());
+        assertTrue(server.getBookie().isReadOnly());
+        server.shutdown();
+        bkClient.close();
     }
 
     /**

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