You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@bookkeeper.apache.org by eo...@apache.org on 2022/01/12 15:47:27 UTC

[bookkeeper] branch master updated: BP-46: Data integrity check for running without journal (#2936)

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 a65e888  BP-46: Data integrity check for running without journal (#2936)
a65e888 is described below

commit a65e888b25eb966921ff39032e04f367a553d634
Author: Jack Vanlightly <va...@gmail.com>
AuthorDate: Wed Jan 12 16:47:18 2022 +0100

    BP-46: Data integrity check for running without journal (#2936)
    
    * Data integrity check
    
    The integrity check is comprised two parts: a preboot check that
    is triggered by either an unclean shutdown or an invalid cookie.
    The preboot check marks any open ledgers as both fenced and in
    limbo to prevent ledgers affected by potential data loss from
    being written to. The limbo status prevents NoSuchEntry and
    NoSuchLedger responses from being sent which avoid ledger
    truncation from any ledger recovery operations. Finally it sets
    a storage flag that a full check is required.
    
    A new data integrity check service has been to run the full
    integrity check once the bookie is running. If the service
    sees that the full check storage flag is set then it runs
    a full check. This involves scanning the index and comparing
    it against metadata to discover missing entries. Any
    missing entries are sourced from peer bookies by the
    EntryCopier and written to ledger storage.
    
    The data integrity check also has a different cookie validation
    implementation.
    
    The following configurations have been added to the conf
    file:
    - dataIntegrityCheckingEnabled=true/false. False by default.
      This config enables or disables data integrity checking.
      When set to false the legacy cookie validation is used.
    - dataIntegrityStampMissingCookies=true/false. False by
      default. This config allows the data integrity process
      to stamp new cookies if a cookie is missing from a
      directory. The full check will repair any lost data if
      the directory data was lost.
    
    * Cookie verification for data integrity checking
    
    The algorithm differs from that of LegacyCookieValidation in the
    following ways:
    
    - A empty directory isn't considered a fatal condition. It just means
      that the preboot phase of the data integrity checker must run. Once
      the preboot phase runs, it should be safe to stamp the cookies
      again.
    - Bookies are not allowed to change their identity. If they do, manual
      operator intervention is required (which is ok as it is expected
      that an operator would have to intervene to change the identity in
      the first place).
    - A missing cookie in zookeeper is only valid, if there are no cookies
      in any of the directories, as this is considered a new
      boot. Otherwise, manual operator intervention is required.
    
    * Async iterator for ledger metadata
    
    Common code for iterating over ledger metadata. There is already
    asyncProcessLedgers in LedgerManager, but that only gives the ledgerId
    and the API is nasty (it even uses ZK specific callbacks).
    
    This change adds a more modern iterator, which takes a function which
    returns a CompletableFuture. The iterator has rudimentary rate
    limiting, by limiting the number of ledgers which can be processed at
    a time. We should add something more advanced later, which takes into
    account the response time from ZK.
    
    * Add limbo state to bookie ledger representation
    
    Limbo state for a ledger means that we don't know whether we should
    have an entry for the ledger or not, which can happen when a bookie is
    started after having its disk wiped. We cannot response with a
    NoSuchEntryException or NoSuchLedgerException as this tell the client
    that we never had the requested entry, which may or may not be true,
    but if we tell it to the client, the client will act like it's true
    and possible mark the end of the ledger at an incorrect point.
    
    This change also adds locking to LedgerMetadataIndex. Previously it
    relied on the good graces to the calling code to avoid modifying the
    same ledger concurrently. Now that we are also using the index to
    store limbo state, we can't be so blasé.
    
    * Add entryExists call to ledgerStorage
    
    Currently the only way to check if an entry exists in the storage is
    to try to read the entry. However, this means pulling data out of the
    entrylog, which it should be sufficient to check that the entry exists
    in the index.
    
    This change adds the entryExists call to ledgerStorage. This has only
    been implemented for DbLedgerStorage. The implementation for the
    others should be trivial, but it needs to be tested.
    
    * Pregenerate the writeset from ledger metadata
    
    The bookkeeper client uses DistributionSchedule (of which
    RoundRobinDistributionSchedule is the only impl) to decide which
    members of the ensemble it writes an entry to. This writeset is
    generated for each entry. However, there is only |ensemble| possible
    writesets, so we should pregenerate them for the ledger and stop
    trashing memory.
    
    WriteSets represent a set of pregenerated writesets as would be
    otherwise generated from the distribution schedule. The constructor
    takes a list of indices (which should be generated based on the list
    of bookies in the ensemble), which specifies the preferred order that
    bookies should be tried for reads.
    
    * Storage state flags for LedgerStorage
    
    If a bookie crashes in the middle of a full integrity check, it needs
    to know to start it again when it reboots. For this, we need to
    persist some flag to persistent storage.
    
    This change adds persistent flags to the ledger storage
    interface. Multiple flags can be added in future, for example to mark
    the storage as dirty on boot, so we can detect non-clean shutdown.
    
    Flags are only implemented for DbLedgerStorage. The flags are stored
    in the metadata index, with a negative ledger id as key. The key of
    the storage selected for ledger 0 is used. This does mean flags will
    be lost if there is a change in the storage disk configuration, but
    data integrity checks will run in this case regardless.
    
    * EUNKNOWN code
    
    A new response code has been added to communicate that the state
    of an entry is unknown due the ledger being in limbo.
    
    * Added bookie unclean shutdown detection
    
    Adds unclean shutdown detection. When running with journal
    writes disabled and data integrity checking enabled, if
    the prior shutdown was unclean (not a graceful shutdown)
    then the data integrity checks are triggered. These checks
    avoid additional data loss scenarios and repair any lost
    data caused by the loss of unflushed data at the time
    of the unclean shutdown.
    
    The BookieServer registers start-up and shutdown with the
    UncleanShutdownDetection class. This class adds a dirty
    file to each ledger dir on registering start-up and clears
    all these files on registering shutdown. The presence of
    any of these files on boot-up indicates the prior shutdown
    was unclean.
    
    Co-authored-by: Ivan Kelly <ik...@splunk.com>
---
 bookkeeper-common/build.gradle                     |    1 +
 .../component/RxSchedulerLifecycleComponent.java   |   60 +
 .../apache/bookkeeper/common/util/MockTicker.java  |   42 +
 bookkeeper-dist/all/build.gradle                   |    1 +
 bookkeeper-dist/bkctl/build.gradle                 |    1 +
 bookkeeper-dist/server/build.gradle                |    1 +
 bookkeeper-dist/src/assemble/bin-all.xml           |    1 +
 bookkeeper-dist/src/assemble/bin-server.xml        |    1 +
 bookkeeper-dist/src/assemble/bkctl.xml             |    1 +
 .../src/main/resources/LICENSE-all.bin.txt         |    8 +
 .../src/main/resources/LICENSE-bkctl.bin.txt       |    8 +
 .../src/main/resources/LICENSE-server.bin.txt      |    8 +
 .../resources/deps/reactivestreams-1.0.3/LICENSE   |    8 +
 .../src/main/proto/BookkeeperProtocol.proto        |    1 +
 .../main/proto/DbLedgerStorageDataFormats.proto    |    1 +
 bookkeeper-server/build.gradle                     |    1 +
 .../java/org/apache/bookkeeper/bookie/Bookie.java  |    6 +-
 .../apache/bookkeeper/bookie/BookieException.java  |   27 +
 .../org/apache/bookkeeper/bookie/BookieImpl.java   |   16 +-
 .../apache/bookkeeper/bookie/CheckpointSource.java |    2 +-
 .../java/org/apache/bookkeeper/bookie/Cookie.java  |   31 +-
 .../apache/bookkeeper/bookie/CookieValidation.java |    2 +-
 .../bookie/InterleavedLedgerStorage.java           |   43 +
 .../apache/bookkeeper/bookie/LedgerDescriptor.java |    8 +-
 .../bookkeeper/bookie/LedgerDescriptorImpl.java    |    8 +-
 .../apache/bookkeeper/bookie/LedgerStorage.java    |   61 +-
 .../bookkeeper/bookie/SortedLedgerStorage.java     |   44 +-
 .../bookie/UncleanShutdownDetection.java           |   34 +
 .../bookie/UncleanShutdownDetectionImpl.java       |  112 ++
 .../bookie/datainteg/DataIntegrityCheck.java       |   58 +
 .../bookie/datainteg/DataIntegrityCheckImpl.java   |  556 +++++++
 .../datainteg/DataIntegrityCookieValidation.java   |  164 +++
 .../bookie/datainteg/DataIntegrityService.java     |  111 ++
 .../bookkeeper/bookie/datainteg/EntryCopier.java   |   48 +
 .../bookie/datainteg/EntryCopierImpl.java          |  293 ++++
 .../apache/bookkeeper/bookie/datainteg/Events.java |  235 +++
 .../bookie/datainteg/MetadataAsyncIterator.java    |  123 ++
 .../bookkeeper/bookie/datainteg/WriteSets.java     |   88 ++
 .../bookkeeper/bookie/datainteg/package-info.java  |   23 +
 .../bookie/storage/ldb/DbLedgerStorage.java        |   51 +-
 .../bookie/storage/ldb/LedgerMetadataIndex.java    |  255 +++-
 .../bookkeeper/bookie/storage/ldb/ReadCache.java   |   21 +
 .../ldb/SingleDirectoryDbLedgerStorage.java        |  165 ++-
 .../bookkeeper/bookie/storage/ldb/WriteCache.java  |    4 +
 .../org/apache/bookkeeper/client/BKException.java  |    9 +
 .../org/apache/bookkeeper/client/BookKeeper.java   |    2 +-
 .../apache/bookkeeper/client/ClientContext.java    |    2 +-
 .../apache/bookkeeper/client/api/BKException.java  |    7 +
 .../bookkeeper/conf/ServerConfiguration.java       |   52 +
 .../apache/bookkeeper/proto/BookieProtocol.java    |    5 +
 .../org/apache/bookkeeper/proto/BookieServer.java  |   12 +-
 .../proto/LongPollReadEntryProcessorV3.java        |    3 +-
 .../bookkeeper/proto/PerChannelBookieClient.java   |    2 +
 .../bookkeeper/proto/ReadEntryProcessor.java       |    3 +
 .../bookkeeper/proto/ReadEntryProcessorV3.java     |    9 +-
 .../bookkeeper/proto/ReadLacProcessorV3.java       |   15 +-
 .../java/org/apache/bookkeeper/server/Main.java    |  116 +-
 .../bookkeeper/server/service/BookieService.java   |    9 +-
 .../apache/bookkeeper/util/LocalBookKeeper.java    |   10 +-
 .../bookie/BookieInitializationTest.java           |  127 +-
 .../bookie/EnableZkSecurityBasicTest.java          |    3 +-
 .../org/apache/bookkeeper/bookie/EntryLogTest.java |    2 +-
 .../bookie/LedgerStorageCheckpointTest.java        |   18 +-
 .../bookkeeper/bookie/MockLedgerStorage.java       |  307 ++++
 .../bookie/MockUncleanShutdownDetection.java       |   53 +
 .../apache/bookkeeper/bookie/SyncThreadTest.java   |   38 +
 .../bookie/UncleanShutdownDetectionTest.java       |  154 ++
 .../bookie/datainteg/CookieValidationTest.java     |  329 +++++
 .../bookie/datainteg/DataIntegrityCheckTest.java   | 1544 ++++++++++++++++++++
 .../bookie/datainteg/DataIntegrityServiceTest.java |  169 +++
 .../bookie/datainteg/EntryCopierTest.java          |  635 ++++++++
 .../datainteg/MetadataAsyncIteratorTest.java       |  301 ++++
 .../bookie/datainteg/MockDataIntegrityCheck.java   |   40 +
 .../bookkeeper/bookie/datainteg/WriteSetsTest.java |  174 +++
 .../bookie/storage/ldb/DbLedgerStorageTest.java    |  225 ++-
 .../bookie/storage/ldb/ReadCacheTest.java          |   23 +
 .../bookie/storage/ldb/WriteCacheTest.java         |   20 +
 .../BookKeeperClientTestsWithBookieErrors.java     |    3 +-
 .../bookkeeper/client/BookKeeperCloseTest.java     |    2 +-
 .../bookkeeper/client/BookieWriteLedgerTest.java   |    3 +-
 .../client/ParallelLedgerRecoveryTest.java         |    2 +-
 .../client/TestReadLastConfirmedAndEntry.java      |    5 +-
 .../discover/MockRegistrationManager.java          |  141 ++
 .../org/apache/bookkeeper/meta/GcLedgersTest.java  |   37 +
 .../bookkeeper/meta/LedgerManagerTestCase.java     |   37 +
 .../apache/bookkeeper/meta/MockLedgerManager.java  |   35 +-
 .../proto/TestPerChannelBookieClient.java          |    3 +-
 .../bookkeeper/test/BookKeeperClusterTestCase.java |   12 +-
 .../apache/bookkeeper/test/BookieClientTest.java   |    5 +-
 .../apache/bookkeeper/test/BookieZKExpireTest.java |    4 +-
 .../apache/bookkeeper/test/ZooKeeperCluster.java   |    6 +
 build.gradle                                       |    7 +-
 dependencies.gradle                                |    2 +
 .../etcd/testing/EtcdBKClusterTestBase.java        |    4 +-
 .../stream/server/service/BookieService.java       |    6 +-
 95 files changed, 7256 insertions(+), 209 deletions(-)

diff --git a/bookkeeper-common/build.gradle b/bookkeeper-common/build.gradle
index 4b44f68..9699d09 100644
--- a/bookkeeper-common/build.gradle
+++ b/bookkeeper-common/build.gradle
@@ -38,6 +38,7 @@ dependencies {
     implementation depLibs.jacksonDatabind
     implementation depLibs.jctools
     implementation depLibs.nettyCommon
+    implementation depLibs.rxjava
     implementation depLibs.slf4j
     testCompileOnly depLibs.lombok
     testImplementation depLibs.commonsLang3
diff --git a/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/component/RxSchedulerLifecycleComponent.java b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/component/RxSchedulerLifecycleComponent.java
new file mode 100644
index 0000000..ba17597
--- /dev/null
+++ b/bookkeeper-common/src/main/java/org/apache/bookkeeper/common/component/RxSchedulerLifecycleComponent.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.bookkeeper.common.component;
+
+import io.reactivex.rxjava3.core.Scheduler;
+import java.util.concurrent.ExecutorService;
+import org.apache.bookkeeper.common.conf.ComponentConfiguration;
+import org.apache.bookkeeper.stats.StatsLogger;
+
+/**
+ * The scheduler for rxjava based jobs, such as data integrity checking.
+ */
+public class RxSchedulerLifecycleComponent extends AbstractLifecycleComponent<ComponentConfiguration> {
+    private final Scheduler scheduler;
+    private final ExecutorService rxExecutor;
+
+    public RxSchedulerLifecycleComponent(String componentName,
+                                         ComponentConfiguration conf,
+                                         StatsLogger stats,
+                                         Scheduler scheduler,
+                                         ExecutorService rxExecutor) {
+        super(componentName, conf, stats);
+        this.scheduler = scheduler;
+        this.rxExecutor = rxExecutor;
+    }
+
+    @Override
+    protected void doStart() {
+        scheduler.start();
+    }
+
+    @Override
+    protected void doStop() {
+        scheduler.shutdown();
+        rxExecutor.shutdown();
+    }
+
+    @Override
+    public void doClose() {
+        scheduler.shutdown();
+        rxExecutor.shutdown();
+    }
+}
diff --git a/bookkeeper-common/src/test/java/org/apache/bookkeeper/common/util/MockTicker.java b/bookkeeper-common/src/test/java/org/apache/bookkeeper/common/util/MockTicker.java
new file mode 100644
index 0000000..3df980f
--- /dev/null
+++ b/bookkeeper-common/src/test/java/org/apache/bookkeeper/common/util/MockTicker.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.bookkeeper.common.util;
+
+import com.google.common.base.Ticker;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * Test implementation of Ticker.
+ */
+public class MockTicker extends Ticker {
+    private AtomicLong tick = new AtomicLong(0);
+
+    public void advance(int period, TimeUnit unit) {
+        tick.addAndGet(unit.toNanos(period));
+    }
+
+    @Override
+    public long read() {
+        return tick.get();
+    }
+}
+
+
diff --git a/bookkeeper-dist/all/build.gradle b/bookkeeper-dist/all/build.gradle
index 022de5b..68f1043 100644
--- a/bookkeeper-dist/all/build.gradle
+++ b/bookkeeper-dist/all/build.gradle
@@ -64,6 +64,7 @@ def depLicences = [
         "paranamer-2.8/LICENSE.txt",
         "protobuf-3.14.0/LICENSE",
         "protobuf-3.12.0/LICENSE",
+        "reactivestreams-1.0.3/LICENSE",
         "scala-library-2.11.7/LICENSE.md",
         "scala-parser-combinators_2.11-1.0.4/LICENSE.md",
         "scala-reflect-2.11.8/LICENSE.md",
diff --git a/bookkeeper-dist/bkctl/build.gradle b/bookkeeper-dist/bkctl/build.gradle
index 9216673..2495b7b 100644
--- a/bookkeeper-dist/bkctl/build.gradle
+++ b/bookkeeper-dist/bkctl/build.gradle
@@ -53,6 +53,7 @@ def depLicences = [
         "bouncycastle-1.0.2.1/LICENSE.html",
         "protobuf-3.14.0/LICENSE",
         "protobuf-3.12.0/LICENSE",
+        "reactivestreams-1.0.3/LICENSE",
         "slf4j-1.7.32/LICENSE.txt",
 ]
 
diff --git a/bookkeeper-dist/server/build.gradle b/bookkeeper-dist/server/build.gradle
index e7b767f..ba2d50d 100644
--- a/bookkeeper-dist/server/build.gradle
+++ b/bookkeeper-dist/server/build.gradle
@@ -57,6 +57,7 @@ def depLicences = [
         "bouncycastle-1.0.2.1/LICENSE.html",
         "protobuf-3.14.0/LICENSE",
         "protobuf-3.12.0/LICENSE",
+        "reactivestreams-1.0.3/LICENSE",
         "slf4j-1.7.32/LICENSE.txt",
 ]
 distributions {
diff --git a/bookkeeper-dist/src/assemble/bin-all.xml b/bookkeeper-dist/src/assemble/bin-all.xml
index 9b89133..4ec9dea 100644
--- a/bookkeeper-dist/src/assemble/bin-all.xml
+++ b/bookkeeper-dist/src/assemble/bin-all.xml
@@ -63,6 +63,7 @@
         <include>paranamer-2.8/LICENSE.txt</include>
         <include>protobuf-3.14.0/LICENSE</include>
         <include>protobuf-3.12.0/LICENSE</include>
+        <include>reactivestreams-1.0.3/LICENSE</include>
         <include>scala-library-2.11.7/LICENSE.md</include>
         <include>scala-parser-combinators_2.11-1.0.4/LICENSE.md</include>
         <include>scala-reflect-2.11.8/LICENSE.md</include>
diff --git a/bookkeeper-dist/src/assemble/bin-server.xml b/bookkeeper-dist/src/assemble/bin-server.xml
index d07baf6..cfb176d 100644
--- a/bookkeeper-dist/src/assemble/bin-server.xml
+++ b/bookkeeper-dist/src/assemble/bin-server.xml
@@ -56,6 +56,7 @@
         <include>bouncycastle-1.0.2.1/LICENSE.html</include>
         <include>protobuf-3.14.0/LICENSE</include>
         <include>protobuf-3.12.0/LICENSE</include>
+        <include>reactivestreams-1.0.3/LICENSE</include>
         <include>slf4j-1.7.32/LICENSE.txt</include>
       </includes>
       <fileMode>644</fileMode>
diff --git a/bookkeeper-dist/src/assemble/bkctl.xml b/bookkeeper-dist/src/assemble/bkctl.xml
index e10ac31..855f410 100644
--- a/bookkeeper-dist/src/assemble/bkctl.xml
+++ b/bookkeeper-dist/src/assemble/bkctl.xml
@@ -70,6 +70,7 @@
         <include>bouncycastle-1.0.2.1/LICENSE.html</include>
         <include>protobuf-3.14.0/LICENSE</include>
         <include>protobuf-3.12.0/LICENSE</include>
+        <include>reactivestreams-1.0.3/LICENSE</include>
         <include>slf4j-1.7.32/LICENSE.txt</include>
       </includes>
       <fileMode>644</fileMode>
diff --git a/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt b/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt
index ad693a1..310d536 100644
--- a/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt
+++ b/bookkeeper-dist/src/main/resources/LICENSE-all.bin.txt
@@ -307,6 +307,7 @@ Apache Software License, Version 2.
 - lib/io.perfmark-perfmark-api-0.23.0.jar [48]
 - lib/org.conscrypt-conscrypt-openjdk-uber-2.5.1.jar [49]
 - lib/org.xerial.snappy-snappy-java-1.1.7.jar [50]
+- lib/io.reactivex.rxjava3-rxjava-3.0.1.jar [51]
 
 [1] Source available at https://github.com/FasterXML/jackson-annotations/tree/jackson-annotations-2.11.0
 [2] Source available at https://github.com/FasterXML/jackson-core/tree/jackson-core-2.11.3
@@ -355,6 +356,7 @@ Apache Software License, Version 2.
 [48] Source available at https://github.com/perfmark/perfmark/releases/tag/v0.23.0
 [49] Source available at https://github.com/google/conscrypt/releases/tag/2.5.1
 [50] Source available at https://github.com/google/snappy/releases/tag/1.1.7
+[51] Source available at https://github.com/ReactiveX/RxJava/tree/v3.0.1
 
 ------------------------------------------------------------------------------------
 lib/io.netty-netty-codec-4.1.72.Final.jar bundles some 3rd party dependencies
@@ -666,3 +668,9 @@ MIT License. For details, see deps/checker-qual-3.5.0/LICENSE
 
 Bundles as
   - lib/org.checkerframework-checker-qual-3.5.0.jar
+------------------------------------------------------------------------------------
+This product bundles the Reactive Streams library, which is licensed under
+Public Domain (CC0). For details, see deps/reactivestreams-1.0.3/LICENSE
+
+Bundles as
+  - lib/org.reactivestreams-reactive-streams-1.0.3.jar
\ No newline at end of file
diff --git a/bookkeeper-dist/src/main/resources/LICENSE-bkctl.bin.txt b/bookkeeper-dist/src/main/resources/LICENSE-bkctl.bin.txt
index 87ad8ea..db53766 100644
--- a/bookkeeper-dist/src/main/resources/LICENSE-bkctl.bin.txt
+++ b/bookkeeper-dist/src/main/resources/LICENSE-bkctl.bin.txt
@@ -284,6 +284,7 @@ Apache Software License, Version 2.
 - lib/io.perfmark-perfmark-api-0.23.0.jar [47]
 - lib/org.conscrypt-conscrypt-openjdk-uber-2.5.1.jar [49]
 - lib/org.xerial.snappy-snappy-java-1.1.7.jar [50]
+- lib/io.reactivex.rxjava3-rxjava-3.0.1.jar [51]
 
 [1] Source available at https://github.com/FasterXML/jackson-annotations/tree/jackson-annotations-2.11.0
 [2] Source available at https://github.com/FasterXML/jackson-core/tree/jackson-core-2.11.3
@@ -324,6 +325,7 @@ Apache Software License, Version 2.
 [47] Source available at https://github.com/perfmark/perfmark/releases/tag/v0.23.0
 [49] Source available at https://github.com/google/conscrypt/releases/tag/2.5.1
 [50] Source available at https://github.com/google/snappy/releases/tag/1.1.7
+[51] Source available at https://github.com/ReactiveX/RxJava/tree/v3.0.1
 
 ------------------------------------------------------------------------------------
 lib/io.netty-netty-codec-4.1.72.Final.jar bundles some 3rd party dependencies
@@ -593,3 +595,9 @@ MIT License. For details, see deps/checker-qual-3.5.0/LICENSE
 
 Bundles as
   - lib/org.checkerframework-checker-qual-3.5.0.jar
+------------------------------------------------------------------------------------
+This product bundles the Reactive Streams library, which is licensed under
+Public Domain (CC0). For details, see deps/reactivestreams-1.0.3/LICENSE
+
+Bundles as
+  - lib/org.reactivestreams-reactive-streams-1.0.3.jar
\ No newline at end of file
diff --git a/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt b/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt
index 7007748..445bab3 100644
--- a/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt
+++ b/bookkeeper-dist/src/main/resources/LICENSE-server.bin.txt
@@ -305,6 +305,7 @@ Apache Software License, Version 2.
 - lib/io.perfmark-perfmark-api-0.23.0.jar [48]
 - lib/org.conscrypt-conscrypt-openjdk-uber-2.5.1.jar [49]
 - lib/org.xerial.snappy-snappy-java-1.1.7.jar [50]
+- lib/io.reactivex.rxjava3-rxjava-3.0.1.jar [51]
 
 [1] Source available at https://github.com/FasterXML/jackson-annotations/tree/jackson-annotations-2.11.0
 [2] Source available at https://github.com/FasterXML/jackson-core/tree/jackson-core-2.11.3
@@ -353,6 +354,7 @@ Apache Software License, Version 2.
 [48] Source available at https://github.com/perfmark/perfmark/releases/tag/v0.23.0
 [49] Source available at https://github.com/google/conscrypt/releases/tag/2.5.1
 [50] Source available at https://github.com/google/snappy/releases/tag/1.1.7
+[51] Source available at https://github.com/ReactiveX/RxJava/tree/v3.0.1
 
 ------------------------------------------------------------------------------------
 lib/io.netty-netty-codec-4.1.72.Final.jar bundles some 3rd party dependencies
@@ -658,3 +660,9 @@ MIT License. For details, see deps/checker-qual-3.5.0/LICENSE
 
 Bundles as
   - lib/org.checkerframework-checker-qual-3.5.0.jar
+------------------------------------------------------------------------------------
+This product bundles the Reactive Streams library, which is licensed under
+Public Domain (CC0). For details, see deps/reactivestreams-1.0.3/LICENSE
+
+Bundles as
+  - lib/org.reactivestreams-reactive-streams-1.0.3.jar
\ No newline at end of file
diff --git a/bookkeeper-dist/src/main/resources/deps/reactivestreams-1.0.3/LICENSE b/bookkeeper-dist/src/main/resources/deps/reactivestreams-1.0.3/LICENSE
new file mode 100644
index 0000000..eadae05
--- /dev/null
+++ b/bookkeeper-dist/src/main/resources/deps/reactivestreams-1.0.3/LICENSE
@@ -0,0 +1,8 @@
+Licensed under Public Domain (CC0)
+
+To the extent possible under law, the person who associated CC0 with
+this code has waived all copyright and related or neighboring
+rights to this code.
+
+You should have received a copy of the CC0 legalcode along with this
+work. If not, see <http://creativecommons.org/publicdomain/zero/1.0/>.
\ No newline at end of file
diff --git a/bookkeeper-proto/src/main/proto/BookkeeperProtocol.proto b/bookkeeper-proto/src/main/proto/BookkeeperProtocol.proto
index 4f178aa..2bf72a4 100644
--- a/bookkeeper-proto/src/main/proto/BookkeeperProtocol.proto
+++ b/bookkeeper-proto/src/main/proto/BookkeeperProtocol.proto
@@ -46,6 +46,7 @@ enum StatusCode {
     EFENCED = 504;
     EREADONLY = 505;
     ETOOMANYREQUESTS = 506;
+    EUNKNOWNLEDGERSTATE = 507;
 }
 
 /**
diff --git a/bookkeeper-proto/src/main/proto/DbLedgerStorageDataFormats.proto b/bookkeeper-proto/src/main/proto/DbLedgerStorageDataFormats.proto
index ab75347..6df357d 100644
--- a/bookkeeper-proto/src/main/proto/DbLedgerStorageDataFormats.proto
+++ b/bookkeeper-proto/src/main/proto/DbLedgerStorageDataFormats.proto
@@ -28,4 +28,5 @@ message LedgerData {
     required bool fenced = 2;
     required bytes masterKey = 3;
     optional bytes explicitLac = 4;
+    optional bool limbo = 5;
 }
diff --git a/bookkeeper-server/build.gradle b/bookkeeper-server/build.gradle
index aa85d09..5b0cd26 100644
--- a/bookkeeper-server/build.gradle
+++ b/bookkeeper-server/build.gradle
@@ -52,6 +52,7 @@ dependencies {
     implementation depLibs.nettyTransportNativeEpoll
     implementation depLibs.protobuf
     implementation depLibs.rocksDb
+    implementation depLibs.rxjava
     implementation depLibs.zookeeper
     runtimeOnly depLibs.metricsCore
     runtimeOnly depLibs.snappy
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 cf705e3..90c8acf 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
@@ -48,7 +48,7 @@ public interface Bookie {
     void forceLedger(long ledgerId, WriteCallback cb, Object ctx);
     void setExplicitLac(ByteBuf entry, WriteCallback writeCallback, Object ctx, byte[] masterKey)
             throws IOException, InterruptedException, BookieException;
-    ByteBuf getExplicitLac(long ledgerId) throws IOException, NoLedgerException;
+    ByteBuf getExplicitLac(long ledgerId) throws IOException, NoLedgerException, BookieException;
 
     // these can probably be moved out and called directly on ledgerdirmanager
     long getTotalDiskSpace() throws IOException;
@@ -56,8 +56,8 @@ public interface Bookie {
 
     // TODO: Shouldn't this be async?
     ByteBuf readEntry(long ledgerId, long entryId)
-            throws IOException, NoLedgerException;
-    long readLastAddConfirmed(long ledgerId) throws IOException;
+            throws IOException, NoLedgerException, BookieException;
+    long readLastAddConfirmed(long ledgerId) throws IOException, BookieException;
     PrimitiveIterator.OfLong getListOfEntriesOfLedger(long ledgerId) throws IOException, NoLedgerException;
 
     /**
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieException.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieException.java
index dce6d0b..2f86052 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieException.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieException.java
@@ -69,6 +69,8 @@ public abstract class BookieException extends Exception {
             return new MetadataStoreException();
         case Code.UnknownBookieIdException:
             return new UnknownBookieIdException();
+        case Code.DataUnknownException:
+            return new DataUnknownException();
         default:
             return new BookieIllegalOpException();
         }
@@ -92,6 +94,7 @@ public abstract class BookieException extends Exception {
         int OperationRejectedException = -108;
         int CookieExistsException = -109;
         int EntryLogMetadataMapException = -110;
+        int DataUnknownException = -111;
     }
 
     public int getCode() {
@@ -137,6 +140,9 @@ public abstract class BookieException extends Exception {
         case Code.OperationRejectedException:
             err = "Operation rejected";
             break;
+        case Code.DataUnknownException:
+            err = "Unable to respond, ledger is in unknown state";
+            break;
         default:
             err = "Invalid operation";
             break;
@@ -337,4 +343,25 @@ public abstract class BookieException extends Exception {
         }
     }
 
+    /**
+     * Signal when a ledger is in a limbo state and certain operations
+     * cannot be performed on it.
+     */
+    public static class DataUnknownException extends BookieException {
+        public DataUnknownException() {
+            super(Code.DataUnknownException);
+        }
+
+        public DataUnknownException(Throwable t) {
+            super(Code.DataUnknownException, t);
+        }
+
+        public DataUnknownException(String reason) {
+            super(Code.DataUnknownException, reason);
+        }
+
+        public DataUnknownException(String reason, Throwable t) {
+            super(Code.DataUnknownException, reason, t);
+        }
+    }
 }
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieImpl.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieImpl.java
index ed57d7b..f0a002f 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieImpl.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/BookieImpl.java
@@ -169,7 +169,7 @@ public class BookieImpl extends BookieCriticalThread implements Bookie {
      * first run and the filesystem structure is up to date.
      */
     private void checkEnvironment(RegistrationManager registrationManager)
-            throws BookieException, IOException {
+            throws BookieException, IOException, InterruptedException {
         List<File> allLedgerDirs = new ArrayList<File>(ledgerDirsManager.getAllLedgerDirs().size()
                 + indexDirsManager.getAllLedgerDirs().size());
         allLedgerDirs.addAll(ledgerDirsManager.getAllLedgerDirs());
@@ -363,7 +363,7 @@ public class BookieImpl extends BookieCriticalThread implements Bookie {
         ledgerStorage.setCheckpointSource(new CheckpointSource() {
                 @Override
                 public Checkpoint newCheckpoint() {
-                    return Checkpoint.MAX;
+                    return Checkpoint.MIN;
                 }
 
                 @Override
@@ -488,6 +488,12 @@ public class BookieImpl extends BookieCriticalThread implements Bookie {
     }
 
     void readJournal() throws IOException, BookieException {
+        if (!conf.getJournalWriteData()) {
+            LOG.warn("Journal disabled for add entry requests. Running BookKeeper this way can "
+                    + "lead to data loss. It is recommended to use data integrity checking when "
+                    + "running without the journal to minimize data loss risk");
+        }
+
         long startTs = System.currentTimeMillis();
         JournalScanner scanner = new JournalScanner() {
             @Override
@@ -991,7 +997,7 @@ public class BookieImpl extends BookieCriticalThread implements Bookie {
         }
     }
 
-    public ByteBuf getExplicitLac(long ledgerId) throws IOException, Bookie.NoLedgerException {
+    public ByteBuf getExplicitLac(long ledgerId) throws IOException, Bookie.NoLedgerException, BookieException {
         ByteBuf lac;
         LedgerDescriptor handle = handles.getReadOnlyHandle(ledgerId);
         synchronized (handle) {
@@ -1066,7 +1072,7 @@ public class BookieImpl extends BookieCriticalThread implements Bookie {
     }
 
     public ByteBuf readEntry(long ledgerId, long entryId)
-            throws IOException, NoLedgerException {
+            throws IOException, NoLedgerException, BookieException {
         long requestNanos = MathUtils.nowInNano();
         boolean success = false;
         int entrySize = 0;
@@ -1091,7 +1097,7 @@ public class BookieImpl extends BookieCriticalThread implements Bookie {
         }
     }
 
-    public long readLastAddConfirmed(long ledgerId) throws IOException {
+    public long readLastAddConfirmed(long ledgerId) throws IOException, BookieException {
         LedgerDescriptor handle = handles.getReadOnlyHandle(ledgerId);
         return handle.getLastAddConfirmed();
     }
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/CheckpointSource.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/CheckpointSource.java
index 07ef316..2221632 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/CheckpointSource.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/CheckpointSource.java
@@ -94,7 +94,7 @@ public interface CheckpointSource {
     CheckpointSource DEFAULT = new CheckpointSource() {
             @Override
             public Checkpoint newCheckpoint() {
-                return Checkpoint.MAX;
+                return Checkpoint.MIN;
             }
 
             @Override
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Cookie.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Cookie.java
index e548cbd..d285a89 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Cookie.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/Cookie.java
@@ -30,6 +30,7 @@ import java.io.BufferedWriter;
 import java.io.EOFException;
 import java.io.File;
 import java.io.FileInputStream;
+import java.io.FileNotFoundException;
 import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.InputStreamReader;
@@ -37,6 +38,7 @@ import java.io.OutputStreamWriter;
 import java.io.StringReader;
 import java.net.InetSocketAddress;
 import java.net.UnknownHostException;
+import java.util.Objects;
 import java.util.Set;
 import org.apache.bookkeeper.bookie.BookieException.InvalidCookieException;
 import org.apache.bookkeeper.bookie.BookieException.UnknownBookieIdException;
@@ -211,6 +213,12 @@ public class Cookie {
         return cBuilder;
     }
 
+    public static Cookie parseFromBytes(byte[] bytes) throws IOException {
+        try (BufferedReader reader = new BufferedReader(new StringReader(new String(bytes, UTF_8)))) {
+            return parse(reader).build();
+        }
+    }
+
     public void writeToDirectory(File directory) throws IOException {
         File versionFile = new File(directory,
             BookKeeperConstants.VERSION_FILENAME);
@@ -289,7 +297,7 @@ public class Cookie {
      * @return cookie builder object
      * @throws UnknownHostException
      */
-    static Builder generateCookie(ServerConfiguration conf)
+    public static Builder generateCookie(ServerConfiguration conf)
             throws UnknownHostException {
         Builder builder = Cookie.newBuilder();
         builder.setLayoutVersion(CURRENT_COOKIE_LAYOUT_VERSION);
@@ -346,7 +354,7 @@ public class Cookie {
      * @return cookie object
      * @throws IOException
      */
-    public static Cookie readFromDirectory(File directory) throws IOException {
+    public static Cookie readFromDirectory(File directory) throws IOException, FileNotFoundException {
         File versionFile = new File(directory, BookKeeperConstants.VERSION_FILENAME);
         try (BufferedReader reader = new BufferedReader(
                 new InputStreamReader(new FileInputStream(versionFile), UTF_8))) {
@@ -452,4 +460,23 @@ public class Cookie {
         return new Builder(oldCookie.layoutVersion, oldCookie.bookieId, oldCookie.journalDirs, oldCookie.ledgerDirs,
                 oldCookie.instanceId);
     }
+
+    @Override
+    public boolean equals(Object other) {
+        if (other instanceof Cookie) {
+            Cookie otherCookie = (Cookie) other;
+            return layoutVersion == otherCookie.layoutVersion
+                && Objects.equals(bookieId, otherCookie.bookieId)
+                && Objects.equals(journalDirs, otherCookie.journalDirs)
+                && Objects.equals(ledgerDirs, otherCookie.ledgerDirs)
+                && Objects.equals(instanceId, otherCookie.instanceId);
+        } else {
+            return false;
+        }
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(bookieId, journalDirs, ledgerDirs, instanceId);
+    }
 }
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/CookieValidation.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/CookieValidation.java
index 9b88ff8..06a7026 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/CookieValidation.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/CookieValidation.java
@@ -25,5 +25,5 @@ import java.util.List;
  * Interface for cookie validation.
  */
 public interface CookieValidation {
-    void checkCookies(List<File> directories) throws BookieException, UnknownHostException;
+    void checkCookies(List<File> directories) throws BookieException, UnknownHostException, InterruptedException;
 }
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InterleavedLedgerStorage.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InterleavedLedgerStorage.java
index 4cc34c7..dcfeed0 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InterleavedLedgerStorage.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/InterleavedLedgerStorage.java
@@ -42,6 +42,7 @@ import java.io.File;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
+import java.util.EnumSet;
 import java.util.List;
 import java.util.Map;
 import java.util.NavigableMap;
@@ -328,6 +329,13 @@ public class InterleavedLedgerStorage implements CompactableLedgerStorage, Entry
     }
 
     @Override
+    public boolean entryExists(long ledgerId, long entryId) throws IOException {
+        //Implementation should be as simple as what's below, but this needs testing
+        //return ledgerCache.getEntryOffset(ledgerId, entryId) > 0;
+        throw new UnsupportedOperationException("entry exists not supported");
+    }
+
+    @Override
     public long getLastAddConfirmed(long ledgerId) throws IOException {
         Long lac = ledgerCache.getLastAddConfirmed(ledgerId);
         if (lac == null) {
@@ -657,4 +665,39 @@ public class InterleavedLedgerStorage implements CompactableLedgerStorage, Entry
     public OfLong getListOfEntriesOfLedger(long ledgerId) throws IOException {
         return ledgerCache.getEntriesIterator(ledgerId);
     }
+
+    @Override
+    public void setLimboState(long ledgerId) throws IOException {
+        throw new UnsupportedOperationException(
+                "Limbo state only supported for DbLedgerStorage");
+    }
+
+    @Override
+    public boolean hasLimboState(long ledgerId) throws IOException {
+        throw new UnsupportedOperationException(
+                "Limbo state only supported for DbLedgerStorage");
+    }
+
+    @Override
+    public void clearLimboState(long ledgerId) throws IOException {
+        throw new UnsupportedOperationException(
+                "Limbo state only supported for DbLedgerStorage");
+    }
+
+    @Override
+    public EnumSet<StorageState> getStorageStateFlags() throws IOException {
+        return EnumSet.noneOf(StorageState.class);
+    }
+
+    @Override
+    public void setStorageStateFlag(StorageState flags) throws IOException {
+        throw new UnsupportedOperationException(
+                "Storage state only flags supported for DbLedgerStorage");
+    }
+
+    @Override
+    public void clearStorageStateFlag(StorageState flags) throws IOException {
+        throw new UnsupportedOperationException(
+                "Storage state flags only supported for DbLedgerStorage");
+    }
 }
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDescriptor.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDescriptor.java
index a862b7b..f491024 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDescriptor.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDescriptor.java
@@ -64,7 +64,7 @@ public abstract class LedgerDescriptor {
     abstract long getLedgerId();
 
     abstract boolean setFenced() throws IOException;
-    abstract boolean isFenced() throws IOException;
+    abstract boolean isFenced() throws IOException, BookieException;
     /**
      * When we fence a ledger, we need to first set ledger to fenced state in memory and
      * then log the fence entry in Journal so that we can rebuild the state.
@@ -75,9 +75,9 @@ public abstract class LedgerDescriptor {
     abstract CompletableFuture<Boolean> fenceAndLogInJournal(Journal journal) throws IOException;
 
     abstract long addEntry(ByteBuf entry) throws IOException, BookieException;
-    abstract ByteBuf readEntry(long entryId) throws IOException;
+    abstract ByteBuf readEntry(long entryId) throws IOException, BookieException;
 
-    abstract long getLastAddConfirmed() throws IOException;
+    abstract long getLastAddConfirmed() throws IOException, BookieException;
     abstract boolean waitForLastAddConfirmedUpdate(long previousLAC,
                                                    Watcher<LastAddConfirmedUpdateNotification> watcher)
         throws IOException;
@@ -86,7 +86,7 @@ public abstract class LedgerDescriptor {
 
     abstract void setExplicitLac(ByteBuf entry) throws IOException;
 
-    abstract  ByteBuf getExplicitLac() throws IOException;
+    abstract  ByteBuf getExplicitLac() throws IOException, BookieException;
 
     abstract OfLong getListOfEntriesOfLedger(long ledgerId) throws IOException;
 }
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDescriptorImpl.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDescriptorImpl.java
index 2496d69..5b8141f 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDescriptorImpl.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerDescriptorImpl.java
@@ -74,7 +74,7 @@ public class LedgerDescriptorImpl extends LedgerDescriptor {
     }
 
     @Override
-    boolean isFenced() throws IOException {
+    boolean isFenced() throws IOException, BookieException {
         return ledgerStorage.isFenced(ledgerId);
     }
 
@@ -84,7 +84,7 @@ public class LedgerDescriptorImpl extends LedgerDescriptor {
     }
 
     @Override
-    ByteBuf getExplicitLac() throws IOException {
+    ByteBuf getExplicitLac() throws IOException, BookieException {
         return ledgerStorage.getExplicitLac(ledgerId);
     }
 
@@ -157,12 +157,12 @@ public class LedgerDescriptorImpl extends LedgerDescriptor {
     }
 
     @Override
-    ByteBuf readEntry(long entryId) throws IOException {
+    ByteBuf readEntry(long entryId) throws IOException, BookieException {
         return ledgerStorage.getEntry(ledgerId, entryId);
     }
 
     @Override
-    long getLastAddConfirmed() throws IOException {
+    long getLastAddConfirmed() throws IOException, BookieException {
         return ledgerStorage.getLastAddConfirmed(ledgerId);
     }
 
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerStorage.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerStorage.java
index a671832..eab61af 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerStorage.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/LedgerStorage.java
@@ -28,6 +28,7 @@ import io.netty.buffer.ByteBufAllocator;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
+import java.util.EnumSet;
 import java.util.List;
 import java.util.Optional;
 import java.util.PrimitiveIterator;
@@ -78,6 +79,11 @@ public interface LedgerStorage {
     boolean ledgerExists(long ledgerId) throws IOException;
 
     /**
+     * Whether an entry exists.
+     */
+    boolean entryExists(long ledgerId, long entryId) throws IOException, BookieException;
+
+    /**
      * Fenced the ledger id in ledger storage.
      *
      * @param ledgerId Ledger Id.
@@ -91,7 +97,28 @@ public interface LedgerStorage {
      * @param ledgerId Ledger ID.
      * @throws IOException
      */
-    boolean isFenced(long ledgerId) throws IOException;
+    boolean isFenced(long ledgerId) throws IOException, BookieException;
+
+    /**
+     * Set a ledger to limbo state.
+     * When a ledger is in limbo state, we cannot answer any requests about it.
+     * For example, if a client asks for an entry, we cannot say we don't have it because
+     * it may have been written to us in the past, but we are waiting for data integrity checks
+     * to copy it over.
+     */
+    void setLimboState(long ledgerId) throws IOException;
+
+    /**
+     * Check whether a ledger is in limbo state.
+     * @see #setLimboState(long)
+     */
+    boolean hasLimboState(long ledgerId) throws IOException;
+
+    /**
+     * Clear the limbo state of a ledger.
+     * @see #setLimboState(long)
+     */
+    void clearLimboState(long ledgerId) throws IOException;
 
     /**
      * Set the master key for a ledger.
@@ -116,7 +143,7 @@ public interface LedgerStorage {
     /**
      * Read an entry from storage.
      */
-    ByteBuf getEntry(long ledgerId, long entryId) throws IOException;
+    ByteBuf getEntry(long ledgerId, long entryId) throws IOException, BookieException;
 
     /**
      * Get last add confirmed.
@@ -125,7 +152,7 @@ public interface LedgerStorage {
      * @return last add confirmed.
      * @throws IOException
      */
-    long getLastAddConfirmed(long ledgerId) throws IOException;
+    long getLastAddConfirmed(long ledgerId) throws IOException, BookieException;
 
     /**
      * Wait for last add confirmed update.
@@ -189,7 +216,7 @@ public interface LedgerStorage {
 
     void setExplicitLac(long ledgerId, ByteBuf lac) throws IOException;
 
-    ByteBuf getExplicitLac(long ledgerId) throws IOException;
+    ByteBuf getExplicitLac(long ledgerId) throws IOException, BookieException;
 
     // for testability
     default LedgerStorage getUnderlyingLedgerStorage() {
@@ -276,4 +303,30 @@ public interface LedgerStorage {
      * @throws Exception
      */
     PrimitiveIterator.OfLong getListOfEntriesOfLedger(long ledgerId) throws IOException;
+
+    /**
+     * Get the storage state flags currently set for the storage instance.
+     */
+    EnumSet<StorageState> getStorageStateFlags() throws IOException;
+
+    /**
+     * Set a storage state flag for the storage instance.
+     * Implementations must ensure this method is atomic, and the flag
+     * is persisted to storage when the method returns.
+     */
+    void setStorageStateFlag(StorageState flags) throws IOException;
+
+    /**
+     * Clear a storage state flag for the storage instance.
+     * Implementations must ensure this method is atomic, and the flag
+     * is persisted to storage when the method returns.
+     */
+    void clearStorageStateFlag(StorageState flags) throws IOException;
+
+    /**
+     * StorageState flags.
+     */
+    enum StorageState {
+        NEEDS_INTEGRITY_CHECK
+    }
 }
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/SortedLedgerStorage.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/SortedLedgerStorage.java
index b8a46c1..305cdc0 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/SortedLedgerStorage.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/SortedLedgerStorage.java
@@ -28,6 +28,7 @@ import io.netty.buffer.ByteBuf;
 import io.netty.buffer.ByteBufAllocator;
 
 import java.io.IOException;
+import java.util.EnumSet;
 import java.util.List;
 import java.util.Optional;
 import java.util.PrimitiveIterator;
@@ -164,6 +165,12 @@ public class SortedLedgerStorage
     }
 
     @Override
+    public boolean entryExists(long ledgerId, long entryId) throws IOException {
+        // can probably be implemented as above, but I'm not going to test it
+        throw new UnsupportedOperationException("Not supported for SortedLedgerStorage");
+    }
+
+    @Override
     public boolean setFenced(long ledgerId) throws IOException {
         return interleavedLedgerStorage.setFenced(ledgerId);
     }
@@ -209,7 +216,7 @@ public class SortedLedgerStorage
     }
 
     @Override
-    public ByteBuf getEntry(long ledgerId, long entryId) throws IOException {
+    public ByteBuf getEntry(long ledgerId, long entryId) throws IOException, BookieException {
         if (entryId == BookieProtocol.LAST_ADD_CONFIRMED) {
             return getLastEntryId(ledgerId);
         }
@@ -384,4 +391,39 @@ public class SortedLedgerStorage
         PrimitiveIterator.OfLong entriesFromILSItr = interleavedLedgerStorage.getListOfEntriesOfLedger(ledgerId);
         return IteratorUtility.mergePrimitiveLongIterator(entriesInMemtableItr, entriesFromILSItr);
     }
+
+    @Override
+    public void setLimboState(long ledgerId) throws IOException {
+        throw new UnsupportedOperationException(
+                "Limbo state only supported for DbLedgerStorage");
+    }
+
+    @Override
+    public boolean hasLimboState(long ledgerId) throws IOException {
+        throw new UnsupportedOperationException(
+                "Limbo state only supported for DbLedgerStorage");
+    }
+
+    @Override
+    public void clearLimboState(long ledgerId) throws IOException {
+        throw new UnsupportedOperationException(
+                "Limbo state only supported for DbLedgerStorage");
+    }
+
+    @Override
+    public EnumSet<StorageState> getStorageStateFlags() throws IOException {
+        return EnumSet.noneOf(StorageState.class);
+    }
+
+    @Override
+    public void setStorageStateFlag(StorageState flags) throws IOException {
+        throw new UnsupportedOperationException(
+                "Storage state only flags supported for DbLedgerStorage");
+    }
+
+    @Override
+    public void clearStorageStateFlag(StorageState flags) throws IOException {
+        throw new UnsupportedOperationException(
+                "Storage state flags only supported for DbLedgerStorage");
+    }
 }
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/UncleanShutdownDetection.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/UncleanShutdownDetection.java
new file mode 100644
index 0000000..4826f80
--- /dev/null
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/UncleanShutdownDetection.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.bookkeeper.bookie;
+
+import java.io.IOException;
+
+/**
+ * An interface for unclean shutdown detection. The bookie
+ * must register its start-up and then register its graceful
+ * shutdown. Abrupt termination will not register the clean
+ * shutdown.
+ */
+public interface UncleanShutdownDetection {
+    void registerStartUp() throws IOException;
+    void registerCleanShutdown();
+    boolean lastShutdownWasUnclean();
+}
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/UncleanShutdownDetectionImpl.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/UncleanShutdownDetectionImpl.java
new file mode 100644
index 0000000..879a288
--- /dev/null
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/UncleanShutdownDetectionImpl.java
@@ -0,0 +1,112 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.bookkeeper.bookie;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Used to determine if the prior shutdown was unclean or not. It does so
+ * by adding a file to each ledger directory after successful start-up
+ * and removing the file on graceful shutdown.
+ * Any abrupt termination will cause one or more of these files to not be cleared
+ * and so on the subsequent boot-up, the presence of any of these files will
+ * indicate an unclean shutdown.
+ */
+public class UncleanShutdownDetectionImpl implements UncleanShutdownDetection {
+    private static final Logger LOG = LoggerFactory.getLogger(UncleanShutdownDetectionImpl.class);
+    private final LedgerDirsManager ledgerDirsManager;
+    static final String DIRTY_FILENAME = "DIRTY";
+
+    public UncleanShutdownDetectionImpl(LedgerDirsManager ledgerDirsManager) {
+        this.ledgerDirsManager = ledgerDirsManager;
+    }
+
+    @Override
+    public void registerStartUp() throws IOException {
+        for (File ledgerDir : ledgerDirsManager.getAllLedgerDirs()) {
+            try {
+                File dirtyFile = new File(ledgerDir, DIRTY_FILENAME);
+                dirtyFile.createNewFile();
+                LOG.info("Created dirty file in ledger dir: {}", ledgerDir.getAbsolutePath());
+            } catch (IOException e) {
+                LOG.error("Unable to register start-up (so an unclean shutdown cannot"
+                        + " be detected). Dirty file of ledger dir {} could not be created.",
+                        ledgerDir.getAbsolutePath(), e);
+                throw e;
+            }
+        }
+    }
+
+    @Override
+    public void registerCleanShutdown() {
+        for (File ledgerDir : ledgerDirsManager.getAllLedgerDirs()) {
+            try {
+                File dirtyFile = new File(ledgerDir, DIRTY_FILENAME);
+                if (dirtyFile.exists()) {
+                    boolean deleted = dirtyFile.delete();
+
+                    if (!deleted) {
+                        LOG.error("Unable to register a clean shutdown. The dirty file of "
+                                        + " ledger dir {} could not be deleted.",
+                                ledgerDir.getAbsolutePath());
+                    }
+                } else {
+                    LOG.error("Unable to register a clean shutdown. The dirty file of "
+                                    + " ledger dir {} does not exist.",
+                            ledgerDir.getAbsolutePath());
+                }
+            } catch (Throwable t) {
+                LOG.error("Unable to register a clean shutdown. An error occurred while deleting "
+                        + " the dirty file of ledger dir {}.",
+                        ledgerDir.getAbsolutePath(), t);
+            }
+        }
+    }
+
+    @Override
+    public boolean lastShutdownWasUnclean() {
+        boolean unclean = false;
+        List<String> dirtyFiles = new ArrayList<>();
+        try {
+            for (File ledgerDir : ledgerDirsManager.getAllLedgerDirs()) {
+                File dirtyFile = new File(ledgerDir, DIRTY_FILENAME);
+                if (dirtyFile.exists()) {
+                    dirtyFiles.add(dirtyFile.getAbsolutePath());
+                    unclean = true;
+                }
+            }
+        } catch (Throwable t) {
+            LOG.error("Unable to determine if last shutdown was unclean (defaults to unclean)", t);
+            unclean = true;
+        }
+
+        if (!dirtyFiles.isEmpty()) {
+            LOG.info("Dirty files exist on boot-up indicating an unclean shutdown. Dirty files: {}",
+                    String.join(",", dirtyFiles));
+        }
+
+        return unclean;
+    }
+}
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/datainteg/DataIntegrityCheck.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/datainteg/DataIntegrityCheck.java
new file mode 100644
index 0000000..6d7af66
--- /dev/null
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/datainteg/DataIntegrityCheck.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.bookkeeper.bookie.datainteg;
+
+import java.io.IOException;
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * The interface for the data integrity check feature. This feature allows
+ * a bookie to handle data loss scenarios such as when running without
+ * the journal or after a disk failure has caused the loss of all data.
+ */
+public interface DataIntegrityCheck {
+    /**
+     * Run quick preboot check. This check should do enough to ensure that
+     * it is safe to complete the boot sequence without compromising correctness.
+     * To this end, if it finds that this bookie is part of the last ensemble of
+     * an unclosed ledger, it must prevent the bookie from being able store new
+     * entries for that ledger and must prevent the bookie from taking part in
+     * the discovery of the last entry of that ledger.
+     */
+    CompletableFuture<Void> runPreBootCheck(String reason);
+
+    /**
+     * Whether we need to run a full check.
+     * This condition can be set by the runPreBoot() call to run a full check
+     * in the background once the bookie is running. This can later be used
+     * to run the full check periodically, or to exponentially backoff and retry
+     * when some transient condition prevents a ledger being fixed during a
+     * full check.
+     */
+    boolean needsFullCheck() throws IOException;
+
+    /**
+     * Run full check of bookies local data. This check should ensure that
+     * if the metadata service states that it should have an entry, then it
+     * should have that entry. If the entry is missing, it should copy it
+     * from another available source.
+     */
+    CompletableFuture<Void> runFullCheck();
+}
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/datainteg/DataIntegrityCheckImpl.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/datainteg/DataIntegrityCheckImpl.java
new file mode 100644
index 0000000..319ceac
--- /dev/null
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/datainteg/DataIntegrityCheckImpl.java
@@ -0,0 +1,556 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.bookkeeper.bookie.datainteg;
+
+import com.google.common.collect.ImmutableSortedMap;
+import io.reactivex.rxjava3.core.Flowable;
+import io.reactivex.rxjava3.core.Maybe;
+import io.reactivex.rxjava3.core.Scheduler;
+import io.reactivex.rxjava3.core.Single;
+import java.io.IOException;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.Optional;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.stream.Collectors;
+
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.bookie.BookieException;
+import org.apache.bookkeeper.bookie.LedgerStorage;
+import org.apache.bookkeeper.bookie.LedgerStorage.StorageState;
+import org.apache.bookkeeper.client.BKException;
+import org.apache.bookkeeper.client.BookKeeperAdmin;
+import org.apache.bookkeeper.client.api.LedgerMetadata;
+import org.apache.bookkeeper.common.concurrent.FutureUtils;
+import org.apache.bookkeeper.meta.LedgerManager;
+import org.apache.bookkeeper.net.BookieId;
+
+/**
+ * An implementation of the DataIntegrityCheck interface.
+ */
+@Slf4j
+public class DataIntegrityCheckImpl implements DataIntegrityCheck {
+    private static final int MAX_INFLIGHT = 300;
+    private static final int MAX_ENTRIES_INFLIGHT = 3000;
+    private static final int ZK_TIMEOUT_S = 30;
+    private final BookieId bookieId;
+    private final LedgerManager ledgerManager;
+    private final LedgerStorage ledgerStorage;
+    private final EntryCopier entryCopier;
+    private final BookKeeperAdmin admin;
+    private final Scheduler scheduler;
+    private final AtomicReference<Map<Long, LedgerMetadata>> ledgersCacheRef =
+        new AtomicReference<>(null);
+    private CompletableFuture<Void> preBootFuture;
+
+    public DataIntegrityCheckImpl(BookieId bookieId,
+                                  LedgerManager ledgerManager,
+                                  LedgerStorage ledgerStorage,
+                                  EntryCopier entryCopier,
+                                  BookKeeperAdmin admin,
+                                  Scheduler scheduler) {
+        this.bookieId = bookieId;
+        this.ledgerManager = ledgerManager;
+        this.ledgerStorage = ledgerStorage;
+        this.entryCopier = entryCopier;
+        this.admin = admin;
+        this.scheduler = scheduler;
+    }
+
+    @Override
+    public CompletableFuture<Void> runPreBootCheck(String reason) {
+        // we only run this once, it could be kicked off by different checks
+        synchronized (this) {
+            if (preBootFuture == null) {
+                preBootFuture = runPreBootSequence(reason);
+            }
+        }
+        return preBootFuture;
+
+    }
+
+    private CompletableFuture<Void> runPreBootSequence(String reason) {
+        String runId = UUID.randomUUID().toString();
+        log.info("Event: {}, RunId: {}, Reason: {}", Events.PREBOOT_START, runId, reason);
+        try {
+            this.ledgerStorage.setStorageStateFlag(StorageState.NEEDS_INTEGRITY_CHECK);
+        } catch (IOException ioe) {
+            log.error("Event: {}, RunId: {}", Events.PREBOOT_ERROR, runId, ioe);
+            return FutureUtils.exception(ioe);
+        }
+
+        MetadataAsyncIterator iter = new MetadataAsyncIterator(scheduler,
+                ledgerManager, MAX_INFLIGHT, ZK_TIMEOUT_S, TimeUnit.SECONDS);
+        CompletableFuture<Void> promise = new CompletableFuture<>();
+        Map<Long, LedgerMetadata> ledgersCache =
+            new ConcurrentSkipListMap<>(Comparator.<Long>naturalOrder().reversed());
+        iter.forEach((ledgerId, metadata) -> {
+                if (ensemblesContainBookie(metadata, bookieId)) {
+                    ledgersCache.put(ledgerId, metadata);
+                    try {
+                        if (!ledgerStorage.ledgerExists(ledgerId)) {
+                            ledgerStorage.setMasterKey(ledgerId, new byte[0]);
+                        }
+                    } catch (IOException ioe) {
+                        log.error("Event: {}, RunId: {}, LedgerId: {}",
+                                Events.ENSURE_LEDGER_ERROR, runId, ledgerId, ioe);
+                        return FutureUtils.exception(ioe);
+                    }
+                }
+                return processPreBoot(ledgerId, metadata, runId);
+            })
+            .whenComplete((ignore, exception) -> {
+                    if (exception != null) {
+                        log.error("Event: {}, runId: {}", Events.PREBOOT_ERROR, runId, exception);
+                        promise.completeExceptionally(exception);
+                    } else {
+                        try {
+                            this.ledgerStorage.flush();
+
+                            updateMetadataCache(ledgersCache);
+
+                            log.info("Event: {}, runId: {}, processed: {}",
+                                    Events.PREBOOT_END, runId, ledgersCache.size());
+                            promise.complete(null);
+                        } catch (Throwable t) {
+                            log.error("Event: {}, runId: {}", Events.PREBOOT_ERROR, runId, t);
+                            promise.completeExceptionally(t);
+                        }
+                    }
+                });
+        return promise;
+    }
+
+    @Override
+    public boolean needsFullCheck() throws IOException {
+        return this.ledgerStorage.getStorageStateFlags()
+            .contains(StorageState.NEEDS_INTEGRITY_CHECK);
+    }
+
+    @Override
+    public CompletableFuture<Void> runFullCheck() {
+        String runId = UUID.randomUUID().toString();
+
+        log.info("Event: {}, runId: {}", Events.FULL_CHECK_INIT, runId);
+        return getCachedOrReadMetadata(runId)
+            .thenCompose(
+                    (ledgers) -> {
+                        log.info("Event: {}, runId: {}, ledgerCount: {}",
+                                Events.FULL_CHECK_START, runId, ledgers.size());
+                        return checkAndRecoverLedgers(ledgers, runId).thenApply((resolved) -> {
+                                for (LedgerResult r : resolved) {
+                                    if (r.isMissing() || r.isOK()) {
+                                        ledgers.remove(r.getLedgerId());
+                                    } else if (r.isError()) {
+                                        // if there was an error, make sure we have the latest
+                                        // metadata for the next iteration
+                                        ledgers.put(r.getLedgerId(), r.getMetadata());
+                                    }
+                                }
+                                Optional<Throwable> firstError = resolved.stream().filter(r -> r.isError())
+                                    .map(r -> r.getThrowable()).findFirst();
+
+                                if (firstError.isPresent()) {
+                                    log.error("Event: {}, runId: {}, ok: {}"
+                                            + ", error: {}, missing: {}, ledgersToRetry: {}",
+                                            Events.FULL_CHECK_END, runId,
+                                            resolved.stream().filter(r -> r.isOK()).count(),
+                                            resolved.stream().filter(r -> r.isError()).count(),
+                                            resolved.stream().filter(r -> r.isMissing()).count(),
+                                            ledgers.size(), firstError.get());
+                                } else {
+                                    log.info("Event: {}, runId: {}, ok: {}, error: 0, missing: {}, ledgersToRetry: {}",
+                                            Events.FULL_CHECK_END, runId,
+                                            resolved.stream().filter(r -> r.isOK()).count(),
+                                            resolved.stream().filter(r -> r.isMissing()).count(),
+                                            ledgers.size());
+                                }
+                                return ledgers;
+                            });
+                    })
+            .thenCompose(
+                    (ledgers) -> {
+                        CompletableFuture<Void> promise = new CompletableFuture<>();
+                        try {
+                            this.ledgerStorage.flush();
+                            if (ledgers.isEmpty()) {
+                                log.info("Event: {}, runId: {}", Events.CLEAR_INTEGCHECK_FLAG, runId);
+                                this.ledgerStorage.clearStorageStateFlag(
+                                        StorageState.NEEDS_INTEGRITY_CHECK);
+                            }
+                            // not really needed as we are modifying the map in place
+                            updateMetadataCache(ledgers);
+                            log.info("Event: {}, runId: {}", Events.FULL_CHECK_COMPLETE, runId);
+                            promise.complete(null);
+                        } catch (IOException ioe) {
+                            log.error("Event: {}, runId: {}", Events.FULL_CHECK_ERROR, runId, ioe);
+                            promise.completeExceptionally(ioe);
+                        }
+                        return promise;
+                    });
+    }
+
+    void updateMetadataCache(Map<Long, LedgerMetadata> ledgers) {
+        ledgersCacheRef.set(ledgers);
+    }
+
+    CompletableFuture<Map<Long, LedgerMetadata>> getCachedOrReadMetadata(String runId) {
+        Map<Long, LedgerMetadata> map = ledgersCacheRef.get();
+        if (map != null) {
+            log.info("Event: {}, runId: {}, ledgerCount: {}", Events.USE_CACHED_METADATA, runId,
+                    map.size());
+            return CompletableFuture.completedFuture(map);
+        } else {
+            log.info("Event: {}, runId: {}", Events.REFRESH_METADATA, runId);
+            MetadataAsyncIterator iter = new MetadataAsyncIterator(scheduler,
+                    ledgerManager, MAX_INFLIGHT, ZK_TIMEOUT_S, TimeUnit.SECONDS);
+            Map<Long, LedgerMetadata> ledgersCache =
+                new ConcurrentSkipListMap<>(Comparator.<Long>naturalOrder().reversed());
+            return iter.forEach((ledgerId, metadata) -> {
+                    if (ensemblesContainBookie(metadata, bookieId)) {
+                        ledgersCache.put(ledgerId, metadata);
+                    }
+                    return CompletableFuture.completedFuture(null);
+                })
+                .thenApply(ignore -> {
+                        updateMetadataCache(ledgersCache);
+                        return ledgersCache;
+                    });
+        }
+    }
+
+    /**
+     * Check whether the current bookie exists in the last ensemble of the bookie.
+     * If it does, and the ledger is not closed, then this bookie may have accepted a fencing
+     * request or an entry which it no longer contains. The only way to resolve this is to
+     * open/recover the ledger. This bookie should not take part in the recovery, so the bookie
+     * must be marked as in limbo. This will stop the bookie from responding to read requests for
+     * that ledger, so clients will not be able to take into account the response of the bookie
+     * during recovery. Effectively we are telling the client that we don't know whether we had
+     * certain entries or not, so go look elsewhere.
+     * We also fence all ledgers with this bookie in the last segment, to prevent any new writes,
+     * so that after the limbo state is cleared, we won't accept any new writes.
+
+     * We only need to consider final ensembles in non-closed ledgers at the moment of time that
+     * the preboot check commences. If this bookie is added to a new ensemble after that point in
+     * time, we know that we haven't received any entries for that segment, nor have we received
+     * a fencing request, because we are still in the preboot sequence.
+     */
+    private CompletableFuture<Void> processPreBoot(long ledgerId, LedgerMetadata metadata,
+                                                   String runId) {
+        Map.Entry<Long, ? extends List<BookieId>> lastEnsemble = metadata.getAllEnsembles().lastEntry();
+        CompletableFuture<Void> promise = new CompletableFuture<>();
+        if (lastEnsemble == null) {
+            log.error("Event: {}, runId: {}, metadata: {}, ledger: {}",
+                    Events.INVALID_METADATA, runId, metadata, ledgerId);
+            promise.completeExceptionally(
+                    new IllegalStateException(
+                            String.format("All metadata must have at least one ensemble, %d does not", ledgerId)));
+            return promise;
+        }
+
+
+        if (!metadata.isClosed() && lastEnsemble.getValue().contains(bookieId)) {
+            try {
+                log.info("Event: {}, runId: {}, metadata: {}, ledger: {}",
+                        Events.MARK_LIMBO, runId, metadata, ledgerId);
+                ledgerStorage.setLimboState(ledgerId);
+                ledgerStorage.setFenced(ledgerId);
+                promise.complete(null);
+            } catch (IOException ioe) {
+                log.info("Event: {}, runId: {}, metadata: {}, ledger: {}",
+                        Events.LIMBO_OR_FENCE_ERROR, runId, metadata, ledgerId, ioe);
+                promise.completeExceptionally(ioe);
+            }
+        } else {
+            promise.complete(null);
+        }
+        return promise;
+    }
+
+    static class LedgerResult {
+        enum State {
+            MISSING, ERROR, OK
+        };
+
+        static LedgerResult missing(long ledgerId) {
+            return new LedgerResult(State.MISSING, ledgerId, null, null);
+        }
+
+        static LedgerResult ok(long ledgerId, LedgerMetadata metadata) {
+            return new LedgerResult(State.OK, ledgerId, metadata, null);
+        }
+
+        static LedgerResult error(long ledgerId, LedgerMetadata metadata, Throwable t) {
+            return new LedgerResult(State.ERROR, ledgerId, metadata, t);
+        }
+
+        private final State state;
+        private final long ledgerId;
+        private final LedgerMetadata metadata;
+        private final Throwable throwable;
+
+        private LedgerResult(State state, long ledgerId,
+                             LedgerMetadata metadata, Throwable throwable) {
+            this.state = state;
+            this.ledgerId = ledgerId;
+            this.metadata = metadata;
+            this.throwable = throwable;
+        }
+
+        boolean isMissing() {
+            return state == State.MISSING;
+        }
+        boolean isOK() {
+            return state == State.OK;
+        }
+        boolean isError() {
+            return state == State.ERROR;
+        }
+        long getLedgerId() {
+            return ledgerId;
+        }
+        LedgerMetadata getMetadata() {
+            return metadata;
+        }
+        Throwable getThrowable() {
+            return throwable;
+        }
+    }
+
+    /**
+     * Check each ledger passed.
+     * If the ledger is in limbo, recover it.
+     * Check that the bookie has all entries that it is expected to have.
+     * Copy any entries that are missing.
+     * @return The set of results for all ledgers passed. A result can be OK, Missing or Error.
+     *         OK and missing ledgers do not need to be looked at again. Error should be retried.
+     */
+    CompletableFuture<Set<LedgerResult>> checkAndRecoverLedgers(Map<Long, LedgerMetadata> ledgers,
+                                                                String runId) {
+        CompletableFuture<Set<LedgerResult>> promise = new CompletableFuture<>();
+        Flowable.fromIterable(ledgers.entrySet())
+            .subscribeOn(scheduler, false)
+            .flatMapSingle((mapEntry) -> {
+                    long ledgerId = mapEntry.getKey();
+                    LedgerMetadata originalMetadata = mapEntry.getValue();
+                    return recoverLedgerIfInLimbo(ledgerId, mapEntry.getValue(), runId)
+                        .map(newMetadata -> LedgerResult.ok(ledgerId, newMetadata))
+                        .onErrorReturn(t -> LedgerResult.error(ledgerId, originalMetadata, t))
+                        .defaultIfEmpty(LedgerResult.missing(ledgerId))
+                        .flatMap((res) -> {
+                                try {
+                                    if (res.isOK()) {
+                                        this.ledgerStorage.clearLimboState(ledgerId);
+                                    }
+                                    return Single.just(res);
+                                } catch (IOException ioe) {
+                                    return Single.just(LedgerResult.error(res.getLedgerId(),
+                                                                          res.getMetadata(), ioe));
+                                }
+                            });
+                },
+                true /* delayErrors */,
+                MAX_INFLIGHT)
+            .flatMapSingle((res) -> {
+                    if (res.isOK()) {
+                        return checkAndRecoverLedgerEntries(res.getLedgerId(),
+                                                            res.getMetadata(), runId)
+                            .map(ignore -> LedgerResult.ok(res.getLedgerId(),
+                                                           res.getMetadata()))
+                            .onErrorReturn(t -> LedgerResult.error(res.getLedgerId(),
+                                                                   res.getMetadata(), t));
+                    } else {
+                        return Single.just(res);
+                    }
+                },
+                true /* delayErrors */,
+                1 /* copy 1 ledger at a time to keep entries together in entrylog */)
+            .collect(Collectors.toSet())
+            .subscribe(resolved -> promise.complete(resolved),
+                       throwable -> promise.completeExceptionally(throwable));
+        return promise;
+    }
+
+    /**
+     * Run ledger recovery on all a ledger if it has been marked as in limbo.
+     * @return a maybe with the most up to date metadata we have for he ledger.
+     *         If the ledger has been deleted, returns empty.
+     */
+    Maybe<LedgerMetadata> recoverLedgerIfInLimbo(long ledgerId, LedgerMetadata origMetadata,
+                                                 String runId) {
+        try {
+            if (!this.ledgerStorage.ledgerExists(ledgerId)) {
+                this.ledgerStorage.setMasterKey(ledgerId, new byte[0]);
+            }
+            if (this.ledgerStorage.hasLimboState(ledgerId)) {
+                log.info("Event: {}, runId: {}, metadata: {}, ledger: {}",
+                        Events.RECOVER_LIMBO_LEDGER, runId, origMetadata, ledgerId);
+                return recoverLedger(ledgerId, runId)
+                    .toMaybe()
+                    .onErrorResumeNext(t -> {
+                            if (t instanceof BKException.BKNoSuchLedgerExistsOnMetadataServerException) {
+                                log.info("Event: {}, runId: {}, metadata: {}, ledger: {}",
+                                        Events.RECOVER_LIMBO_LEDGER_MISSING, runId, origMetadata, ledgerId);
+                                return Maybe.empty();
+                            } else {
+                                log.info("Event: {}, runId: {}, metadata: {}, ledger: {}",
+                                        Events.RECOVER_LIMBO_LEDGER_ERROR, runId, origMetadata, ledgerId);
+                                return Maybe.error(t);
+                            }
+                        });
+            } else {
+                return Maybe.just(origMetadata);
+            }
+        } catch (IOException ioe) {
+            return Maybe.error(ioe);
+        }
+    }
+
+    Single<LedgerMetadata> recoverLedger(long ledgerId, String runId) {
+        return Single.create((emitter) ->
+                admin.asyncOpenLedger(ledgerId, (rc, handle, ctx) -> {
+                        if (rc != BKException.Code.OK) {
+                            emitter.onError(BKException.create(rc));
+                        } else {
+                            LedgerMetadata metadata = handle.getLedgerMetadata();
+                            handle.closeAsync().whenComplete((ignore, exception) -> {
+                                    if (exception != null) {
+                                        log.warn("Event: {}, runId: {}, ledger: {}",
+                                                Events.RECOVER_LIMBO_LEDGER_CLOSE_ERROR, runId, ledgerId, exception);
+                                    }
+                                });
+                            emitter.onSuccess(metadata);
+                        }
+                    }, null));
+
+    }
+
+    /**
+     * Check whether the local storage has all the entries as specified in the metadata.
+     * If not, copy them from other available nodes.
+
+     * Returns a single value which is the ledgerId or an error if any entry failed to copy
+     * should throw error if any entry failed to copy.
+     */
+    Single<Long> checkAndRecoverLedgerEntries(long ledgerId, LedgerMetadata metadata,
+                                              String runId) {
+        WriteSets writeSets = new WriteSets(metadata.getEnsembleSize(),
+                                            metadata.getWriteQuorumSize());
+
+        NavigableMap<Long, Integer> bookieIndices = metadata.getAllEnsembles()
+            .entrySet().stream()
+            .collect(ImmutableSortedMap.toImmutableSortedMap(Comparator.naturalOrder(),
+                                                             e -> e.getKey(),
+                                                             e -> e.getValue().indexOf(bookieId)));
+
+        long lastKnownEntry;
+        if (metadata.isClosed()) {
+            lastKnownEntry = metadata.getLastEntryId();
+        } else {
+            // if ledger is not closed, last known entry is the last entry of
+            // the penultimate ensemble
+            lastKnownEntry = metadata.getAllEnsembles().lastEntry().getKey() - 1;
+        }
+        if (lastKnownEntry < 0) {
+            return Single.just(ledgerId);
+        }
+
+        EntryCopier.Batch batch;
+        try {
+            batch = entryCopier.newBatch(ledgerId, metadata);
+        } catch (IOException ioe) {
+            return Single.error(ioe);
+        }
+        AtomicLong byteCount = new AtomicLong(0);
+        AtomicInteger count = new AtomicInteger(0);
+        AtomicInteger errorCount = new AtomicInteger(0);
+        AtomicReference<Throwable> firstError = new AtomicReference<>(null);
+        log.info("Event: {}, runId: {}, metadata: {}, ledger: {}",
+                Events.LEDGER_CHECK_AND_COPY_START, runId, metadata, ledgerId);
+        return Flowable.rangeLong(0, lastKnownEntry + 1)
+            .subscribeOn(scheduler, false)
+            .flatMapMaybe((entryId) -> {
+                    return maybeCopyEntry(writeSets, bookieIndices, ledgerId, entryId, batch)
+                        .doOnError((t) -> {
+                                firstError.compareAndSet(null, t);
+                                errorCount.incrementAndGet();
+                            });
+                }, true /* delayErrors */, MAX_ENTRIES_INFLIGHT)
+            .doOnNext((bytes) -> {
+                    byteCount.addAndGet(bytes);
+                    count.incrementAndGet();
+                })
+            .count() // do nothing with result, but gives a single even if empty
+            .doOnTerminate(() -> {
+                    if (firstError.get() != null) {
+                        log.warn("Event: {}, runId: {}, metadata: {}, ledger: {}, entries: {}, bytes: {}, errors: {}",
+                                Events.LEDGER_CHECK_AND_COPY_END, runId,
+                                metadata, ledgerId, count.get(), byteCount.get(), firstError.get());
+                    } else {
+                        log.info("Event: {}, runId: {}, metadata: {}, ledger: {}, entries: {}, bytes: {}, errors: 0",
+                                Events.LEDGER_CHECK_AND_COPY_END, runId,
+                                metadata, ledgerId, count.get(), byteCount.get());
+                    }
+                })
+            .map(ignore -> ledgerId);
+    }
+
+    /**
+     * @return the number of bytes copied.
+     */
+    Maybe<Long> maybeCopyEntry(WriteSets writeSets, NavigableMap<Long, Integer> bookieIndices,
+                                     long ledgerId, long entryId, EntryCopier.Batch batch) {
+        try {
+            if (isEntryMissing(writeSets, bookieIndices, ledgerId, entryId)) {
+                return Maybe.fromCompletionStage(batch.copyFromAvailable(entryId));
+            } else {
+                return Maybe.empty();
+            }
+        } catch (BookieException | IOException ioe) {
+            return Maybe.error(ioe);
+        }
+    }
+
+    boolean isEntryMissing(WriteSets writeSets, NavigableMap<Long, Integer> bookieIndices,
+                           long ledgerId, long entryId) throws IOException, BookieException {
+        int bookieIndexForEntry = bookieIndices.floorEntry(entryId).getValue();
+        if (bookieIndexForEntry < 0) {
+            return false;
+        }
+
+        return writeSets.getForEntry(entryId).contains(bookieIndexForEntry)
+            && !ledgerStorage.entryExists(ledgerId, entryId);
+    }
+
+    static boolean ensemblesContainBookie(LedgerMetadata metadata, BookieId bookieId) {
+        return metadata.getAllEnsembles().values().stream()
+            .anyMatch(ensemble -> ensemble.contains(bookieId));
+    }
+}
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/datainteg/DataIntegrityCookieValidation.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/datainteg/DataIntegrityCookieValidation.java
new file mode 100644
index 0000000..75ce5b1
--- /dev/null
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/datainteg/DataIntegrityCookieValidation.java
@@ -0,0 +1,164 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.bookkeeper.bookie.datainteg;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.net.UnknownHostException;
+import java.text.MessageFormat;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.ExecutionException;
+import org.apache.bookkeeper.bookie.BookieException;
+import org.apache.bookkeeper.bookie.BookieImpl;
+import org.apache.bookkeeper.bookie.Cookie;
+import org.apache.bookkeeper.bookie.CookieValidation;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.discover.RegistrationManager;
+import org.apache.bookkeeper.net.BookieId;
+import org.apache.bookkeeper.versioning.Version;
+import org.apache.bookkeeper.versioning.Versioned;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An implementation of the CookieValidation interface that allows for auto-stamping
+ * cookies when configured and used in conjunction with the data integrity service.
+ * Because the data integrity service can heal a bookie with lost data due to a disk
+ * failure, a bookie can auto stamp new cookies as part of the healing process.
+ */
+public class DataIntegrityCookieValidation implements CookieValidation {
+    private static final Logger log = LoggerFactory.getLogger(DataIntegrityCookieValidation.class);
+    private final ServerConfiguration conf;
+    private final BookieId bookieId;
+    private final RegistrationManager registrationManager;
+    private final DataIntegrityCheck dataIntegCheck;
+
+    public DataIntegrityCookieValidation(ServerConfiguration conf,
+                                         RegistrationManager registrationManager,
+                                         DataIntegrityCheck dataIntegCheck)
+            throws UnknownHostException {
+        this.conf = conf;
+        this.registrationManager = registrationManager;
+        this.bookieId = BookieImpl.getBookieId(conf);
+        this.dataIntegCheck = dataIntegCheck;
+    }
+
+    private Optional<Versioned<Cookie>> getRegManagerCookie() throws BookieException {
+        try {
+            return Optional.of(Cookie.readFromRegistrationManager(registrationManager, bookieId));
+        } catch (BookieException.CookieNotFoundException noCookieException) {
+            return Optional.empty();
+        }
+    }
+
+    private List<Optional<Cookie>> collectDirectoryCookies(List<File> directories) throws BookieException {
+        List<Optional<Cookie>> cookies = new ArrayList<>();
+        for (File d : directories) {
+            try {
+                cookies.add(Optional.of(Cookie.readFromDirectory(d)));
+            } catch (FileNotFoundException fnfe) {
+                cookies.add(Optional.empty());
+            } catch (IOException ioe) {
+                throw new BookieException.InvalidCookieException(ioe);
+            }
+        }
+        return cookies;
+    }
+
+    private void stampCookie(Cookie masterCookie, Version expectedVersion, List<File> directories)
+            throws BookieException {
+        // stamp to ZK first as it's the authoritive cookie. If this fails part way through
+        // stamping the directories, then a data integrity check will occur.
+        log.info("Stamping cookie to ZK");
+        masterCookie.writeToRegistrationManager(registrationManager, conf, expectedVersion);
+        for (File d : directories) {
+            try {
+                log.info("Stamping cookie to directory {}", d);
+                masterCookie.writeToDirectory(d);
+            } catch (IOException ioe) {
+                log.error("Exception writing cookie to {}", ioe);
+                throw new BookieException.InvalidCookieException(ioe);
+            }
+        }
+    }
+
+    @Override
+    public void checkCookies(List<File> directories)
+            throws BookieException, InterruptedException {
+        String instanceId = registrationManager.getClusterInstanceId();
+        if (instanceId == null) {
+            throw new BookieException.InvalidCookieException("Cluster instance ID unavailable");
+        }
+        Cookie masterCookie;
+        try {
+            masterCookie = Cookie.generateCookie(conf).setInstanceId(instanceId).build();
+        } catch (UnknownHostException uhe) {
+            throw new BookieException.InvalidCookieException(uhe);
+        }
+
+        // collect existing cookies
+        Optional<Versioned<Cookie>> regManagerCookie = getRegManagerCookie();
+        List<Optional<Cookie>> directoryCookies = collectDirectoryCookies(directories);
+
+        // if master is empty, everything must be empty, otherwise the cluster is messed up
+        if (!regManagerCookie.isPresent()) {
+            // if everything is empty, it's a new install, just stamp the cookies
+            if (directoryCookies.stream().noneMatch(Optional::isPresent)) {
+                log.info("New environment found. Stamping cookies");
+                stampCookie(masterCookie, Version.NEW, directories);
+            } else {
+                String errorMsg =
+                    "Cookie missing from ZK. Either it was manually deleted, "
+                    + "or the bookie was started pointing to a different ZK cluster "
+                    + "than the one it was originally started with. "
+                    + "This requires manual intervention to fix";
+                log.error(errorMsg);
+                throw new BookieException.InvalidCookieException(errorMsg);
+            }
+        } else if (!regManagerCookie.get().getValue().equals(masterCookie)
+                   || !directoryCookies.stream().allMatch(c -> c.map(masterCookie::equals).orElse(false))) {
+            if (conf.isDataIntegrityStampMissingCookiesEnabled()) {
+                log.warn("ZK cookie({}) or directory cookies({}) do not match master cookie ({}), running check",
+                        regManagerCookie, directoryCookies, masterCookie);
+                try {
+                    dataIntegCheck.runPreBootCheck("INVALID_COOKIE").get();
+                } catch (ExecutionException ee) {
+                    if (ee.getCause() instanceof BookieException) {
+                        throw (BookieException) ee.getCause();
+                    } else {
+                        throw new BookieException.InvalidCookieException(ee.getCause());
+                    }
+                }
+                log.info("Environment should be in a sane state. Stamp new cookies");
+                stampCookie(masterCookie, regManagerCookie.get().getVersion(), directories);
+            } else {
+                String errorMsg = MessageFormat.format(
+                        "ZK cookie({0}) or directory cookies({1}) do not match master cookie ({2})"
+                                + " and missing cookie stamping is disabled.",
+                        regManagerCookie, directoryCookies, masterCookie);
+                log.error(errorMsg);
+                throw new BookieException.InvalidCookieException(errorMsg);
+            }
+        } // else all cookies match the masterCookie, meaning nothing has changed in the configuration
+    }
+}
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/datainteg/DataIntegrityService.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/datainteg/DataIntegrityService.java
new file mode 100644
index 0000000..3d2c040
--- /dev/null
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/datainteg/DataIntegrityService.java
@@ -0,0 +1,111 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.bookkeeper.bookie.datainteg;
+
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import java.io.IOException;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.common.component.AbstractLifecycleComponent;
+import org.apache.bookkeeper.server.conf.BookieConfiguration;
+import org.apache.bookkeeper.stats.StatsLogger;
+
+/**
+ * An abstract lifecycle component that can perform data integrity checking.
+ */
+@Slf4j
+public class DataIntegrityService extends AbstractLifecycleComponent<BookieConfiguration> {
+    private final DataIntegrityCheck check;
+    private final ScheduledExecutorService scheduler;
+    private ScheduledFuture<?> scheduledFuture;
+
+    public DataIntegrityService(BookieConfiguration conf,
+                                StatsLogger statsLogger,
+                                DataIntegrityCheck check) {
+        super("data-integ", conf, statsLogger);
+        this.check = check;
+        scheduler = Executors.newSingleThreadScheduledExecutor(
+                new ThreadFactoryBuilder()
+                .setNameFormat("bookie-data-integ-%d")
+                .setUncaughtExceptionHandler(
+                        (t, ex) -> log.error("Event: {}, thread: {}",
+                                Events.DATA_INTEG_SERVICE_UNCAUGHT_ERROR,
+                                t, ex))
+                .build());
+        scheduledFuture = null;
+    }
+
+    // allow tests to reduce interval
+    protected int interval() {
+        return 3;
+    }
+
+    protected TimeUnit intervalUnit() {
+        return TimeUnit.SECONDS;
+    }
+
+    @Override
+    protected void doStart() {
+        log.info("Event: {}, interval: {}, intervalUnit: {}",
+                        Events.DATA_INTEG_SERVICE_START, interval(), intervalUnit());
+        synchronized (this) {
+            scheduledFuture = scheduler.scheduleAtFixedRate(() -> {
+                    try {
+                        if (check.needsFullCheck()) {
+                            check.runFullCheck().get();
+                        }
+                    } catch (InterruptedException ie) {
+                        log.warn("Event: {}", Events.DATA_INTEG_SERVICE_INTERRUPTED, ie);
+                        Thread.currentThread().interrupt();
+                    } catch (Throwable t) {
+                        log.error("Event: {}", Events.DATA_INTEG_SERVICE_ERROR, t);
+                    }
+                }, 0, interval(), intervalUnit());
+        }
+    }
+
+    @Override
+    protected void doStop() {
+        log.info("Event: {}", Events.DATA_INTEG_SERVICE_STOP);
+        synchronized (this) {
+            if (scheduledFuture != null) {
+                scheduledFuture.cancel(true);
+                scheduledFuture = null;
+            }
+        }
+    }
+
+    @Override
+    protected void doClose() throws IOException {
+        synchronized (this) {
+            // just in case stop didn't get called, the scheduledfuture
+            // would stop the scheduler from shutting down
+            if (scheduledFuture != null) {
+                scheduledFuture.cancel(true);
+                scheduledFuture = null;
+            }
+        }
+
+        scheduler.shutdown();
+    }
+}
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/datainteg/EntryCopier.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/datainteg/EntryCopier.java
new file mode 100644
index 0000000..9e3598d
--- /dev/null
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/datainteg/EntryCopier.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.bookkeeper.bookie.datainteg;
+
+import java.io.IOException;
+import java.util.concurrent.CompletableFuture;
+import org.apache.bookkeeper.client.api.LedgerMetadata;
+
+/**
+ * Interface for copying entries from other bookies.
+ * The implementation should take care of selecting the order of the replicas
+ * from which we try to read, taking into account stickiness and errors.
+ * The implementation should take care of rate limiting.
+ */
+public interface EntryCopier {
+    /**
+     * Start copying a new batch. In general, there should be a batch per ledger.
+     */
+    Batch newBatch(long ledgerId, LedgerMetadata metadata) throws IOException;
+
+    /**
+     * An interface for a batch to be copied.
+     */
+    interface Batch {
+        /**
+         * Copy an entry from a remote bookie and store it locally.
+         * @return the number of bytes copied.
+         */
+        CompletableFuture<Long> copyFromAvailable(long entryId);
+    }
+}
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/datainteg/EntryCopierImpl.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/datainteg/EntryCopierImpl.java
new file mode 100644
index 0000000..8d1df94
--- /dev/null
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/datainteg/EntryCopierImpl.java
@@ -0,0 +1,293 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.bookkeeper.bookie.datainteg;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Ticker;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSortedMap;
+
+import io.netty.buffer.ByteBuf;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Random;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.bookie.LedgerStorage;
+import org.apache.bookkeeper.client.BKException;
+import org.apache.bookkeeper.client.api.LedgerMetadata;
+import org.apache.bookkeeper.net.BookieId;
+import org.apache.bookkeeper.proto.BookieClient;
+import org.apache.bookkeeper.proto.BookieProtocol;
+
+/**
+ * Implementation for the EntryCopier interface. Handles the reading of entries
+ * from peer bookies.
+ */
+@Slf4j
+public class EntryCopierImpl implements EntryCopier {
+    private static final long SINBIN_DURATION_MS = TimeUnit.MINUTES.toMillis(1);
+    private final BookieId bookieId;
+    private final BookieClient bookieClient;
+    private final LedgerStorage storage;
+    private final Ticker ticker;
+    private final SinBin sinBin;
+
+    public EntryCopierImpl(BookieId bookieId,
+                    BookieClient bookieClient,
+                    LedgerStorage storage,
+                    Ticker ticker) {
+        this.bookieId = bookieId;
+        this.bookieClient = bookieClient;
+        this.storage = storage;
+        this.ticker = ticker;
+        this.sinBin = new SinBin(ticker);
+    }
+
+    @Override
+    public Batch newBatch(long ledgerId, LedgerMetadata metadata) throws IOException {
+        if (!storage.ledgerExists(ledgerId)) {
+            storage.setMasterKey(ledgerId, metadata.getPassword());
+        }
+        return new BatchImpl(bookieId, ledgerId, metadata, sinBin);
+    }
+
+    @VisibleForTesting
+    class BatchImpl implements Batch {
+        private final long ledgerId;
+        private final LedgerMetadata metadata;
+        private final SinBin sinBin;
+        private volatile ImmutableSortedMap<Long, WriteSets> writeSets;
+
+        BatchImpl(BookieId bookieId,
+                  long ledgerId, LedgerMetadata metadata,
+                  SinBin sinBin) {
+            this.ledgerId = ledgerId;
+            this.metadata = metadata;
+            this.sinBin = sinBin;
+            updateWriteSets();
+        }
+
+        private void updateWriteSets() {
+            // clear non-erroring bookies
+
+            // in theory we should be able to have a single set of writesets per ledger,
+            // however, if there are multiple ensembles, bookies will move around, and we
+            // still want to avoid erroring bookies
+            this.writeSets = preferredBookieIndices(bookieId, metadata,
+                                                    sinBin.getErrorBookies(), ledgerId)
+                .entrySet().stream().collect(
+                        ImmutableSortedMap.toImmutableSortedMap(
+                                Comparator.naturalOrder(),
+                                e -> e.getKey(),
+                                e -> new WriteSets(e.getValue(),
+                                                   metadata.getEnsembleSize(),
+                                                   metadata.getWriteQuorumSize())));
+        }
+
+        @VisibleForTesting
+        void notifyBookieError(BookieId bookie) {
+            if (sinBin.addFailed(bookie)) {
+                updateWriteSets();
+            }
+        }
+
+        @Override
+        public CompletableFuture<Long> copyFromAvailable(long entryId) {
+            if (entryId < 0) {
+                throw new IllegalArgumentException(
+                        String.format("Entry ID (%d) can't be less than 0", entryId));
+            }
+            if (metadata.isClosed() && entryId > metadata.getLastEntryId()) {
+                throw new IllegalArgumentException(
+                        String.format("Invalid entry id (%d), last entry for ledger %d is %d",
+                                      entryId, ledgerId, metadata.getLastEntryId()));
+            }
+            CompletableFuture<Long> promise = new CompletableFuture<>();
+            fetchEntry(entryId).whenComplete((buffer, exception) -> {
+                    if (exception != null) {
+                        promise.completeExceptionally(exception);
+                    } else {
+                        try {
+                            long length = buffer.readableBytes();
+                            storage.addEntry(buffer);
+                            promise.complete(length);
+                        } catch (Throwable t) {
+                            promise.completeExceptionally(t);
+                        } finally {
+                            buffer.release();
+                        }
+                    }
+                });
+            return promise;
+        }
+
+        @VisibleForTesting
+        CompletableFuture<ByteBuf> fetchEntry(long entryId) {
+            List<BookieId> ensemble = metadata.getEnsembleAt(entryId);
+            ImmutableList<Integer> writeSet = writeSets.floorEntry(entryId).getValue().getForEntry(entryId);
+            int attempt = 0;
+            CompletableFuture<ByteBuf> promise = new CompletableFuture<>();
+            fetchRetryLoop(entryId, attempt,
+                           ensemble, writeSet,
+                           promise, Optional.empty());
+            return promise;
+        }
+
+        private void fetchRetryLoop(long entryId, int attempt,
+                                    List<BookieId> ensemble,
+                                    ImmutableList<Integer> writeSet,
+                                    CompletableFuture<ByteBuf> promise,
+                                    Optional<Throwable> firstException) {
+            if (attempt >= writeSet.size()) {
+                promise.completeExceptionally(
+                        firstException.orElse(new BKException.BKReadException()));
+                return;
+            }
+            BookieId bookie = ensemble.get(writeSet.get(attempt));
+            readEntry(bookie, ledgerId, entryId)
+                .whenComplete((buffer, exception) -> {
+                        if (exception != null) {
+                            notifyBookieError(bookie);
+                            Optional<Throwable> firstException1 =
+                                firstException.isPresent() ? firstException : Optional.of(exception);
+                            fetchRetryLoop(entryId, attempt + 1,
+                                           ensemble, writeSet, promise, firstException1);
+                        } else {
+                            promise.complete(buffer);
+                        }
+                    });
+        }
+    }
+
+    // convert callback api to future api
+    private CompletableFuture<ByteBuf> readEntry(BookieId bookieId,
+                                                 long ledgerId, long entryId) {
+        CompletableFuture<ByteBuf> promise = new CompletableFuture<>();
+        bookieClient.readEntry(bookieId, ledgerId, entryId,
+                               (rc, ledgerId1, entryId1, buffer, ctx1) -> {
+                                   if (rc != BKException.Code.OK) {
+                                       promise.completeExceptionally(BKException.create(rc));
+                                   } else {
+                                       buffer.retain();
+                                       promise.complete(buffer);
+                                   }
+                               }, null, BookieProtocol.FLAG_NONE);
+        return promise;
+    }
+
+    /**
+     * Generate a map of preferred bookie indices. For each ensemble, generate the order
+     * in which bookies should be tried for entries, notwithstanding errors.
+     * For example, if a e5,w2,a2 ensemble has the bookies:
+     * [bookie1, bookie2, bookie3, bookie4, bookie5]
+     * and the current bookie is bookie2, then we should return something like:
+     * [4, 2, 0, 3]
+     * Then when retrieving an entry, even though it is only written to 2, we try the bookie
+     * in the order from this list. This will cause more requests to go to the same bookie,
+     * which should give us the benefit of read locality.
+     * We don't want to simply sort by bookie id, as that would cause the same bookies to be
+     * loaded for all ensembles.
+     * Bookies which have presented errors are always tried last.
+     */
+    @VisibleForTesting
+    static ImmutableSortedMap<Long, ImmutableList<Integer>> preferredBookieIndices(
+            BookieId bookieId,
+            LedgerMetadata metadata,
+            Set<BookieId> errorBookies,
+            long seed) {
+        return metadata.getAllEnsembles().entrySet().stream()
+            .collect(ImmutableSortedMap.toImmutableSortedMap(
+                             Comparator.naturalOrder(),
+                             e -> e.getKey(),
+                             e -> {
+                                 List<BookieId> ensemble = e.getValue();
+                                 // get indices of the interesting bookies
+                                 int myIndex = ensemble.indexOf(bookieId);
+                                 Set<Integer> errorIndices = errorBookies.stream()
+                                     .map(b -> ensemble.indexOf(b)).collect(Collectors.toSet());
+
+                                 // turn bookies into positions and filter out my own
+                                 // bookie id (we're not going to try to read from outself)
+                                 List<Integer> indices = IntStream.range(0, ensemble.size())
+                                     .filter(i -> i != myIndex).boxed().collect(Collectors.toList());
+
+                                 // shuffle the indices based seed (normally ledgerId)
+                                 Collections.shuffle(indices, new Random(seed));
+
+                                 // Move the error bookies to the end
+                                 // Collections#sort is stable, so everything else remains the same
+                                 Collections.sort(indices, (a, b) -> {
+                                         boolean aErr = errorIndices.contains(a);
+                                         boolean bErr = errorIndices.contains(b);
+                                         if (aErr && !bErr) {
+                                             return 1;
+                                         } else if (!aErr && bErr) {
+                                             return -1;
+                                         } else {
+                                             return 0;
+                                         }
+                                     });
+                                 return ImmutableList.copyOf(indices);
+                             }));
+    }
+
+    @VisibleForTesting
+    static class SinBin {
+        private final Ticker ticker;
+        private final ConcurrentMap<BookieId, Long> errorBookies = new ConcurrentHashMap<>();
+
+        SinBin(Ticker ticker) {
+            this.ticker = ticker;
+        }
+
+        /**
+         * Returns true if this is the first error for this bookie.
+         */
+        boolean addFailed(BookieId bookie) {
+            long newDeadline = TimeUnit.NANOSECONDS.toMillis(ticker.read()) + SINBIN_DURATION_MS;
+            Long oldDeadline = errorBookies.put(bookie, newDeadline);
+            return oldDeadline == null;
+        }
+
+        Set<BookieId> getErrorBookies() {
+            long now = TimeUnit.NANOSECONDS.toMillis(ticker.read());
+            Iterator<Map.Entry<BookieId, Long>> iterator = errorBookies.entrySet().iterator();
+            while (iterator.hasNext()) {
+                if (iterator.next().getValue() < now) {
+                    iterator.remove();
+                }
+            }
+            return errorBookies.keySet();
+        }
+    }
+}
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/datainteg/Events.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/datainteg/Events.java
new file mode 100644
index 0000000..5d98411
--- /dev/null
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/datainteg/Events.java
@@ -0,0 +1,235 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.bookkeeper.bookie.datainteg;
+
+enum Events {
+    /**
+     * Data integrity service has started
+     * It runs at an interval to check if a full integrity check is needed,
+     * and if so runs it.
+     */
+    DATA_INTEG_SERVICE_START,
+    /**
+     * Data integrity service has been stopped.
+     */
+    DATA_INTEG_SERVICE_STOP,
+    /**
+     * An exception was thrown on the data integrity service executor
+     * and never caught. This is a programmer error and should be reported
+     * as such.
+     */
+    DATA_INTEG_SERVICE_UNCAUGHT_ERROR,
+    /**
+     * Data integrity service thread interrupted.
+     * This is non-fatal and indicates that the bookie is shutting down.
+     * The full check will resume once the bookie is started again.
+     */
+    DATA_INTEG_SERVICE_INTERRUPTED,
+    /**
+     * An error occurred in the in the data integrity service loop.
+     * This normally indicates that an error occurred in the full check.
+     * The full check will be tried again.
+     * It could also indicate an error checking the NEEDS_INTEGRITY_CHECK
+     * flag, which indicates disk issues.
+     */
+    DATA_INTEG_SERVICE_ERROR,
+
+    /**
+     * Mark a ledger as in-limbo. In limbo ledgers are ledgers for whose
+     * entries we cannot safely answer queries positively or negatively.
+     * These are ledgers which have not been closed and where this bookie
+     * appears in the final ensemble.
+
+     * We may have had an entry in the past, but due to disk failures or
+     * configuration changes it may not currently exist locally. However,
+     * we cannot tell clients that the entry doesn't exist, because the client
+     * would understand that to mean that it never existed, and this would
+     * break consistency in the ledger recovery protocol.
+
+     * For limbo ledgers, all entry level queries should throw an exception.
+
+     * We also mark the ledger as fenced at this point, as it may have been set
+     * on this ledger previously. This means no more writes for this ledger
+     * can come to this bookie.
+     */
+    MARK_LIMBO,
+
+    /**
+     * An error occurred marking the ledger as fenced or as in-limbo.
+     * The most likely cause is a bad disk.
+     * This is a fatal error, as we cannot safely serve entries if we cannot
+     * set limbo and fence flags.
+     */
+    LIMBO_OR_FENCE_ERROR,
+
+    /**
+     * Start the preboot check. The preboot check runs when some configuration
+     * has changed regarding the disk configuration. This may be simply a disk
+     * being added, or it could be the disks being wiped. The preboot check
+     * needs to check which ledgers we are supposed to store according to
+     * ledger metadata. Any unclosed ledgers which contain this bookie in its last
+     * ensemble must be marked as in-limbo, as we don't know if entries from that
+     * ledger have previously existed on this bookie.
+
+     * The preboot check doesn't copy any data. That is left up to the full check
+     * which can run in the background while the bookie is serving data for non-limbo
+     * ledgers.
+
+     * The preboot check has a runId associated which can be used to pull together
+     * all the events from the same run.
+     * The preboot check will set the NEEDS_INTEGRITY_CHECK flag on storage to
+     * trigger a full check after the bookie has booted.
+     */
+    PREBOOT_START,
+    /**
+     * The preboot check has completed successfully. The event contains the number
+     * of ledgers that have been processed.
+     */
+    PREBOOT_END,
+    /**
+     * An error occurred during the preboot check. This is a fatal error as we cannot
+     * safely serve data if the correct ledgers have not been marked as in-limbo. The
+     * error could be due to problems accessing the metadata store, or due to disk
+     * issues.
+     */
+    PREBOOT_ERROR,
+    /**
+     * Preboot found an invalid ledger metadata. All ledger metadata must have at least
+     * one ensemble but the process found one with none.
+     */
+    INVALID_METADATA,
+    /**
+     * Preboot must create a ledger that the bookie does not have but that metadata says
+     * the bookie should have. This can happen due to things like ensemble changes and
+     * when a ledger is closed. If the ledger cannot be created on the bookie then
+     * this error will cause preboot to fail.
+     */
+    ENSURE_LEDGER_ERROR,
+    /**
+     * Initialized the full check. If we have cached metadata from a previous run, or
+     * the preboot check, then we use that. Otherwise we read the metadata from the
+     * metadata store.
+
+     * The full check goes through each ledger for which this bookie is supposed to
+     * store entries and checks that these entries exist on the bookie. If they do not
+     * exist, they are copied from another bookie.
+
+     * Each full check has a runId associated which can be used to find all events from
+     * the check.
+     */
+    FULL_CHECK_INIT,
+    /**
+     * The full check has completed.
+     */
+    FULL_CHECK_COMPLETE,
+    /**
+     * Start iterating through the ledger that should be on this bookie.
+     * The event is annotated with the number of ledgers which will be checked,
+     * which may be fewer that the total number of ledgers on the bookie as
+     * a previous run may have verified that some ledgers are ok and don't need
+     * to be checked.
+     */
+    FULL_CHECK_START,
+    /**
+     * The full check has completed. This can be an info event or an error event.
+     * The event is annotated with the number of ledgers which were checked and found
+     * to be ok, the number that were found to be missing and the number for which
+     * errors occurred during the check. The missing ledgers have been deleted on
+     * the cluster, so don't need to be processed again. If there is a non-zero of
+     * ledgers with errors, the whole event is an error.
+
+     * An error for this event is non-fatal. Any ledgers which finished with error
+     * will be processed again the next time the full check runs. The full check
+     * continues retrying until there are no errors.
+     */
+    FULL_CHECK_END,
+    /**
+     * An error occurred during the full check, but not while processing ledgers.
+     * This error could occur while flushing the ledger storage or clearing the
+     * full check flag.
+     */
+    FULL_CHECK_ERROR,
+
+    /**
+     * The full check will use cached metadata.
+     */
+    USE_CACHED_METADATA,
+    /**
+     * The full check will read the metadata from the metadata store.
+     */
+    REFRESH_METADATA,
+
+    /**
+     * The NEEDS_INTEGRITY_CHECK will be cleared from the ledger storage.
+     * This signifies that the ledger storage contains everything it should
+     * and the full check does not need to be retried, even after reboot.
+     */
+    CLEAR_INTEGCHECK_FLAG,
+
+    /**
+     * An error occurred while clearing the limbo flag for a ledger.
+     * This is generally a disk error. This error is non-fatal and the operation
+     * will be tried again on the next full check.
+     */
+    CLEAR_LIMBO_ERROR,
+    /**
+     * Recover a ledger that has been marked as in limbo. This runs the ledger
+     * recovery algorithm to find the last entry of the ledger and mark the ledger
+     * as closed. As the ledger is marked as in-limbo locally, the current bookie
+     * not take part in the recovery process apart from initializing it.
+
+     * Once recovery completes successfully, the limbo flag can be cleared for the
+     * ledger.
+     */
+    RECOVER_LIMBO_LEDGER,
+    /**
+     * The ledger has been deleted from the ledger metadata store, so we don't need
+     * to continue any processing on it.
+     */
+    RECOVER_LIMBO_LEDGER_MISSING,
+    /**
+     * An error occurred during recovery. This could be due to not having enough
+     * bookies available to recover the ledger.
+     * The error is non-fatal. The recovery will be tried again on the next run of
+     * ledger recovery.
+     */
+    RECOVER_LIMBO_LEDGER_ERROR,
+    /**
+     * An error occurred when trying to close the ledger handle of a recovered ledger.
+     * This shouldn't happen, as closing a recovered ledger should not involve any I/O.
+     * This error is non-fatal and the event is registered for informational purposes
+     * only.
+     */
+    RECOVER_LIMBO_LEDGER_CLOSE_ERROR,
+
+    /**
+     * Start checking whether the entries for a ledger exist locally, and copying them
+     * if they do not.
+     */
+    LEDGER_CHECK_AND_COPY_START,
+    /**
+     * Checking and copying has completed for a ledger. If any entry failed to copy
+     * this is a warning event. The ledger will be retried on the next run of the full
+     * check.
+     * This event is annotated with the number of entries copied, the number of errors
+     * and the total number of bytes copied for the ledger.
+     */
+    LEDGER_CHECK_AND_COPY_END
+}
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/datainteg/MetadataAsyncIterator.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/datainteg/MetadataAsyncIterator.java
new file mode 100644
index 0000000..0a86447
--- /dev/null
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/datainteg/MetadataAsyncIterator.java
@@ -0,0 +1,123 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.bookkeeper.bookie.datainteg;
+
+import io.reactivex.rxjava3.core.Completable;
+import io.reactivex.rxjava3.core.Flowable;
+import io.reactivex.rxjava3.core.Scheduler;
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.function.BiFunction;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.client.BKException;
+import org.apache.bookkeeper.client.api.LedgerMetadata;
+import org.apache.bookkeeper.meta.LedgerManager;
+import org.apache.bookkeeper.versioning.Versioned;
+
+/**
+ * An rxjava ledger metadata iterator.
+ */
+@Slf4j
+public class MetadataAsyncIterator {
+    private final Scheduler scheduler;
+    private final LedgerManager ledgerManager;
+    private final long zkTimeoutMs;
+    private final int maxInFlight;
+
+    MetadataAsyncIterator(Scheduler scheduler,
+                          LedgerManager ledgerManager, int maxInFlight,
+                          int zkTimeout, TimeUnit zkTimeoutUnit) {
+        this.scheduler = scheduler;
+        this.ledgerManager = ledgerManager;
+        this.maxInFlight = maxInFlight;
+        this.zkTimeoutMs = zkTimeoutUnit.toMillis(zkTimeout);
+    }
+
+
+    private static class FlatIterator {
+        final LedgerManager.LedgerRangeIterator ranges;
+        Iterator<Long> range = null;
+        FlatIterator(LedgerManager.LedgerRangeIterator ranges) {
+            this.ranges = ranges;
+        }
+        boolean hasNext() throws IOException {
+            if (range == null || !range.hasNext()) {
+                if (ranges.hasNext()) {
+                    range = ranges.next().getLedgers().iterator();
+                }
+            }
+            return range != null && range.hasNext();
+        }
+        Long next() throws IOException {
+            return range.next();
+        }
+    }
+
+    public CompletableFuture<Void> forEach(BiFunction<Long, LedgerMetadata, CompletableFuture<Void>> consumer) {
+        CompletableFuture<Void> promise = new CompletableFuture<>();
+        Flowable.<Long, FlatIterator>generate(
+                () -> new FlatIterator(ledgerManager.getLedgerRanges(zkTimeoutMs)),
+                (iter, emitter) -> {
+                    try {
+                        if (iter.hasNext()) {
+                            emitter.onNext(iter.next());
+                        } else {
+                            emitter.onComplete();
+                        }
+                    } catch (Exception e) {
+                        emitter.onError(e);
+                    }
+                })
+            .subscribeOn(scheduler)
+            .flatMapCompletable((ledgerId) -> Completable.fromCompletionStage(processOne(ledgerId, consumer)),
+                                false /* delayErrors */,
+                                maxInFlight)
+            .subscribe(() -> promise.complete(null),
+                       t -> promise.completeExceptionally(unwrap(t)));
+        return promise;
+    }
+
+    private CompletableFuture<Void> processOne(long ledgerId,
+                                               BiFunction<Long, LedgerMetadata, CompletableFuture<Void>> consumer) {
+        return ledgerManager.readLedgerMetadata(ledgerId)
+            .thenApply(Versioned::getValue)
+            .thenCompose((metadata) -> consumer.apply(ledgerId, metadata))
+            .exceptionally((e) -> {
+                    Throwable realException = unwrap(e);
+                    log.warn("Got exception processing ledger {}", ledgerId, realException);
+                    if (realException instanceof BKException.BKNoSuchLedgerExistsOnMetadataServerException) {
+                        return null;
+                    } else {
+                        throw new CompletionException(realException);
+                    }
+                });
+    }
+
+    static Throwable unwrap(Throwable e) {
+        if (e instanceof CompletionException || e instanceof ExecutionException) {
+            return unwrap(e.getCause());
+        }
+        return e;
+    }
+}
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/datainteg/WriteSets.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/datainteg/WriteSets.java
new file mode 100644
index 0000000..ff9274d
--- /dev/null
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/datainteg/WriteSets.java
@@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.bookkeeper.bookie.datainteg;
+
+import static com.google.common.base.Preconditions.checkState;
+
+import com.google.common.collect.ImmutableList;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Pregenerate the write sets. RoundRobinDistributionSchedule should really be doing this also.
+ */
+class WriteSets {
+    private static final Logger log = LoggerFactory.getLogger(WriteSets.class);
+    private final int ensembleSize;
+    private final ImmutableList<ImmutableList<Integer>> sets;
+
+    WriteSets(List<Integer> preferredOrder,
+              int ensembleSize,
+              int writeQuorumSize) {
+        this.ensembleSize = ensembleSize;
+
+        ImmutableList.Builder<ImmutableList<Integer>> builder =
+            new ImmutableList.Builder<ImmutableList<Integer>>();
+        for (int i = 0; i < ensembleSize; i++) {
+            builder.add(generateWriteSet(preferredOrder, ensembleSize, writeQuorumSize, i));
+        }
+        sets = builder.build();
+    }
+
+    WriteSets(int ensembleSize, int writeQuorumSize) {
+        this(IntStream.range(0, ensembleSize).boxed().collect(Collectors.toList()),
+             ensembleSize, writeQuorumSize);
+    }
+
+    ImmutableList<Integer> getForEntry(long entryId) {
+        return sets.get((int) (entryId % ensembleSize));
+    }
+
+    static ImmutableList<Integer> generateWriteSet(List<Integer> preferredOrder,
+                                                   int ensembleSize,
+                                                   int writeQuorumSize,
+                                                   int offset) {
+        ImmutableList.Builder<Integer> builder =
+            new ImmutableList.Builder<Integer> ();
+        int firstIndex = offset;
+        int lastIndex = (offset + writeQuorumSize - 1) % ensembleSize;
+        for (Integer i : preferredOrder) {
+            if (firstIndex <= lastIndex
+                && i >= firstIndex
+                && i <= lastIndex) {
+                builder.add(i);
+            } else if (lastIndex < firstIndex
+                       && (i <= lastIndex
+                           || i >= firstIndex)) {
+                builder.add(i);
+            }
+        }
+        ImmutableList<Integer> writeSet = builder.build();
+
+        // writeSet may be one smaller than the configured write
+        // set size if we are excluding ourself
+        checkState(writeSet.size() == writeQuorumSize
+                                 || (writeSet.size() == writeQuorumSize - 1));
+        return writeSet;
+    }
+}
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/datainteg/package-info.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/datainteg/package-info.java
new file mode 100644
index 0000000..ff6ab25
--- /dev/null
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/datainteg/package-info.java
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/**
+ * Package of the classes for defining bookie stats.
+ */
+package org.apache.bookkeeper.bookie.datainteg;
\ No newline at end of file
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorage.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorage.java
index 94c4460..2ba63cc 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorage.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorage.java
@@ -36,6 +36,7 @@ import io.netty.util.internal.PlatformDependent;
 import java.io.File;
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.EnumSet;
 import java.util.List;
 import java.util.PrimitiveIterator.OfLong;
 import java.util.concurrent.Executors;
@@ -86,9 +87,14 @@ public class DbLedgerStorage implements LedgerStorage {
             / MB;
     private static final long DEFAULT_READ_CACHE_MAX_SIZE_MB = (long) (0.25 * PlatformDependent.maxDirectMemory())
             / MB;
+
     static final String READ_AHEAD_CACHE_BATCH_SIZE = "dbStorage_readAheadCacheBatchSize";
     private static final int DEFAULT_READ_AHEAD_CACHE_BATCH_SIZE = 100;
 
+    // use the storage assigned to ledger 0 for flags.
+    // if the storage configuration changes, the flags may be lost
+    // but in that case data integrity should kick off anyhow.
+    private static final long STORAGE_FLAGS_KEY = 0L;
     private int numberOfDirs;
     private List<SingleDirectoryDbLedgerStorage> ledgerStorageList;
 
@@ -223,12 +229,17 @@ public class DbLedgerStorage implements LedgerStorage {
     }
 
     @Override
+    public boolean entryExists(long ledgerId, long entryId) throws IOException, BookieException {
+        return getLedgerStorage(ledgerId).entryExists(ledgerId, entryId);
+    }
+
+    @Override
     public boolean setFenced(long ledgerId) throws IOException {
         return getLedgerStorage(ledgerId).setFenced(ledgerId);
     }
 
     @Override
-    public boolean isFenced(long ledgerId) throws IOException {
+    public boolean isFenced(long ledgerId) throws IOException, BookieException {
         return getLedgerStorage(ledgerId).isFenced(ledgerId);
     }
 
@@ -249,12 +260,12 @@ public class DbLedgerStorage implements LedgerStorage {
     }
 
     @Override
-    public ByteBuf getEntry(long ledgerId, long entryId) throws IOException {
+    public ByteBuf getEntry(long ledgerId, long entryId) throws IOException, BookieException {
         return getLedgerStorage(ledgerId).getEntry(ledgerId, entryId);
     }
 
     @Override
-    public long getLastAddConfirmed(long ledgerId) throws IOException {
+    public long getLastAddConfirmed(long ledgerId) throws IOException, BookieException {
         return getLedgerStorage(ledgerId).getLastAddConfirmed(ledgerId);
     }
 
@@ -301,7 +312,7 @@ public class DbLedgerStorage implements LedgerStorage {
     }
 
     @Override
-    public ByteBuf getExplicitLac(long ledgerId) throws IOException {
+    public ByteBuf getExplicitLac(long ledgerId) throws IOException, BookieException {
         return getLedgerStorage(ledgerId).getExplicitLac(ledgerId);
     }
 
@@ -331,7 +342,7 @@ public class DbLedgerStorage implements LedgerStorage {
         return Iterables.concat(listIt);
     }
 
-    public ByteBuf getLastEntry(long ledgerId) throws IOException {
+    public ByteBuf getLastEntry(long ledgerId) throws IOException, BookieException {
         return getLedgerStorage(ledgerId).getLastEntry(ledgerId);
     }
 
@@ -422,4 +433,34 @@ public class DbLedgerStorage implements LedgerStorage {
         throw new UnsupportedOperationException(
                 "getListOfEntriesOfLedger method is currently unsupported for DbLedgerStorage");
     }
+
+    @Override
+    public void setLimboState(long ledgerId) throws IOException {
+        getLedgerStorage(ledgerId).setLimboState(ledgerId);
+    }
+
+    @Override
+    public boolean hasLimboState(long ledgerId) throws IOException {
+        return getLedgerStorage(ledgerId).hasLimboState(ledgerId);
+    }
+
+    @Override
+    public void clearLimboState(long ledgerId) throws IOException {
+        getLedgerStorage(ledgerId).clearLimboState(ledgerId);
+    }
+
+    @Override
+    public EnumSet<StorageState> getStorageStateFlags() throws IOException {
+        return getLedgerStorage(STORAGE_FLAGS_KEY).getStorageStateFlags();
+    }
+
+    @Override
+    public void setStorageStateFlag(StorageState flag) throws IOException {
+        getLedgerStorage(STORAGE_FLAGS_KEY).setStorageStateFlag(flag);
+    }
+
+    @Override
+    public void clearStorageStateFlag(StorageState flag) throws IOException {
+        getLedgerStorage(STORAGE_FLAGS_KEY).clearStorageStateFlag(flag);
+    }
 }
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LedgerMetadataIndex.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LedgerMetadataIndex.java
index 6012d33..eb7548c 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LedgerMetadataIndex.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LedgerMetadataIndex.java
@@ -32,6 +32,7 @@ import java.util.Arrays;
 import java.util.Map.Entry;
 import java.util.concurrent.ConcurrentLinkedQueue;
 import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.ReentrantLock;
 
 import org.apache.bookkeeper.bookie.Bookie;
 import org.apache.bookkeeper.bookie.BookieException;
@@ -50,6 +51,9 @@ import org.slf4j.LoggerFactory;
  * <p>The key is the ledgerId and the value is the {@link LedgerData} content.
  */
 public class LedgerMetadataIndex implements Closeable {
+    // Non-ledger data should have negative ID
+    private static final long STORAGE_FLAGS = -0xeefd;
+
     // Contains all ledgers stored in the bookie
     private final ConcurrentLongHashMap<LedgerData> ledgers;
     private final AtomicInteger ledgersCount;
@@ -62,6 +66,7 @@ public class LedgerMetadataIndex implements Closeable {
 
     // Holds ledger ids that were delete from memory map, and pending to be flushed on db
     private final ConcurrentLinkedQueue<Long> pendingDeletedLedgers;
+    private final ReentrantLock[] locks = new ReentrantLock[16];
 
     public LedgerMetadataIndex(ServerConfiguration conf, KeyValueStorageFactory storageFactory, String basePath,
             StatsLogger stats) throws IOException {
@@ -76,9 +81,11 @@ public class LedgerMetadataIndex implements Closeable {
             while (iterator.hasNext()) {
                 Entry<byte[], byte[]> entry = iterator.next();
                 long ledgerId = ArrayUtil.getLong(entry.getKey(), 0);
-                LedgerData ledgerData = LedgerData.parseFrom(entry.getValue());
-                ledgers.put(ledgerId, ledgerData);
-                ledgersCount.incrementAndGet();
+                if (ledgerId >= 0) {
+                    LedgerData ledgerData = LedgerData.parseFrom(entry.getValue());
+                    ledgers.put(ledgerId, ledgerData);
+                    ledgersCount.incrementAndGet();
+                }
             }
         } finally {
             iterator.close();
@@ -90,6 +97,10 @@ public class LedgerMetadataIndex implements Closeable {
         this.stats = new LedgerMetadataIndexStats(
             stats,
             () -> (long) ledgersCount.get());
+
+        for (int i = 0; i < locks.length; i++) {
+            locks[i] = new ReentrantLock();
+        }
     }
 
     @Override
@@ -112,27 +123,39 @@ public class LedgerMetadataIndex implements Closeable {
     public void set(long ledgerId, LedgerData ledgerData) throws IOException {
         ledgerData = LedgerData.newBuilder(ledgerData).setExists(true).build();
 
-        if (ledgers.put(ledgerId, ledgerData) == null) {
-            if (log.isDebugEnabled()) {
-                log.debug("Added new ledger {}", ledgerId);
+        ReentrantLock lock = lockForLedger(ledgerId);
+        lock.lock();
+        try {
+            if (ledgers.put(ledgerId, ledgerData) == null) {
+                if (log.isDebugEnabled()) {
+                    log.debug("Added new ledger {}", ledgerId);
+                }
+                ledgersCount.incrementAndGet();
             }
-            ledgersCount.incrementAndGet();
-        }
 
-        pendingLedgersUpdates.add(new SimpleEntry<Long, LedgerData>(ledgerId, ledgerData));
-        pendingDeletedLedgers.remove(ledgerId);
+            pendingLedgersUpdates.add(new SimpleEntry<Long, LedgerData>(ledgerId, ledgerData));
+            pendingDeletedLedgers.remove(ledgerId);
+        } finally {
+            lock.unlock();
+        }
     }
 
     public void delete(long ledgerId) throws IOException {
-        if (ledgers.remove(ledgerId) != null) {
-            if (log.isDebugEnabled()) {
-                log.debug("Removed ledger {}", ledgerId);
+        ReentrantLock lock = lockForLedger(ledgerId);
+        lock.lock();
+        try {
+            if (ledgers.remove(ledgerId) != null) {
+                if (log.isDebugEnabled()) {
+                    log.debug("Removed ledger {}", ledgerId);
+                }
+                ledgersCount.decrementAndGet();
             }
-            ledgersCount.decrementAndGet();
-        }
 
-        pendingDeletedLedgers.add(ledgerId);
-        pendingLedgersUpdates.removeIf(e -> e.getKey() == ledgerId);
+            pendingDeletedLedgers.add(ledgerId);
+            pendingLedgersUpdates.removeIf(e -> e.getKey() == ledgerId);
+        } finally {
+            lock.unlock();
+        }
     }
 
     public Iterable<Long> getActiveLedgersInRange(final long firstLedgerId, final long lastLedgerId)
@@ -146,59 +169,131 @@ public class LedgerMetadataIndex implements Closeable {
     }
 
     public boolean setFenced(long ledgerId) throws IOException {
-        LedgerData ledgerData = get(ledgerId);
-        if (ledgerData.getFenced()) {
-            return false;
-        }
+        ReentrantLock lock = lockForLedger(ledgerId);
+        lock.lock();
+        try {
+            LedgerData ledgerData = get(ledgerId);
+            if (ledgerData.getFenced()) {
+                return false;
+            }
 
-        LedgerData newLedgerData = LedgerData.newBuilder(ledgerData).setFenced(true).build();
+            LedgerData newLedgerData = LedgerData.newBuilder(ledgerData).setFenced(true).build();
 
-        if (ledgers.put(ledgerId, newLedgerData) == null) {
-            // Ledger had been deleted
-            if (log.isDebugEnabled()) {
-                log.debug("Re-inserted fenced ledger {}", ledgerId);
+            if (ledgers.put(ledgerId, newLedgerData) == null) {
+                // Ledger had been deleted
+                if (log.isDebugEnabled()) {
+                    log.debug("Re-inserted fenced ledger {}", ledgerId);
+                }
+                ledgersCount.incrementAndGet();
+            } else {
+                if (log.isDebugEnabled()) {
+                    log.debug("Set fenced ledger {}", ledgerId);
+                }
             }
-            ledgersCount.incrementAndGet();
-        } else {
-            if (log.isDebugEnabled()) {
-                log.debug("Set fenced ledger {}", ledgerId);
+
+            pendingLedgersUpdates.add(new SimpleEntry<Long, LedgerData>(ledgerId, newLedgerData));
+            pendingDeletedLedgers.remove(ledgerId);
+            return true;
+        } finally {
+            lock.unlock();
+        }
+    }
+
+    public boolean setLimbo(long ledgerId) throws IOException {
+        ReentrantLock lock = lockForLedger(ledgerId);
+        lock.lock();
+        try {
+            LedgerData ledgerData = get(ledgerId);
+            if (ledgerData.getLimbo()) {
+                return false;
             }
+
+            LedgerData newLedgerData = LedgerData.newBuilder(ledgerData).setLimbo(true).build();
+
+            if (ledgers.put(ledgerId, newLedgerData) == null) {
+                // Ledger had been deleted
+                if (log.isDebugEnabled()) {
+                    log.debug("Re-inserted limbo ledger {}", ledgerId);
+                }
+                ledgersCount.incrementAndGet();
+            } else {
+                if (log.isDebugEnabled()) {
+                    log.debug("Set limbo ledger {}", ledgerId);
+                }
+            }
+
+            pendingLedgersUpdates.add(new SimpleEntry<Long, LedgerData>(ledgerId, newLedgerData));
+            pendingDeletedLedgers.remove(ledgerId);
+            return true;
+        } finally {
+            lock.unlock();
         }
+    }
+
+    public boolean clearLimbo(long ledgerId) throws IOException {
+        ReentrantLock lock = lockForLedger(ledgerId);
+        lock.lock();
+        try {
+            LedgerData ledgerData = get(ledgerId);
+            boolean oldValue = ledgerData != null ? ledgerData.getLimbo() : false;
+            LedgerData newLedgerData = LedgerData.newBuilder(ledgerData).setLimbo(false).build();
+
+            if (ledgers.put(ledgerId, newLedgerData) == null) {
+                // Ledger had been deleted
+                if (log.isDebugEnabled()) {
+                    log.debug("Re-inserted limbo ledger {}", ledgerId);
+                }
+                ledgersCount.incrementAndGet();
+            } else {
+                if (log.isDebugEnabled()) {
+                    log.debug("Set limbo ledger {}", ledgerId);
+                }
+            }
 
-        pendingLedgersUpdates.add(new SimpleEntry<Long, LedgerData>(ledgerId, newLedgerData));
-        pendingDeletedLedgers.remove(ledgerId);
-        return true;
+            pendingLedgersUpdates.add(new SimpleEntry<Long, LedgerData>(ledgerId, newLedgerData));
+            pendingDeletedLedgers.remove(ledgerId);
+            return oldValue;
+        } finally {
+            lock.unlock();
+        }
     }
 
+
     public void setMasterKey(long ledgerId, byte[] masterKey) throws IOException {
-        LedgerData ledgerData = ledgers.get(ledgerId);
-        if (ledgerData == null) {
-            // New ledger inserted
-            ledgerData = LedgerData.newBuilder().setExists(true).setFenced(false)
+        ReentrantLock lock = lockForLedger(ledgerId);
+        lock.lock();
+        try {
+            LedgerData ledgerData = ledgers.get(ledgerId);
+            if (ledgerData == null) {
+                // New ledger inserted
+                ledgerData = LedgerData.newBuilder().setExists(true).setFenced(false)
                     .setMasterKey(ByteString.copyFrom(masterKey)).build();
-            if (log.isDebugEnabled()) {
-                log.debug("Inserting new ledger {}", ledgerId);
-            }
-        } else {
-            byte[] storedMasterKey = ledgerData.getMasterKey().toByteArray();
-            if (ArrayUtil.isArrayAllZeros(storedMasterKey)) {
-                // update master key of the ledger
-                ledgerData = LedgerData.newBuilder(ledgerData).setMasterKey(ByteString.copyFrom(masterKey)).build();
                 if (log.isDebugEnabled()) {
-                    log.debug("Replace old master key {} with new master key {}", storedMasterKey, masterKey);
+                    log.debug("Inserting new ledger {}", ledgerId);
+                }
+            } else {
+                byte[] storedMasterKey = ledgerData.getMasterKey().toByteArray();
+                if (ArrayUtil.isArrayAllZeros(storedMasterKey)) {
+                    // update master key of the ledger
+                    ledgerData = LedgerData.newBuilder(ledgerData).setMasterKey(ByteString.copyFrom(masterKey)).build();
+                    if (log.isDebugEnabled()) {
+                        log.debug("Replace old master key {} with new master key {}", storedMasterKey, masterKey);
+                    }
+                } else if (!Arrays.equals(storedMasterKey, masterKey) && !ArrayUtil.isArrayAllZeros(masterKey)) {
+                    log.warn("Ledger {} masterKey in db can only be set once.", ledgerId);
+                    throw new IOException(BookieException.create(BookieException.Code.IllegalOpException));
                 }
-            } else if (!Arrays.equals(storedMasterKey, masterKey) && !ArrayUtil.isArrayAllZeros(masterKey)) {
-                log.warn("Ledger {} masterKey in db can only be set once.", ledgerId);
-                throw new IOException(BookieException.create(BookieException.Code.IllegalOpException));
             }
-        }
 
-        if (ledgers.put(ledgerId, ledgerData) == null) {
-            ledgersCount.incrementAndGet();
-        }
+            if (ledgers.put(ledgerId, ledgerData) == null) {
+                ledgersCount.incrementAndGet();
+            }
 
-        pendingLedgersUpdates.add(new SimpleEntry<Long, LedgerData>(ledgerId, ledgerData));
-        pendingDeletedLedgers.remove(ledgerId);
+            pendingLedgersUpdates.add(new SimpleEntry<Long, LedgerData>(ledgerId, ledgerData));
+            pendingDeletedLedgers.remove(ledgerId);
+        } finally {
+            lock.unlock();
+        }
     }
 
     /**
@@ -249,6 +344,56 @@ public class LedgerMetadataIndex implements Closeable {
         key.recycle();
     }
 
+    private ReentrantLock lockForLedger(long ledgerId) {
+        return locks[Math.abs((int) ledgerId) % locks.length];
+    }
+
+    int getStorageStateFlags() throws IOException {
+        LongWrapper keyWrapper = LongWrapper.get();
+        LongWrapper currentWrapper = LongWrapper.get();
+
+        try {
+            keyWrapper.set(STORAGE_FLAGS);
+            synchronized (ledgersDb) {
+                int current = 0;
+                if (ledgersDb.get(keyWrapper.array, currentWrapper.array) >= 0) {
+                    current = (int) currentWrapper.getValue();
+                }
+                return current;
+            }
+        } finally {
+            keyWrapper.recycle();
+            currentWrapper.recycle();
+        }
+    }
+
+    boolean setStorageStateFlags(int expected, int newFlags) throws IOException {
+        LongWrapper keyWrapper = LongWrapper.get();
+        LongWrapper currentWrapper = LongWrapper.get();
+        LongWrapper newFlagsWrapper = LongWrapper.get();
+
+        try {
+            keyWrapper.set(STORAGE_FLAGS);
+            newFlagsWrapper.set(newFlags);
+            synchronized (ledgersDb) {
+                int current = 0;
+                if (ledgersDb.get(keyWrapper.array, currentWrapper.array) >= 0) {
+                    current = (int) currentWrapper.getValue();
+                }
+                if (current == expected) {
+                    ledgersDb.put(keyWrapper.array, newFlagsWrapper.array);
+                    ledgersDb.sync();
+                    return true;
+                }
+            }
+        } finally {
+            keyWrapper.recycle();
+            currentWrapper.recycle();
+            newFlagsWrapper.recycle();
+        }
+        return false;
+    }
+
     private static final Logger log = LoggerFactory.getLogger(LedgerMetadataIndex.class);
 
     void setExplicitLac(long ledgerId, ByteBuf lac) throws IOException {
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/ReadCache.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/ReadCache.java
index 986c741..803666b 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/ReadCache.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/ReadCache.java
@@ -155,6 +155,27 @@ public class ReadCache implements Closeable {
         return null;
     }
 
+    public boolean hasEntry(long ledgerId, long entryId) {
+        lock.readLock().lock();
+
+        try {
+            int size = cacheSegments.size();
+            for (int i = 0; i < size; i++) {
+                int segmentIdx = (currentSegmentIdx + (size - i)) % size;
+
+                LongPair res = cacheIndexes.get(segmentIdx).get(ledgerId, entryId);
+                if (res != null) {
+                    return true;
+                }
+            }
+        } finally {
+            lock.readLock().unlock();
+        }
+
+        // Entry not found in any segment
+        return false;
+    }
+
     /**
      * @return the total size of cached entries
      */
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/SingleDirectoryDbLedgerStorage.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/SingleDirectoryDbLedgerStorage.java
index 4643652..4d2e4ad 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/SingleDirectoryDbLedgerStorage.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/SingleDirectoryDbLedgerStorage.java
@@ -21,8 +21,10 @@
 package org.apache.bookkeeper.bookie.storage.ldb;
 
 import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkState;
 
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Lists;
 import com.google.protobuf.ByteString;
 
@@ -34,7 +36,9 @@ import io.netty.util.concurrent.DefaultThreadFactory;
 import java.io.File;
 import java.io.IOException;
 import java.util.Collections;
+import java.util.EnumSet;
 import java.util.List;
+import java.util.Map;
 import java.util.PrimitiveIterator.OfLong;
 import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.ExecutorService;
@@ -286,11 +290,62 @@ public class SingleDirectoryDbLedgerStorage implements CompactableLedgerStorage
     }
 
     @Override
-    public boolean isFenced(long ledgerId) throws IOException {
+    public boolean entryExists(long ledgerId, long entryId) throws IOException, BookieException {
+        if (entryId == BookieProtocol.LAST_ADD_CONFIRMED) {
+            return false;
+        }
+
+        // We need to try to read from both write caches, since recent entries could be found in either of the two. The
+        // write caches are already thread safe on their own, here we just need to make sure we get references to both
+        // of them. Using an optimistic lock since the read lock is always free, unless we're swapping the caches.
+        long stamp = writeCacheRotationLock.tryOptimisticRead();
+        WriteCache localWriteCache = writeCache;
+        WriteCache localWriteCacheBeingFlushed = writeCacheBeingFlushed;
+        if (!writeCacheRotationLock.validate(stamp)) {
+            // Fallback to regular read lock approach
+            stamp = writeCacheRotationLock.readLock();
+            try {
+                localWriteCache = writeCache;
+                localWriteCacheBeingFlushed = writeCacheBeingFlushed;
+            } finally {
+                writeCacheRotationLock.unlockRead(stamp);
+            }
+        }
+
+        boolean inCache = localWriteCache.hasEntry(ledgerId, entryId)
+             || localWriteCacheBeingFlushed.hasEntry(ledgerId, entryId)
+             || readCache.hasEntry(ledgerId, entryId);
+
+        if (inCache) {
+            return true;
+        }
+
+        // Read from main storage
+        long entryLocation = entryLocationIndex.getLocation(ledgerId, entryId);
+        if (entryLocation != 0) {
+            return true;
+        }
+
+        // Only a negative result while in limbo equates to unknown
+        throwIfLimbo(ledgerId);
+
+        return false;
+    }
+
+    @Override
+    public boolean isFenced(long ledgerId) throws IOException, BookieException {
         if (log.isDebugEnabled()) {
             log.debug("isFenced. ledger: {}", ledgerId);
         }
-        return ledgerIndex.get(ledgerId).getFenced();
+
+        boolean isFenced = ledgerIndex.get(ledgerId).getFenced();
+
+        // Only a negative result while in limbo equates to unknown
+        if (!isFenced) {
+            throwIfLimbo(ledgerId);
+        }
+
+        return isFenced;
     }
 
     @Override
@@ -416,7 +471,7 @@ public class SingleDirectoryDbLedgerStorage implements CompactableLedgerStorage
     }
 
     @Override
-    public ByteBuf getEntry(long ledgerId, long entryId) throws IOException {
+    public ByteBuf getEntry(long ledgerId, long entryId) throws IOException, BookieException {
         long startTime = MathUtils.nowInNano();
         try {
             ByteBuf entry = doGetEntry(ledgerId, entryId);
@@ -428,7 +483,7 @@ public class SingleDirectoryDbLedgerStorage implements CompactableLedgerStorage
         }
     }
 
-    private ByteBuf doGetEntry(long ledgerId, long entryId) throws IOException {
+    private ByteBuf doGetEntry(long ledgerId, long entryId) throws IOException, BookieException {
         if (log.isDebugEnabled()) {
             log.debug("Get Entry: {}@{}", ledgerId, entryId);
         }
@@ -485,6 +540,9 @@ public class SingleDirectoryDbLedgerStorage implements CompactableLedgerStorage
         try {
             entryLocation = entryLocationIndex.getLocation(ledgerId, entryId);
             if (entryLocation == 0) {
+                // Only a negative result while in limbo equates to unknown
+                throwIfLimbo(ledgerId);
+
                 throw new NoEntryException(ledgerId, entryId);
             }
         } finally {
@@ -556,7 +614,10 @@ public class SingleDirectoryDbLedgerStorage implements CompactableLedgerStorage
         }
     }
 
-    public ByteBuf getLastEntry(long ledgerId) throws IOException {
+    public ByteBuf getLastEntry(long ledgerId) throws IOException, BookieException {
+        throwIfLimbo(ledgerId);
+
+        long startTime = MathUtils.nowInNano();
         long stamp = writeCacheRotationLock.readLock();
         try {
             // First try to read from the write cache of recent entries
@@ -799,7 +860,9 @@ public class SingleDirectoryDbLedgerStorage implements CompactableLedgerStorage
     }
 
     @Override
-    public long getLastAddConfirmed(long ledgerId) throws IOException {
+    public long getLastAddConfirmed(long ledgerId) throws IOException, BookieException {
+        throwIfLimbo(ledgerId);
+
         TransientLedgerInfo ledgerInfo = transientLedgerInfoCache.get(ledgerId);
         long lac = null != ledgerInfo ? ledgerInfo.getLastAddConfirmed() : TransientLedgerInfo.NOT_ASSIGNED_LAC;
         if (lac == TransientLedgerInfo.NOT_ASSIGNED_LAC) {
@@ -837,7 +900,8 @@ public class SingleDirectoryDbLedgerStorage implements CompactableLedgerStorage
     }
 
     @Override
-    public ByteBuf getExplicitLac(long ledgerId) throws IOException {
+    public ByteBuf getExplicitLac(long ledgerId) throws IOException, BookieException {
+        throwIfLimbo(ledgerId);
         if (log.isDebugEnabled()) {
             log.debug("getExplicitLac ledger {}", ledgerId);
         }
@@ -1013,4 +1077,91 @@ public class SingleDirectoryDbLedgerStorage implements CompactableLedgerStorage
             }
         };
     }
+
+    @Override
+    public void setLimboState(long ledgerId) throws IOException {
+        if (log.isDebugEnabled()) {
+            log.debug("setLimboState. ledger: {}", ledgerId);
+        }
+        ledgerIndex.setLimbo(ledgerId);
+    }
+
+    @Override
+    public boolean hasLimboState(long ledgerId) throws IOException {
+        if (log.isDebugEnabled()) {
+            log.debug("hasLimboState. ledger: {}", ledgerId);
+        }
+        return ledgerIndex.get(ledgerId).getLimbo();
+    }
+
+    @Override
+    public void clearLimboState(long ledgerId) throws IOException {
+        if (log.isDebugEnabled()) {
+            log.debug("clearLimboState. ledger: {}", ledgerId);
+        }
+        ledgerIndex.clearLimbo(ledgerId);
+    }
+
+    private void throwIfLimbo(long ledgerId) throws IOException, BookieException {
+        if (hasLimboState(ledgerId)) {
+            if (log.isDebugEnabled()) {
+                log.debug("Accessing ledger({}) in limbo state, throwing exception", ledgerId);
+            }
+            throw BookieException.create(BookieException.Code.DataUnknownException);
+        }
+    }
+
+    /**
+     * Mapping of enums to bitmaps. The bitmaps must not overlap so that we can
+     * do bitwise operations on them.
+     */
+    private static final Map<StorageState, Integer> stateBitmaps = ImmutableMap.of(
+            StorageState.NEEDS_INTEGRITY_CHECK, 0x00000001);
+
+    @Override
+    public EnumSet<StorageState> getStorageStateFlags() throws IOException {
+        int flags = ledgerIndex.getStorageStateFlags();
+        EnumSet<StorageState> flagsEnum = EnumSet.noneOf(StorageState.class);
+        for (Map.Entry<StorageState, Integer> e : stateBitmaps.entrySet()) {
+            int value = e.getValue();
+            if ((flags & value) == value) {
+                flagsEnum.add(e.getKey());
+            }
+            flags = flags & ~value;
+        }
+        checkState(flags == 0, "Unknown storage state flag found " + flags);
+        return flagsEnum;
+    }
+
+    @Override
+    public void setStorageStateFlag(StorageState flag) throws IOException {
+        checkArgument(stateBitmaps.containsKey(flag), "Unsupported flag " + flag);
+        int flagInt = stateBitmaps.get(flag);
+        while (true) {
+            int curFlags = ledgerIndex.getStorageStateFlags();
+            int newFlags = curFlags | flagInt;
+            if (ledgerIndex.setStorageStateFlags(curFlags, newFlags)) {
+                return;
+            } else {
+                log.info("Conflict updating storage state flags {} -> {}, retrying",
+                        curFlags, newFlags);
+            }
+        }
+    }
+
+    @Override
+    public void clearStorageStateFlag(StorageState flag) throws IOException {
+        checkArgument(stateBitmaps.containsKey(flag), "Unsupported flag " + flag);
+        int flagInt = stateBitmaps.get(flag);
+        while (true) {
+            int curFlags = ledgerIndex.getStorageStateFlags();
+            int newFlags = curFlags & ~flagInt;
+            if (ledgerIndex.setStorageStateFlags(curFlags, newFlags)) {
+                return;
+            } else {
+                log.info("Conflict updating storage state flags {} -> {}, retrying",
+                        curFlags, newFlags);
+            }
+        }
+    }
 }
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/WriteCache.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/WriteCache.java
index ac58e8e..66eeb3b 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/WriteCache.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/WriteCache.java
@@ -196,6 +196,10 @@ public class WriteCache implements Closeable {
         return entry;
     }
 
+    public boolean hasEntry(long ledgerId, long entryId) {
+        return index.get(ledgerId, entryId) != null;
+    }
+
     public ByteBuf getLastEntry(long ledgerId) {
         long lastEntryId = lastEntryMap.get(ledgerId);
         if (lastEntryId == -1) {
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BKException.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BKException.java
index 393dde9..7435c67 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BKException.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BKException.java
@@ -468,6 +468,15 @@ public abstract class BKException extends org.apache.bookkeeper.client.api.BKExc
     }
 
     /**
+     * Bookkeeper ledger in limbo and data may or may not exist.
+     */
+    public static class BKDataUnknownException extends BKException {
+        public BKDataUnknownException() {
+            super(BKException.Code.DataUnknownException);
+        }
+    }
+
+    /**
      * Extract an exception code from an BKException, or use a default if it's another type.
      * The throwable is null, assume that no exception took place and return
      * {@link BKException.Code.OK}.
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeper.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeper.java
index fdefd28..1602490 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeper.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeper.java
@@ -1646,7 +1646,7 @@ public class BookKeeper implements org.apache.bookkeeper.client.api.BookKeeper {
             }
         };
 
-    ClientContext getClientCtx() {
+    public ClientContext getClientCtx() {
         return clientCtx;
     }
 }
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ClientContext.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ClientContext.java
index da3abde..aef3e08 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ClientContext.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ClientContext.java
@@ -33,7 +33,7 @@ import org.apache.bookkeeper.proto.BookieClient;
  * but they are present to the LedgerHandle through this interface to allow
  * tests to easily inject mocked versions.
  */
-interface ClientContext {
+public interface ClientContext {
     ClientInternalConf getConf();
     LedgerManager getLedgerManager();
     BookieWatcher getBookieWatcher();
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/api/BKException.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/api/BKException.java
index 0ec0997..396260a 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/api/BKException.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/api/BKException.java
@@ -171,6 +171,8 @@ public class BKException extends Exception {
             return "Failed to establish a secure connection";
         case Code.MetadataSerializationException:
             return "Failed to serialize metadata";
+        case Code.DataUnknownException:
+            return "Ledger in limbo";
         default:
             return "Unexpected condition";
         }
@@ -286,6 +288,11 @@ public class BKException extends Exception {
         int MetadataSerializationException = -107;
 
         /**
+         * Operations failed due to ledger data in an unknown state.
+         */
+        int DataUnknownException = -108;
+
+        /**
          * Generic exception code used to propagate in replication pipeline.
          */
         int ReplicationException = -200;
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 0f6dcc3..8854524 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
@@ -317,6 +317,9 @@ public class ServerConfiguration extends AbstractConfiguration<ServerConfigurati
     // Certificate role based authorization
     protected static final String AUTHORIZED_ROLES = "authorizedRoles";
 
+    protected static final String DATA_INTEGRITY_CHECKING_ENABLED = "dataIntegrityChecking";
+    protected static final String DATA_INTEGRITY_COOKIE_STAMPING_ENABLED = "dataIntegrityStampMissingCookies";
+
     /**
      * Construct a default configuration object.
      */
@@ -3746,4 +3749,53 @@ public class ServerConfiguration extends AbstractConfiguration<ServerConfigurati
         setProperty(REPLICATION_RATE_BY_BYTES, rate);
         return this;
     }
+
+    /**
+     * Enabled data integrity checker.
+     * The data integrity checker checks that the bookie has all the entries which
+     * ledger metadata asserts it has.
+     * The checker runs on startup (periodic will be added later).
+     * The changes how cookies are handled. If a directory is found to be missing a cookie,
+     * the check runs. The check is divided into two parts, preboot and full.
+     * The preboot check ensures that it is safe to boot the bookie; the bookie will not
+     * vote in any operation that contradicts a previous vote.
+     * The full check ensures that any ledger that claims to have entries on the bookie,
+     * truly does have data on the bookie. Any missing entries are copies from available
+     * replicas.
+     */
+    public ServerConfiguration setDataIntegrityCheckingEnabled(boolean enabled) {
+        this.setProperty(DATA_INTEGRITY_CHECKING_ENABLED,
+                         Boolean.toString(enabled));
+        return this;
+    }
+
+    /**
+     * @see #setDataIntegrityCheckingEnabled
+     */
+    public boolean isDataIntegrityCheckingEnabled() {
+        return this.getBoolean(DATA_INTEGRITY_CHECKING_ENABLED, false);
+    }
+
+    /**
+     * When this config is set to true and the data integrity checker is also enabled then
+     * any missing cookie files in the ledger directories do not prevent the bookie from
+     * booting. Missing cookie files usually indicate an empty disk has been mounted, which
+     * might be after a disk failure (all data lost) or a provisioning error (wrong disk mounted).
+     * If there are missing cookie files then:
+     * - a new cookie is stamped (written to each ledger directory and to the co-ordination service, eg: zookeeper).
+     * - the data integrity checker will attempt to repair any lost data by sourcing the lost entries from other bookies
+     * If any cookies do not match the master cookie, then cookie validation still fails as normal.
+     */
+    public ServerConfiguration setDataIntegrityStampMissingCookiesEnabled(boolean enabled) {
+        this.setProperty(DATA_INTEGRITY_COOKIE_STAMPING_ENABLED,
+                Boolean.toString(enabled));
+        return this;
+    }
+
+    /**
+     * @see #setDataIntegrityStampMissingCookiesEnabled
+     */
+    public boolean isDataIntegrityStampMissingCookiesEnabled() {
+        return this.getBoolean(DATA_INTEGRITY_COOKIE_STAMPING_ENABLED, false);
+    }
 }
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieProtocol.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieProtocol.java
index 93e8219..cc290b0 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieProtocol.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieProtocol.java
@@ -180,6 +180,11 @@ public interface BookieProtocol {
      */
     int ETOOMANYREQUESTS = 106;
 
+    /**
+     * Ledger in unknown state.
+     */
+    int EUNKNOWNLEDGERSTATE = 107;
+
     short FLAG_NONE = 0x0;
     short FLAG_DO_FENCING = 0x0001;
     short FLAG_RECOVERY_ADD = 0x0002;
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 d060d86..2c1eb3a 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
@@ -35,6 +35,7 @@ import org.apache.bookkeeper.bookie.BookieCriticalThread;
 import org.apache.bookkeeper.bookie.BookieException;
 import org.apache.bookkeeper.bookie.BookieImpl;
 import org.apache.bookkeeper.bookie.ExitCode;
+import org.apache.bookkeeper.bookie.UncleanShutdownDetection;
 import org.apache.bookkeeper.common.util.JsonUtil.ParseJsonException;
 import org.apache.bookkeeper.conf.ServerConfiguration;
 import org.apache.bookkeeper.net.BookieId;
@@ -61,6 +62,7 @@ public class BookieServer {
     private volatile boolean running = false;
     private final Bookie bookie;
     DeathWatcher deathWatcher;
+    UncleanShutdownDetection uncleanShutdownDetection;
     private static final Logger LOG = LoggerFactory.getLogger(BookieServer.class);
 
     int exitCode = ExitCode.OK;
@@ -77,7 +79,8 @@ public class BookieServer {
     public BookieServer(ServerConfiguration conf,
                         Bookie bookie,
                         StatsLogger statsLogger,
-                        ByteBufAllocator allocator)
+                        ByteBufAllocator allocator,
+                        UncleanShutdownDetection uncleanShutdownDetection)
             throws IOException, KeeperException, InterruptedException,
             BookieException, UnavailableException, CompatibilityException, SecurityException {
         this.conf = conf;
@@ -93,6 +96,7 @@ public class BookieServer {
         this.statsLogger = statsLogger;
         this.bookie = bookie;
         this.nettyServer = new BookieNettyServer(this.conf, null, allocator);
+        this.uncleanShutdownDetection = uncleanShutdownDetection;
 
         final SecurityHandlerFactory shFactory;
 
@@ -115,14 +119,17 @@ public class BookieServer {
         this.uncaughtExceptionHandler = exceptionHandler;
     }
 
-    public void start() throws InterruptedException {
+    public void start() throws InterruptedException, IOException {
         this.bookie.start();
+
         // fail fast, when bookie startup is not successful
         if (!this.bookie.isRunning()) {
             exitCode = bookie.getExitCode();
             this.requestProcessor.close();
             return;
         }
+
+        this.uncleanShutdownDetection.registerStartUp();
         this.nettyServer.start();
 
         running = true;
@@ -187,6 +194,7 @@ public class BookieServer {
         }
         this.requestProcessor.close();
         exitCode = bookie.shutdown();
+        uncleanShutdownDetection.registerCleanShutdown();
         running = false;
     }
 
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/LongPollReadEntryProcessorV3.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/LongPollReadEntryProcessorV3.java
index d9ab2e6..7c18e7a 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/LongPollReadEntryProcessorV3.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/LongPollReadEntryProcessorV3.java
@@ -28,6 +28,7 @@ import java.util.concurrent.Future;
 import java.util.concurrent.RejectedExecutionException;
 import java.util.concurrent.TimeUnit;
 import org.apache.bookkeeper.bookie.Bookie;
+import org.apache.bookkeeper.bookie.BookieException;
 import org.apache.bookkeeper.bookie.LastAddConfirmedUpdateNotification;
 import org.apache.bookkeeper.common.util.Watcher;
 import org.apache.bookkeeper.proto.BookkeeperProtocol.ReadResponse;
@@ -79,7 +80,7 @@ class LongPollReadEntryProcessorV3 extends ReadEntryProcessorV3 implements Watch
     protected ReadResponse readEntry(ReadResponse.Builder readResponseBuilder,
                                      long entryId,
                                      Stopwatch startTimeSw)
-            throws IOException {
+            throws IOException, BookieException {
         if (RequestUtils.shouldPiggybackEntry(readRequest)) {
             if (!readRequest.hasPreviousLAC() || (BookieProtocol.LAST_ADD_CONFIRMED != entryId)) {
                 // This is not a valid request - client bug?
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClient.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClient.java
index 7346d19..19ad306 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClient.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/PerChannelBookieClient.java
@@ -2288,6 +2288,8 @@ public class PerChannelBookieClient extends ChannelInboundHandlerAdapter {
                 return BKException.Code.WriteOnReadOnlyBookieException;
             case ETOOMANYREQUESTS:
                 return BKException.Code.TooManyRequestsException;
+            case EUNKNOWNLEDGERSTATE:
+                return BKException.Code.DataUnknownException;
             default:
                 return BKException.Code.UNINITIALIZED;
         }
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadEntryProcessor.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadEntryProcessor.java
index c8313bb..3c7d2fa 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadEntryProcessor.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadEntryProcessor.java
@@ -99,6 +99,9 @@ class ReadEntryProcessor extends PacketProcessorBase<ReadRequest> {
                 LOG.debug("Error reading {}", request, e);
             }
             errorCode = BookieProtocol.EIO;
+        } catch (BookieException.DataUnknownException e) {
+            LOG.error("Ledger {} is in an unknown state", request.getLedgerId(), e);
+            errorCode = BookieProtocol.EUNKNOWNLEDGERSTATE;
         } catch (BookieException e) {
             LOG.error("Unauthorized access to ledger {}", request.getLedgerId(), e);
             errorCode = BookieProtocol.EUA;
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadEntryProcessorV3.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadEntryProcessorV3.java
index a563cc0..e03cb35 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadEntryProcessorV3.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadEntryProcessorV3.java
@@ -149,7 +149,7 @@ class ReadEntryProcessorV3 extends PacketProcessorBaseV3 {
     protected ReadResponse readEntry(ReadResponse.Builder readResponseBuilder,
                                      long entryId,
                                      Stopwatch startTimeSw)
-        throws IOException {
+        throws IOException, BookieException {
         return readEntry(readResponseBuilder, entryId, false, startTimeSw);
     }
 
@@ -169,7 +169,7 @@ class ReadEntryProcessorV3 extends PacketProcessorBaseV3 {
                                      long entryId,
                                      boolean readLACPiggyBack,
                                      Stopwatch startTimeSw)
-        throws IOException {
+        throws IOException, BookieException {
         ByteBuf entryBody = requestProcessor.getBookie().readEntry(ledgerId, entryId);
         if (null != fenceResult) {
             handleReadResultForFenceRead(entryBody, readResponseBuilder, entryId, startTimeSw);
@@ -232,6 +232,11 @@ class ReadEntryProcessorV3 extends PacketProcessorBaseV3 {
         } catch (IOException e) {
             LOG.error("IOException while reading entry: {} from ledger {} ", entryId, ledgerId, e);
             return buildResponse(readResponse, StatusCode.EIO, startTimeSw);
+        } catch (BookieException.DataUnknownException e) {
+            if (LOG.isDebugEnabled()) {
+                LOG.debug("Ledger has unknown state for entry: {} from ledger {}", entryId, ledgerId);
+            }
+            return buildResponse(readResponse, StatusCode.EUNKNOWNLEDGERSTATE, startTimeSw);
         } catch (BookieException e) {
             LOG.error(
                 "Unauthorized access to ledger:{} while reading entry:{} in request from address: {}",
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadLacProcessorV3.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadLacProcessorV3.java
index 374bfe2..16f2471 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadLacProcessorV3.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/ReadLacProcessorV3.java
@@ -30,6 +30,7 @@ import java.io.IOException;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.bookkeeper.bookie.Bookie;
+import org.apache.bookkeeper.bookie.BookieException;
 import org.apache.bookkeeper.proto.BookkeeperProtocol.ReadLacRequest;
 import org.apache.bookkeeper.proto.BookkeeperProtocol.ReadLacResponse;
 import org.apache.bookkeeper.proto.BookkeeperProtocol.Request;
@@ -74,10 +75,13 @@ class ReadLacProcessorV3 extends PacketProcessorBaseV3 implements Runnable {
             }
         } catch (Bookie.NoLedgerException e) {
             status = StatusCode.ENOLEDGER;
-            logger.warn("No ledger found while performing readLac from ledger: {}", ledgerId, e);
-        } catch (IOException e) {
+            logger.error("No ledger found while performing readLac from ledger: {}", ledgerId, e);
+        } catch (BookieException.DataUnknownException e) {
+            status = StatusCode.EUNKNOWNLEDGERSTATE;
+            logger.error("Ledger {} in unknown state and cannot serve reacLac requests", ledgerId, e);
+        } catch (BookieException | IOException e) {
             status = StatusCode.EIO;
-            logger.error("IOException while performing readLac from ledger: {}", ledgerId);
+            logger.error("IOException while performing readLac from ledger: {}", ledgerId, e);
         } finally {
             ReferenceCountUtil.release(lac);
         }
@@ -93,7 +97,10 @@ class ReadLacProcessorV3 extends PacketProcessorBaseV3 implements Runnable {
         } catch (Bookie.NoEntryException e) {
             status = StatusCode.ENOENTRY;
             logger.warn("No Entry found while trying to read last entry: {}", ledgerId, e);
-        } catch (IOException e) {
+        } catch (BookieException.DataUnknownException e) {
+            status = StatusCode.EUNKNOWNLEDGERSTATE;
+            logger.error("Ledger in an unknown state while trying to read last entry: {}", ledgerId, e);
+        } catch (BookieException | IOException e) {
             status = StatusCode.EIO;
             logger.error("IOException while trying to read last entry: {}", ledgerId, e);
         } finally {
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/Main.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/Main.java
index 15677d7..3c7d7fe 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/Main.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/Main.java
@@ -18,12 +18,18 @@
 
 package org.apache.bookkeeper.server;
 
+import static com.google.common.base.Preconditions.checkNotNull;
 import static org.apache.bookkeeper.bookie.BookKeeperServerStats.BOOKIE_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.client.BookKeeperClientStats.CLIENT_SCOPE;
 import static org.apache.bookkeeper.replication.ReplicationStats.REPLICATION_SCOPE;
 import static org.apache.bookkeeper.server.component.ServerLifecycleComponent.loadServerComponents;
 
+import com.google.common.base.Ticker;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import io.reactivex.rxjava3.core.Scheduler;
+import io.reactivex.rxjava3.schedulers.Schedulers;
 import java.io.File;
 import java.io.IOException;
 import java.net.MalformedURLException;
@@ -31,6 +37,8 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
 import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
 import java.util.function.Supplier;
 import java.util.stream.Collectors;
 import lombok.extern.slf4j.Slf4j;
@@ -44,12 +52,24 @@ import org.apache.bookkeeper.bookie.LedgerStorage;
 import org.apache.bookkeeper.bookie.LegacyCookieValidation;
 import org.apache.bookkeeper.bookie.ReadOnlyBookie;
 import org.apache.bookkeeper.bookie.ScrubberStats;
+import org.apache.bookkeeper.bookie.UncleanShutdownDetection;
+import org.apache.bookkeeper.bookie.UncleanShutdownDetectionImpl;
+import org.apache.bookkeeper.bookie.datainteg.DataIntegrityCheck;
+import org.apache.bookkeeper.bookie.datainteg.DataIntegrityCheckImpl;
+import org.apache.bookkeeper.bookie.datainteg.DataIntegrityCookieValidation;
+import org.apache.bookkeeper.bookie.datainteg.DataIntegrityService;
+import org.apache.bookkeeper.bookie.datainteg.EntryCopier;
+import org.apache.bookkeeper.bookie.datainteg.EntryCopierImpl;
+import org.apache.bookkeeper.client.BookKeeper;
+import org.apache.bookkeeper.client.BookKeeperAdmin;
 import org.apache.bookkeeper.common.allocator.ByteBufAllocatorWithOomHandler;
 import org.apache.bookkeeper.common.component.AutoCloseableLifecycleComponent;
 import org.apache.bookkeeper.common.component.ComponentInfoPublisher;
 import org.apache.bookkeeper.common.component.ComponentStarter;
 import org.apache.bookkeeper.common.component.LifecycleComponent;
 import org.apache.bookkeeper.common.component.LifecycleComponentStack;
+import org.apache.bookkeeper.common.component.RxSchedulerLifecycleComponent;
+import org.apache.bookkeeper.conf.ClientConfiguration;
 import org.apache.bookkeeper.conf.ServerConfiguration;
 import org.apache.bookkeeper.conf.UncheckedConfigurationException;
 import org.apache.bookkeeper.discover.BookieServiceInfo;
@@ -58,6 +78,7 @@ import org.apache.bookkeeper.discover.RegistrationManager;
 import org.apache.bookkeeper.meta.LedgerManager;
 import org.apache.bookkeeper.meta.LedgerManagerFactory;
 import org.apache.bookkeeper.meta.MetadataBookieDriver;
+import org.apache.bookkeeper.net.BookieId;
 import org.apache.bookkeeper.server.component.ServerLifecycleComponent;
 import org.apache.bookkeeper.server.conf.BookieConfiguration;
 import org.apache.bookkeeper.server.http.BKHttpServiceProvider;
@@ -85,7 +106,6 @@ import org.apache.commons.lang3.StringUtils;
  */
 @Slf4j
 public class Main {
-
     static final Options BK_OPTS = new Options();
     static {
         BK_OPTS.addOption("c", "conf", true, "Configuration for Bookie Server");
@@ -229,7 +249,6 @@ public class Main {
     }
 
     static int doMain(String[] args) {
-
         ServerConfiguration conf;
 
         // 0. parse command line
@@ -340,14 +359,62 @@ public class Main {
         LedgerDirsManager indexDirsManager = BookieResources.createIndexDirsManager(
                 conf.getServerConf(), diskChecker, bookieStats.scope(LD_INDEX_SCOPE), ledgerDirsManager);
 
-        CookieValidation cookieValidation = new LegacyCookieValidation(conf.getServerConf(), rm);
-        cookieValidation.checkCookies(storageDirectoriesFromConf(conf.getServerConf()));
-
         ByteBufAllocatorWithOomHandler allocator = BookieResources.createAllocator(conf.getServerConf());
 
+        UncleanShutdownDetection uncleanShutdownDetection = new UncleanShutdownDetectionImpl(ledgerDirsManager);
+        if (uncleanShutdownDetection.lastShutdownWasUnclean()) {
+            log.info("Unclean shutdown detected. The bookie did not register a graceful shutdown prior to this boot.");
+        }
+
         // bookie takes ownership of storage, so shuts it down
-        LedgerStorage storage = BookieResources.createLedgerStorage(
-                conf.getServerConf(), ledgerManager, ledgerDirsManager, indexDirsManager, bookieStats, allocator);
+        LedgerStorage storage = null;
+        DataIntegrityCheck integCheck = null;
+
+        if (conf.getServerConf().isDataIntegrityCheckingEnabled()) {
+            StatsLogger clientStats = bookieStats.scope(CLIENT_SCOPE);
+            ClientConfiguration clientConfiguration = new ClientConfiguration(conf.getServerConf());
+            clientConfiguration.setClientRole(ClientConfiguration.CLIENT_ROLE_SYSTEM);
+            BookKeeper bkc = BookKeeper.forConfig(clientConfiguration).statsLogger(clientStats).build();
+            serverBuilder.addComponent(new AutoCloseableLifecycleComponent("bkc", bkc));
+
+            BookieId bookieId = BookieImpl.getBookieId(conf.getServerConf());
+            ExecutorService rxExecutor = Executors.newFixedThreadPool(
+                    2, new ThreadFactoryBuilder().setNameFormat("rx-schedule-%d")
+                    .setUncaughtExceptionHandler(
+                            (t, ex) -> log.error("Uncaught exception on thread {}", t.getName(), ex))
+                    .build());
+            Scheduler rxScheduler = Schedulers.from(rxExecutor);
+            serverBuilder.addComponent(
+                    new RxSchedulerLifecycleComponent("rx-scheduler", conf, bookieStats,
+                            rxScheduler, rxExecutor));
+
+            storage = BookieResources.createLedgerStorage(
+                    conf.getServerConf(), ledgerManager, ledgerDirsManager, indexDirsManager, bookieStats, allocator);
+
+            EntryCopier copier = new EntryCopierImpl(bookieId,
+                    ((org.apache.bookkeeper.client.BookKeeper) bkc).getClientCtx().getBookieClient(),
+                    storage, Ticker.systemTicker());
+
+            integCheck = new DataIntegrityCheckImpl(bookieId,
+                                                    ledgerManager, storage, copier,
+                                                    new BookKeeperAdmin(bkc, clientStats, clientConfiguration),
+                                                    rxScheduler);
+
+            // if we're running with journal writes disabled and an unclean shutdown occurred then
+            // run the preboot check to protect against data loss and to perform data repair
+            if (!conf.getServerConf().getJournalWriteData()
+                    && uncleanShutdownDetection.lastShutdownWasUnclean()) {
+                integCheck.runPreBootCheck("UNCLEAN_SHUTDOWN");
+            }
+            CookieValidation cookieValidation = new DataIntegrityCookieValidation(conf.getServerConf(),
+                                                                 rm, integCheck);
+            cookieValidation.checkCookies(storageDirectoriesFromConf(conf.getServerConf()));
+        } else {
+            CookieValidation cookieValidation = new LegacyCookieValidation(conf.getServerConf(), rm);
+            cookieValidation.checkCookies(storageDirectoriesFromConf(conf.getServerConf()));
+            storage = BookieResources.createLedgerStorage(
+                    conf.getServerConf(), ledgerManager, ledgerDirsManager, indexDirsManager, bookieStats, allocator);
+        }
 
         Bookie bookie;
         if (conf.getServerConf().isForceReadOnlyBookie()) {
@@ -366,7 +433,7 @@ public class Main {
 
         // 5. build bookie server
         BookieService bookieService =
-            new BookieService(conf, bookie, rootStatsLogger, allocator);
+            new BookieService(conf, bookie, rootStatsLogger, allocator, uncleanShutdownDetection);
 
         serverBuilder.addComponent(bookieService);
         log.info("Load lifecycle component : {}", BookieService.class.getName());
@@ -387,7 +454,18 @@ public class Main {
             log.info("Load lifecycle component : {}", AutoRecoveryService.class.getName());
         }
 
-        // 7. build http service
+        // 7. build data integrity check service
+        if (conf.getServerConf().isDataIntegrityCheckingEnabled()) {
+            checkNotNull(integCheck,
+                    "integCheck should have been initialized with the cookie validation");
+            DataIntegrityService dataIntegrityService =
+                    new DataIntegrityService(conf, rootStatsLogger.scope(REPLICATION_SCOPE),
+                    integCheck);
+            serverBuilder.addComponent(dataIntegrityService);
+            log.info("Load lifecycle component : {}", DataIntegrityService.class.getName());
+        }
+
+        // 8. build http service
         if (conf.getServerConf().isHttpServerEnabled()) {
             BKHttpServiceProvider provider = new BKHttpServiceProvider.Builder()
                 .setBookieServer(bookieService.getServer())
@@ -401,7 +479,7 @@ public class Main {
             log.info("Load lifecycle component : {}", HttpService.class.getName());
         }
 
-        // 8. build extra services
+        // 9. build extra services
         String[] extraComponents = conf.getServerConf().getExtraServerComponents();
         if (null != extraComponents) {
             try {
@@ -455,8 +533,10 @@ public class Main {
         if (journalDirs != null) {
             for (File j : journalDirs) {
                 File cur = BookieImpl.getCurrentDirectory(j);
-                BookieImpl.checkDirectoryStructure(cur);
-                dirs.add(cur);
+                if (!dirs.stream().anyMatch(f -> f.equals(cur))) {
+                    BookieImpl.checkDirectoryStructure(cur);
+                    dirs.add(cur);
+                }
             }
         }
 
@@ -464,16 +544,20 @@ public class Main {
         if (ledgerDirs != null) {
             for (File l : ledgerDirs) {
                 File cur = BookieImpl.getCurrentDirectory(l);
-                BookieImpl.checkDirectoryStructure(cur);
-                dirs.add(cur);
+                if (!dirs.stream().anyMatch(f -> f.equals(cur))) {
+                    BookieImpl.checkDirectoryStructure(cur);
+                    dirs.add(cur);
+                }
             }
         }
         File[] indexDirs = conf.getIndexDirs();
         if (indexDirs != null) {
             for (File i : indexDirs) {
                 File cur = BookieImpl.getCurrentDirectory(i);
-                BookieImpl.checkDirectoryStructure(cur);
-                dirs.add(cur);
+                if (!dirs.stream().anyMatch(f -> f.equals(cur))) {
+                    BookieImpl.checkDirectoryStructure(cur);
+                    dirs.add(cur);
+                }
             }
         }
         return dirs;
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/service/BookieService.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/service/BookieService.java
index 641bf6f..67943dd 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/service/BookieService.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/server/service/BookieService.java
@@ -24,6 +24,7 @@ import java.net.UnknownHostException;
 import java.util.ArrayList;
 import java.util.List;
 import org.apache.bookkeeper.bookie.Bookie;
+import org.apache.bookkeeper.bookie.UncleanShutdownDetection;
 import org.apache.bookkeeper.common.allocator.ByteBufAllocatorWithOomHandler;
 import org.apache.bookkeeper.common.component.ComponentInfoPublisher;
 import org.apache.bookkeeper.common.component.ComponentInfoPublisher.EndpointInfo;
@@ -50,13 +51,15 @@ public class BookieService extends ServerLifecycleComponent {
     public BookieService(BookieConfiguration conf,
                          Bookie bookie,
                          StatsLogger statsLogger,
-                         ByteBufAllocatorWithOomHandler allocator)
+                         ByteBufAllocatorWithOomHandler allocator,
+                         UncleanShutdownDetection uncleanShutdownDetection)
             throws Exception {
         super(NAME, conf, statsLogger);
         this.server = new BookieServer(conf.getServerConf(),
                                        bookie,
                                        statsLogger,
-                                       allocator);
+                                       allocator,
+                                       uncleanShutdownDetection);
         this.allocator = allocator;
     }
 
@@ -83,7 +86,7 @@ public class BookieService extends ServerLifecycleComponent {
     protected void doStart() {
         try {
             this.server.start();
-        } catch (InterruptedException exc) {
+        } catch (InterruptedException | IOException exc) {
             throw new RuntimeException("Failed to start bookie server", exc);
         }
     }
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/LocalBookKeeper.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/LocalBookKeeper.java
index 3982c80..b7d514d 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/LocalBookKeeper.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/LocalBookKeeper.java
@@ -41,6 +41,8 @@ import org.apache.bookkeeper.bookie.BookieImpl;
 import org.apache.bookkeeper.bookie.BookieResources;
 import org.apache.bookkeeper.bookie.LedgerDirsManager;
 import org.apache.bookkeeper.bookie.LedgerStorage;
+import org.apache.bookkeeper.bookie.UncleanShutdownDetection;
+import org.apache.bookkeeper.bookie.UncleanShutdownDetectionImpl;
 import org.apache.bookkeeper.common.allocator.ByteBufAllocatorWithOomHandler;
 import org.apache.bookkeeper.common.component.ComponentInfoPublisher;
 import org.apache.bookkeeper.conf.ServerConfiguration;
@@ -524,6 +526,7 @@ public class LocalBookKeeper implements AutoCloseable {
             LedgerStorage storage = BookieResources.createLedgerStorage(
                     conf, ledgerManager, ledgerDirsManager, indexDirsManager,
                     NullStatsLogger.INSTANCE, allocator);
+            UncleanShutdownDetection shutdownManager = new UncleanShutdownDetectionImpl(ledgerDirsManager);
 
             final ComponentInfoPublisher componentInfoPublisher = new ComponentInfoPublisher();
             final Supplier<BookieServiceInfo> bookieServiceInfoProvider =
@@ -531,9 +534,10 @@ public class LocalBookKeeper implements AutoCloseable {
 
             componentInfoPublisher.startupFinished();
             bookie = new BookieImpl(conf, registrationManager, storage, diskChecker,
-                    ledgerDirsManager, indexDirsManager,
-                    NullStatsLogger.INSTANCE, allocator, bookieServiceInfoProvider);
-            server = new BookieServer(conf, bookie, NullStatsLogger.INSTANCE, allocator);
+                                    ledgerDirsManager, indexDirsManager,
+                                    NullStatsLogger.INSTANCE, allocator, bookieServiceInfoProvider);
+            server = new BookieServer(conf, bookie, NullStatsLogger.INSTANCE, allocator,
+                                      shutdownManager);
         }
 
         void start() throws Exception {
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 e94027b..d919bc8 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
@@ -223,7 +223,8 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
         server = new BookieServer(
                 conf,
                 TestBookieImpl.buildReadOnly(conf),
-                NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
+                NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT,
+                new MockUncleanShutdownDetection());
 
         for (int i = 0; i < journalDirs.length; i++) {
             Journal journal = ((BookieImpl) server.getBookie()).journals.get(i);
@@ -267,7 +268,8 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
             server = new BookieServer(
                     conf,
                     TestBookieImpl.buildReadOnly(conf),
-                    NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
+                    NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT,
+                    new MockUncleanShutdownDetection());
         }
         bkClient.close();
     }
@@ -298,7 +300,8 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
             .withMetadataDriver(metadataDriver)
             .withRegistrationManager(rm).build();
         BookieServer bkServer = new BookieServer(conf, new TestBookieImpl(resources),
-                                                 NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
+                                                 NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT,
+                                                 new MockUncleanShutdownDetection());
 
         bkServer.start();
         bkServer.join();
@@ -572,7 +575,8 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
         try {
             bs1 = new BookieServer(
                     conf, new TestBookieImpl(conf),
-                    NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
+                    NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT,
+                    new MockUncleanShutdownDetection());
 
             fail("Bookkeeper should not have started since current user isn't in permittedStartupUsers");
         } catch (AccessControlException buae) {
@@ -608,7 +612,8 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
         try {
             bs1 = new BookieServer(
                     conf, new TestBookieImpl(conf),
-                    NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
+                    NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT,
+                    new MockUncleanShutdownDetection());
             bs1.start();
         } catch (AccessControlException buae) {
             fail("Bookkeeper should have started since current user is in permittedStartupUsers");
@@ -624,7 +629,8 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
         try {
             bs1 = new BookieServer(
                     conf, new TestBookieImpl(conf),
-                    NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
+                    NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT,
+                    new MockUncleanShutdownDetection());
             bs1.start();
         } catch (AccessControlException buae) {
             fail("Bookkeeper should have started since current user is in permittedStartupUsers");
@@ -655,7 +661,8 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
         try {
             bs1 = new BookieServer(
                     conf, new TestBookieImpl(conf),
-                    NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
+                    NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT,
+                    new MockUncleanShutdownDetection());
             bs1.start();
         } catch (AccessControlException buae) {
             fail("Bookkeeper should have started since permittedStartupUser is not specified");
@@ -682,14 +689,16 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
             .setMetadataServiceUri(metadataServiceUri);
         BookieServer bs1 = new BookieServer(
                 conf, new TestBookieImpl(conf),
-                NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
+                NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT,
+                new MockUncleanShutdownDetection());
         bs1.start();
         BookieServer bs2 = null;
         // starting bk server with same conf
         try {
             bs2 = new BookieServer(
                     conf, new TestBookieImpl(conf),
-                    NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
+                    NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT,
+                    new MockUncleanShutdownDetection());
             bs2.start();
             fail("Should throw BindException, as the bk server is already running!");
         } catch (BindException e) {
@@ -718,8 +727,8 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
         BookieConfiguration bkConf = new BookieConfiguration(conf);
         BookieService service = new BookieService(
                 bkConf, new TestBookieImpl(conf),
-                NullStatsLogger.INSTANCE,
-                BookieResources.createAllocator(conf));
+                NullStatsLogger.INSTANCE, BookieResources.createAllocator(conf),
+                new MockUncleanShutdownDetection());
         CompletableFuture<Void> startFuture = ComponentStarter.startComponent(service);
 
         // shutdown the bookie service
@@ -779,7 +788,8 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
             /*
              * create cookie and write it to JournalDir/LedgerDir.
              */
-            Cookie.Builder cookieBuilder = Cookie.generateCookie(conf);
+            String instanceId = rm.getClusterInstanceId();
+            Cookie.Builder cookieBuilder = Cookie.generateCookie(conf).setInstanceId(instanceId);
             Cookie cookie = cookieBuilder.build();
             cookie.writeToDirectory(new File(journalDir, "current"));
             cookie.writeToDirectory(new File(ledgerDir, "current"));
@@ -882,7 +892,8 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
         assertEquals(0, conf1.getBookiePort());
         BookieServer bs1 = new BookieServer(
                 conf1, new TestBookieImpl(conf1),
-                NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
+                NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT,
+                new MockUncleanShutdownDetection());
         bs1.start();
         assertFalse(0 == conf1.getBookiePort());
 
@@ -895,7 +906,8 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
             .setMetadataServiceUri(null);
         BookieServer bs2 = new BookieServer(
                 conf2, new TestBookieImpl(conf2),
-                NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
+                NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT,
+                new MockUncleanShutdownDetection());
 
         bs2.start();
         assertFalse(0 == conf2.getBookiePort());
@@ -1039,14 +1051,43 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
         bk.shutdown();
     }
 
+    @Test
+    public void testStartUpRegisteredWithUncleanShutdownDetection() throws Exception {
+        MockUncleanShutdownDetection uncleanShutdownDetection = new MockUncleanShutdownDetection();
+        final ServerConfiguration conf = newServerConfiguration();
+        BookieServer server = new MockBookieServer(conf, uncleanShutdownDetection);
+        server.start();
+        assertTrue(uncleanShutdownDetection.getStartRegistered());
+        server.shutdown();
+    }
+
+    @Test
+    public void testShutdownRegisteredWithUncleanShutdownDetection() throws Exception {
+        MockUncleanShutdownDetection uncleanShutdownDetection = new MockUncleanShutdownDetection();
+        final ServerConfiguration conf = newServerConfiguration();
+        BookieServer server = new MockBookieServer(conf, uncleanShutdownDetection);
+        server.start();
+        server.shutdown();
+        assertTrue(uncleanShutdownDetection.getShutdownRegistered());
+    }
+
     class MockBookieServer extends BookieServer {
         ServerConfiguration conf;
 
         public MockBookieServer(ServerConfiguration conf) throws Exception {
             super(conf,
                   new MockBookieWithNoopShutdown(conf),
-                  NullStatsLogger.INSTANCE,
-                  UnpooledByteBufAllocator.DEFAULT);
+                  NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT,
+                  new MockUncleanShutdownDetection());
+            this.conf = conf;
+        }
+
+        public MockBookieServer(ServerConfiguration conf,
+                                MockUncleanShutdownDetection uncleanShutdownDetection) throws Exception {
+            super(conf,
+                    new MockBookieWithNoopShutdown(conf),
+                    NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT,
+                    uncleanShutdownDetection);
             this.conf = conf;
         }
     }
@@ -1103,7 +1144,8 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
                 .setMinUsableSizeForIndexFileCreation(Long.MAX_VALUE);
         server = new BookieServer(
                 conf, new TestBookieImpl(conf),
-                NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
+                NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT,
+                new MockUncleanShutdownDetection());
 
         // Now we are trying to start the Bookie, which tries to replay the
         // Journal. While replaying the Journal it tries to create the IndexFile
@@ -1121,7 +1163,8 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
         conf.setMinUsableSizeForIndexFileCreation(1 * 1024);
         server = new BookieServer(
                 conf, new TestBookieImpl(conf),
-                NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
+                NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT,
+                new MockUncleanShutdownDetection());
 
         server.start();
         Thread.sleep((conf.getDiskCheckInterval() * 2) + 1000);
@@ -1181,7 +1224,8 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
         try {
             bs1 = new BookieServer(
                     conf, new TestBookieImpl(conf),
-                    NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
+                    NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT,
+                    new MockUncleanShutdownDetection());
             fail("Bookkeeper should not have started since AllowMultipleDirsUnderSameDiskPartition is not enabled");
         } catch (DiskPartitionDuplicationException dpde) {
             // Expected
@@ -1206,7 +1250,8 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
         try {
             bs1 = new BookieServer(
                     conf, new TestBookieImpl(conf),
-                    NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
+                    NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT,
+                    new MockUncleanShutdownDetection());
             fail("Bookkeeper should not have started since AllowMultipleDirsUnderSameDiskPartition is not enabled");
         } catch (DiskPartitionDuplicationException dpde) {
             // Expected
@@ -1231,7 +1276,8 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
         try {
             bs1 = new BookieServer(
                     conf, new TestBookieImpl(conf),
-                    NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
+                    NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT,
+                    new MockUncleanShutdownDetection());
             fail("Bookkeeper should not have started since AllowMultipleDirsUnderSameDiskPartition is not enabled");
         } catch (DiskPartitionDuplicationException dpde) {
             // Expected
@@ -1268,7 +1314,8 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
         try {
             bs1 = new BookieServer(
                     conf, new TestBookieImpl(conf),
-                    NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
+                    NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT,
+                    new MockUncleanShutdownDetection());
 
         } catch (DiskPartitionDuplicationException dpde) {
             fail("Bookkeeper should have started since AllowMultipleDirsUnderSameDiskPartition is enabled");
@@ -1302,7 +1349,8 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
             .setMetadataServiceUri(metadataServiceUri);
         BookieServer bookieServer = new BookieServer(
                 conf, new TestBookieImpl(conf),
-                NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
+                NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT,
+                new MockUncleanShutdownDetection());
         bookieServer.start();
         Bookie bookie = bookieServer.getBookie();
         assertFalse(bookie.isReadOnly());
@@ -1314,7 +1362,8 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
         bookieServer.shutdown();
         bookieServer = new BookieServer(
                 conf, new TestBookieImpl(conf),
-                NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
+                NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT,
+                new MockUncleanShutdownDetection());
         bookieServer.start();
         bookie = bookieServer.getBookie();
         assertTrue(bookie.isReadOnly());
@@ -1324,7 +1373,8 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
         bookieServer.shutdown();
         bookieServer = new BookieServer(
                 conf, new TestBookieImpl(conf),
-                NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
+                NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT,
+                new MockUncleanShutdownDetection());
         bookieServer.start();
         bookie = bookieServer.getBookie();
         assertFalse(bookie.isReadOnly());
@@ -1346,7 +1396,8 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
         // start new bookie
         BookieServer bookieServer = new BookieServer(
                 conf, new TestBookieImpl(conf),
-                NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
+                NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT,
+                new MockUncleanShutdownDetection());
         bookieServer.start();
         Bookie bookie = bookieServer.getBookie();
         // persist bookie status
@@ -1362,7 +1413,8 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
         bookieServer = new BookieServer(
                 readOnlyConf,
                 TestBookieImpl.buildReadOnly(readOnlyConf),
-                NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
+                NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT,
+                new MockUncleanShutdownDetection());
 
         bookieServer.start();
         bookie = bookieServer.getBookie();
@@ -1394,7 +1446,8 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
         // start a new bookie
         BookieServer bookieServer = new BookieServer(
                 conf, new TestBookieImpl(conf),
-                NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
+                NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT,
+                new MockUncleanShutdownDetection());
         bookieServer.start();
         // transition in to read only and persist the status on disk
         Bookie bookie = (BookieImpl) bookieServer.getBookie();
@@ -1409,7 +1462,8 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
         bookieServer.shutdown();
         bookieServer = new BookieServer(
                 conf, new TestBookieImpl(conf),
-                NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
+                NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT,
+                new MockUncleanShutdownDetection());
         bookieServer.start();
         bookie = bookieServer.getBookie();
         assertTrue(bookie.isReadOnly());
@@ -1437,7 +1491,8 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
         // start a new bookie
         BookieServer bookieServer = new BookieServer(
                 conf, new TestBookieImpl(conf),
-                NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
+                NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT,
+                new MockUncleanShutdownDetection());
         bookieServer.start();
         // transition in to read only and persist the status on disk
         Bookie bookie = (BookieImpl) bookieServer.getBookie();
@@ -1454,7 +1509,8 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
         bookieServer.shutdown();
         bookieServer = new BookieServer(
                 conf, new TestBookieImpl(conf),
-                NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
+                NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT,
+                new MockUncleanShutdownDetection());
         bookieServer.start();
         bookie = bookieServer.getBookie();
         assertFalse(bookie.isReadOnly());
@@ -1646,7 +1702,8 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
             conf.setMetadataServiceUri("//ledgers");
             try {
                 new BookieServer(conf, new TestBookieImpl(conf),
-                        NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
+                        NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT,
+                        new MockUncleanShutdownDetection());
                 fail("Bookie metadata initialization must fail with metadata service uri: //ledgers");
             } catch (NullPointerException e) {
                 assertTrue(e.getMessage().contains("Invalid metadata service uri : //ledgers"));
@@ -1658,7 +1715,8 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
             conf.setMetadataServiceUri("");
             try {
                 new BookieServer(conf, new TestBookieImpl(conf),
-                        NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
+                        NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT,
+                        new MockUncleanShutdownDetection());
                 fail("Bookie metadata initialization must fail with empty metadata service uri");
             } catch (NullPointerException e) {
                 assertTrue(e.getMessage().contains("Invalid metadata service uri :"));
@@ -1671,7 +1729,8 @@ public class BookieInitializationTest extends BookKeeperClusterTestCase {
         conf.setMetadataServiceUri(uri);
         try {
             new BookieServer(conf, new TestBookieImpl(conf),
-                    NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
+                    NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT,
+                    new MockUncleanShutdownDetection());
             fail("Bookie metadata initialization must fail with an invalid metadata service uri: " + uri);
         } catch (MetadataStoreException e) {
             // ok
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/EnableZkSecurityBasicTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/EnableZkSecurityBasicTest.java
index 5558746..6bdae88 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/EnableZkSecurityBasicTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/EnableZkSecurityBasicTest.java
@@ -118,7 +118,8 @@ public class EnableZkSecurityBasicTest extends BookKeeperClusterTestCase {
 
             if (!fullPath.startsWith("/zookeeper") // skip zookeeper internal nodes
                 && !fullPath.equals("/ledgers") // node created by test setup
-                && !fullPath.equals("/ledgers/" + BookKeeperConstants.AVAILABLE_NODE) // node created by test setup
+                && !fullPath.equals("/ledgers/" + BookKeeperConstants.AVAILABLE_NODE)
+                && !fullPath.equals("/ledgers/" + BookKeeperConstants.INSTANCEID) // node created by test setup
                 ) {
                 assertEquals(1, acls.size());
                 assertEquals(31, acls.get(0).getPerms());
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/EntryLogTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/EntryLogTest.java
index 707c435..e3627f4 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/EntryLogTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/EntryLogTest.java
@@ -625,7 +625,7 @@ public class EntryLogTest {
         }
 
         @Override
-        public Boolean call() throws IOException {
+        public Boolean call() throws IOException, BookieException {
             try {
                 ByteBuf expectedByteBuf = generateEntry(ledgerId, entryId);
                 ByteBuf actualByteBuf = ledgerStorage.getEntry(ledgerId, entryId);
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/LedgerStorageCheckpointTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/LedgerStorageCheckpointTest.java
index 41e3e08..d62c869 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/LedgerStorageCheckpointTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/LedgerStorageCheckpointTest.java
@@ -222,7 +222,8 @@ public class LedgerStorageCheckpointTest {
         File ledgerDir = BookieImpl.getCurrentDirectories(conf.getLedgerDirs())[0];
         BookieServer server = new BookieServer(
                 conf, new TestBookieImpl(conf),
-                NullStatsLogger.INSTANCE, PooledByteBufAllocator.DEFAULT);
+                NullStatsLogger.INSTANCE, PooledByteBufAllocator.DEFAULT,
+                new MockUncleanShutdownDetection());
         server.start();
         ClientConfiguration clientConf = new ClientConfiguration();
         clientConf.setMetadataServiceUri(zkUtil.getMetadataServiceUri());
@@ -353,7 +354,8 @@ public class LedgerStorageCheckpointTest {
         File ledgerDir = BookieImpl.getCurrentDirectories(conf.getLedgerDirs())[0];
         BookieServer server = new BookieServer(
                 conf, new TestBookieImpl(conf),
-                NullStatsLogger.INSTANCE, PooledByteBufAllocator.DEFAULT);
+                NullStatsLogger.INSTANCE, PooledByteBufAllocator.DEFAULT,
+                new MockUncleanShutdownDetection());
         server.start();
         ClientConfiguration clientConf = new ClientConfiguration();
         clientConf.setMetadataServiceUri(zkUtil.getMetadataServiceUri());
@@ -433,7 +435,8 @@ public class LedgerStorageCheckpointTest {
         File ledgerDir = BookieImpl.getCurrentDirectories(conf.getLedgerDirs())[0];
         BookieServer server = new BookieServer(
                 conf, new TestBookieImpl(conf),
-                NullStatsLogger.INSTANCE, PooledByteBufAllocator.DEFAULT);
+                NullStatsLogger.INSTANCE, PooledByteBufAllocator.DEFAULT,
+                new MockUncleanShutdownDetection());
         server.start();
         ClientConfiguration clientConf = new ClientConfiguration();
         clientConf.setMetadataServiceUri(zkUtil.getMetadataServiceUri());
@@ -498,7 +501,8 @@ public class LedgerStorageCheckpointTest {
         File ledgerDir = BookieImpl.getCurrentDirectories(conf.getLedgerDirs())[0];
         BookieServer server = new BookieServer(
                 conf, new TestBookieImpl(conf),
-                NullStatsLogger.INSTANCE, PooledByteBufAllocator.DEFAULT);
+                NullStatsLogger.INSTANCE, PooledByteBufAllocator.DEFAULT,
+                new MockUncleanShutdownDetection());
         server.start();
         ClientConfiguration clientConf = new ClientConfiguration();
         clientConf.setMetadataServiceUri(zkUtil.getMetadataServiceUri());
@@ -604,7 +608,8 @@ public class LedgerStorageCheckpointTest {
         File ledgerDir = BookieImpl.getCurrentDirectories(conf.getLedgerDirs())[0];
         BookieServer server = new BookieServer(
                 conf, new TestBookieImpl(conf),
-                NullStatsLogger.INSTANCE, PooledByteBufAllocator.DEFAULT);
+                NullStatsLogger.INSTANCE, PooledByteBufAllocator.DEFAULT,
+                new MockUncleanShutdownDetection());
         server.start();
         ClientConfiguration clientConf = new ClientConfiguration();
         clientConf.setMetadataServiceUri(zkUtil.getMetadataServiceUri());
@@ -683,7 +688,8 @@ public class LedgerStorageCheckpointTest {
         conf.setLedgerStorageClass(InterleavedLedgerStorage.class.getName());
         server = new BookieServer(
                 conf, new TestBookieImpl(conf),
-                NullStatsLogger.INSTANCE, PooledByteBufAllocator.DEFAULT);
+                NullStatsLogger.INSTANCE, PooledByteBufAllocator.DEFAULT,
+                new MockUncleanShutdownDetection());
         server.start();
         BookKeeper newBKClient = new BookKeeper(clientConf);
         // since Bookie checkpointed successfully before shutdown/crash,
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/MockLedgerStorage.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/MockLedgerStorage.java
new file mode 100644
index 0000000..ea3ff04
--- /dev/null
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/MockLedgerStorage.java
@@ -0,0 +1,307 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.bookkeeper.bookie;
+
+import com.google.common.util.concurrent.RateLimiter;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufAllocator;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.Optional;
+import java.util.PrimitiveIterator;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.bookkeeper.bookie.CheckpointSource.Checkpoint;
+import org.apache.bookkeeper.common.util.Watcher;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.meta.LedgerManager;
+import org.apache.bookkeeper.stats.StatsLogger;
+
+/**
+ * A mock for running tests that require ledger storage.
+ */
+public class MockLedgerStorage implements LedgerStorage {
+
+    private static class LedgerInfo {
+        boolean limbo = false;
+        boolean fenced = false;
+        long lac = -1;
+        final byte[] masterKey;
+
+        LedgerInfo(byte[] masterKey) {
+            this.masterKey = Arrays.copyOf(masterKey, masterKey.length);
+        }
+
+        ConcurrentHashMap<Long, ByteBuf> entries = new ConcurrentHashMap<>();
+    }
+
+    private final ConcurrentHashMap<Long, LedgerInfo> ledgers = new ConcurrentHashMap<>();
+    private final EnumSet<StorageState> storageStateFlags = EnumSet.noneOf(StorageState.class);
+
+    @Override
+    public void initialize(ServerConfiguration conf,
+                           LedgerManager ledgerManager,
+                           LedgerDirsManager ledgerDirsManager,
+                           LedgerDirsManager indexDirsManager,
+                           StatsLogger statsLogger,
+                           ByteBufAllocator allocator)
+            throws IOException {}
+
+    @Override
+    public void setStateManager(StateManager stateManager) {}
+    @Override
+    public void setCheckpointSource(CheckpointSource checkpointSource) {}
+    @Override
+    public void setCheckpointer(Checkpointer checkpointer) {}
+
+    @Override
+    public void start() {}
+    @Override
+    public void shutdown() throws InterruptedException {}
+
+    @Override
+    public boolean ledgerExists(long ledgerId) throws IOException {
+        return ledgers.containsKey(ledgerId);
+    }
+
+    @Override
+    public boolean entryExists(long ledgerId, long entryId) throws IOException {
+        LedgerInfo info = ledgers.get(ledgerId);
+        if (info == null) {
+            throw new Bookie.NoLedgerException(ledgerId);
+        }
+        return info != null && info.entries.containsKey(entryId);
+    }
+
+    @Override
+    public boolean setFenced(long ledgerId) throws IOException {
+        AtomicBoolean ret = new AtomicBoolean(false);
+        LedgerInfo previous = ledgers.computeIfPresent(ledgerId, (ledgerId1, current) -> {
+                if (!current.fenced) {
+                    current.fenced = true;
+                    ret.set(true);
+                } else {
+                    ret.set(false);
+                }
+                return current;
+            });
+        if (previous == null) {
+            throw new Bookie.NoLedgerException(ledgerId);
+        }
+        return ret.get();
+    }
+
+    @Override
+    public boolean isFenced(long ledgerId) throws IOException {
+        LedgerInfo info = ledgers.get(ledgerId);
+        if (info == null) {
+            throw new Bookie.NoLedgerException(ledgerId);
+        }
+        return info != null && info.fenced;
+    }
+
+    @Override
+    public void setLimboState(long ledgerId) throws IOException {
+        LedgerInfo previous = ledgers.computeIfPresent(ledgerId, (ledgerId1, current) -> {
+                current.limbo = true;
+                return current;
+            });
+        if (previous == null) {
+            throw new Bookie.NoLedgerException(ledgerId);
+        }
+    }
+
+    @Override
+    public boolean hasLimboState(long ledgerId) throws IOException {
+        LedgerInfo info = ledgers.get(ledgerId);
+        if (info == null) {
+            throw new Bookie.NoLedgerException(ledgerId);
+        }
+        return info.limbo;
+    }
+    @Override
+    public void clearLimboState(long ledgerId) throws IOException {
+        LedgerInfo previous = ledgers.computeIfPresent(ledgerId, (ledgerId1, current) -> {
+                current.limbo = false;
+                return current;
+            });
+        if (previous == null) {
+            throw new Bookie.NoLedgerException(ledgerId);
+        }
+    }
+
+    @Override
+    public void setMasterKey(long ledgerId, byte[] masterKey) throws IOException {
+        LedgerInfo previous = ledgers.compute(ledgerId, (ledgerId1, current) -> {
+                if (current != null) {
+                    return current;
+                }
+                return new LedgerInfo(masterKey);
+            });
+        if (previous != null && !Arrays.equals(masterKey, previous.masterKey)) {
+            throw new IOException(BookieException.create(BookieException.Code.IllegalOpException));
+        }
+    }
+    @Override
+    public byte[] readMasterKey(long ledgerId) throws IOException, BookieException {
+        LedgerInfo info = ledgers.get(ledgerId);
+        if (info == null) {
+            throw new Bookie.NoLedgerException(ledgerId);
+        }
+        return Arrays.copyOf(info.masterKey, info.masterKey.length);
+    }
+
+    public long extractLedgerId(ByteBuf entry) {
+        return entry.getLong(entry.readerIndex());
+    }
+
+    public long extractEntryId(ByteBuf entry) {
+        return entry.getLong(entry.readerIndex() + 8);
+    }
+
+    public long extractLac(ByteBuf entry) {
+        return entry.getLong(entry.readerIndex() + 16);
+
+    }
+
+    @Override
+    public long addEntry(ByteBuf entry) throws IOException, BookieException {
+        ByteBuf copy = entry.retain().duplicate();
+        long ledgerId = extractLedgerId(copy);
+        long entryId = extractEntryId(copy);
+        long lac = extractLac(copy);
+
+        LedgerInfo previous = ledgers.computeIfPresent(ledgerId, (ledgerId1, current) -> {
+                if (lac > current.lac) {
+                    current.lac = lac;
+                }
+                current.entries.put(entryId, copy);
+                return current;
+            });
+        if (previous == null) {
+            throw new Bookie.NoLedgerException(ledgerId);
+        }
+        return entryId;
+    }
+
+    @Override
+    public ByteBuf getEntry(long ledgerId, long entryId) throws IOException {
+        throw new UnsupportedOperationException("Not supported in mock, implement if you need it");
+    }
+
+    @Override
+    public long getLastAddConfirmed(long ledgerId) throws IOException {
+        throw new UnsupportedOperationException("Not supported in mock, implement if you need it");
+    }
+
+    @Override
+    public boolean waitForLastAddConfirmedUpdate(
+            long ledgerId,
+            long previousLAC,
+            Watcher<LastAddConfirmedUpdateNotification> watcher) throws IOException {
+        throw new UnsupportedOperationException("Not supported in mock, implement if you need it");
+    }
+
+    @Override
+    public void cancelWaitForLastAddConfirmedUpdate(
+            long ledgerId,
+            Watcher<LastAddConfirmedUpdateNotification> watcher) throws IOException {
+        throw new UnsupportedOperationException("Not supported in mock, implement if you need it");
+    }
+
+    @Override
+    public void flush() throws IOException {
+        // this is a noop, as we dont hit disk anyhow
+    }
+
+    @Override
+    public void checkpoint(Checkpoint checkpoint) throws IOException {
+        throw new UnsupportedOperationException("Not supported in mock, implement if you need it");
+    }
+
+    @Override
+    public void deleteLedger(long ledgerId) throws IOException {
+        throw new UnsupportedOperationException("Not supported in mock, implement if you need it");
+    }
+
+    @Override
+    public void registerLedgerDeletionListener(LedgerDeletionListener listener) {
+        throw new UnsupportedOperationException("Not supported in mock, implement if you need it");
+    }
+
+    @Override
+    public void setExplicitLac(long ledgerId, ByteBuf lac) throws IOException {
+        throw new UnsupportedOperationException("Not supported in mock, implement if you need it");
+    }
+
+    @Override
+    public ByteBuf getExplicitLac(long ledgerId) throws IOException {
+        throw new UnsupportedOperationException("Not supported in mock, implement if you need it");
+    }
+
+    @Override
+    public LedgerStorage getUnderlyingLedgerStorage() {
+        return LedgerStorage.super.getUnderlyingLedgerStorage();
+    }
+
+    @Override
+    public void forceGC() {
+        LedgerStorage.super.forceGC();
+    }
+
+    @Override
+    public List<DetectedInconsistency> localConsistencyCheck(Optional<RateLimiter> rateLimiter) throws IOException {
+        return LedgerStorage.super.localConsistencyCheck(rateLimiter);
+    }
+
+    @Override
+    public boolean isInForceGC() {
+        return LedgerStorage.super.isInForceGC();
+    }
+
+    @Override
+    public List<GarbageCollectionStatus> getGarbageCollectionStatus() {
+        return LedgerStorage.super.getGarbageCollectionStatus();
+    }
+
+    @Override
+    public PrimitiveIterator.OfLong getListOfEntriesOfLedger(long ledgerId) throws IOException {
+        throw new UnsupportedOperationException("Not supported in mock, implement if you need it");
+    }
+
+    @Override
+    public EnumSet<StorageState> getStorageStateFlags() throws IOException {
+        return storageStateFlags;
+    }
+
+    @Override
+    public void setStorageStateFlag(StorageState flag) throws IOException {
+        storageStateFlags.add(flag);
+    }
+
+    @Override
+    public void clearStorageStateFlag(StorageState flag) throws IOException {
+        storageStateFlags.remove(flag);
+    }
+}
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/MockUncleanShutdownDetection.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/MockUncleanShutdownDetection.java
new file mode 100644
index 0000000..8fb64a3
--- /dev/null
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/MockUncleanShutdownDetection.java
@@ -0,0 +1,53 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+package org.apache.bookkeeper.bookie;
+
+/**
+ * Mock implementation of UncleanShutdownDetection.
+ */
+public class MockUncleanShutdownDetection implements UncleanShutdownDetection {
+
+    private boolean startRegistered;
+    private boolean shutdownRegistered;
+
+    @Override
+    public void registerStartUp() {
+        startRegistered = true;
+    }
+
+    @Override
+    public void registerCleanShutdown() {
+        shutdownRegistered = true;
+    }
+
+    @Override
+    public boolean lastShutdownWasUnclean() {
+        return startRegistered && !shutdownRegistered;
+    }
+
+    public boolean getStartRegistered() {
+        return startRegistered;
+    }
+
+    public boolean getShutdownRegistered() {
+        return shutdownRegistered;
+    }
+}
\ No newline at end of file
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/SyncThreadTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/SyncThreadTest.java
index 8f0a1e8..5dce85e 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/SyncThreadTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/SyncThreadTest.java
@@ -29,6 +29,7 @@ import io.netty.buffer.ByteBuf;
 import io.netty.buffer.ByteBufAllocator;
 
 import java.io.IOException;
+import java.util.EnumSet;
 import java.util.PrimitiveIterator.OfLong;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ExecutorService;
@@ -303,6 +304,11 @@ public class SyncThreadTest {
         }
 
         @Override
+        public boolean entryExists(long ledgerId, long entryId) throws IOException {
+            return false;
+        }
+
+        @Override
         public boolean setFenced(long ledgerId) throws IOException {
             return true;
         }
@@ -379,6 +385,38 @@ public class SyncThreadTest {
         public OfLong getListOfEntriesOfLedger(long ledgerId) {
             return null;
         }
+
+        @Override
+        public void setLimboState(long ledgerId) throws IOException {
+            throw new UnsupportedOperationException(
+                    "Limbo state only supported for DbLedgerStorage");
+        }
+
+        @Override
+        public boolean hasLimboState(long ledgerId) throws IOException {
+            throw new UnsupportedOperationException(
+                    "Limbo state only supported for DbLedgerStorage");
+        }
+
+        @Override
+        public void clearLimboState(long ledgerId) throws IOException {
+            throw new UnsupportedOperationException(
+                    "Limbo state only supported for DbLedgerStorage");
+        }
+
+        @Override
+        public EnumSet<StorageState> getStorageStateFlags() throws IOException {
+            return EnumSet.noneOf(StorageState.class);
+        }
+
+        @Override
+        public void setStorageStateFlag(StorageState flag) throws IOException {
+        }
+
+        @Override
+        public void clearStorageStateFlag(StorageState flag) throws IOException {
+        }
+
     }
 
 }
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/UncleanShutdownDetectionTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/UncleanShutdownDetectionTest.java
new file mode 100644
index 0000000..deddbdb
--- /dev/null
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/UncleanShutdownDetectionTest.java
@@ -0,0 +1,154 @@
+/**
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ *
+ */
+package org.apache.bookkeeper.bookie;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.conf.TestBKConfiguration;
+import org.apache.bookkeeper.util.DiskChecker;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+/**
+ * Test the unclean shutdown implementation.
+ */
+public class UncleanShutdownDetectionTest {
+
+    @Rule
+    public TemporaryFolder tempDir = new TemporaryFolder();
+
+    @Test
+    public void testRegisterStartWithoutRegisterShutdownEqualsUncleanShutdown() throws IOException {
+        ServerConfiguration conf = TestBKConfiguration.newServerConfiguration();
+        DiskChecker diskChecker = new DiskChecker(conf.getDiskUsageThreshold(), conf.getDiskUsageWarnThreshold());
+        LedgerDirsManager ledgerDirsManager = new LedgerDirsManager(
+                conf, conf.getLedgerDirs(), diskChecker);
+
+        UncleanShutdownDetection uncleanShutdownDetection = new UncleanShutdownDetectionImpl(ledgerDirsManager);
+        uncleanShutdownDetection.registerStartUp();
+
+        assertTrue(uncleanShutdownDetection.lastShutdownWasUnclean());
+    }
+
+    @Test
+    public void testRegisterStartWithRegisterShutdownEqualsCleanShutdown() throws IOException {
+        ServerConfiguration conf = TestBKConfiguration.newServerConfiguration();
+        DiskChecker diskChecker = new DiskChecker(conf.getDiskUsageThreshold(), conf.getDiskUsageWarnThreshold());
+        LedgerDirsManager ledgerDirsManager = new LedgerDirsManager(
+                conf, conf.getLedgerDirs(), diskChecker);
+
+        UncleanShutdownDetection uncleanShutdownDetection = new UncleanShutdownDetectionImpl(ledgerDirsManager);
+        uncleanShutdownDetection.registerStartUp();
+        uncleanShutdownDetection.registerCleanShutdown();
+
+        assertFalse(uncleanShutdownDetection.lastShutdownWasUnclean());
+    }
+
+    @Test
+    public void testRegisterStartWithoutRegisterShutdownEqualsUncleanShutdownMultipleDirs() throws IOException {
+        File ledgerDir1 = tempDir.newFolder("l1");
+        File ledgerDir2 = tempDir.newFolder("l2");
+        File ledgerDir3 = tempDir.newFolder("l3");
+        ServerConfiguration conf = TestBKConfiguration.newServerConfiguration()
+                .setLedgerDirNames(new String[] {ledgerDir1.getAbsolutePath(), ledgerDir2.getAbsolutePath(),
+                        ledgerDir3.getAbsolutePath()});
+        DiskChecker diskChecker = new DiskChecker(conf.getDiskUsageThreshold(), conf.getDiskUsageWarnThreshold());
+        LedgerDirsManager ledgerDirsManager = new LedgerDirsManager(
+                conf, conf.getLedgerDirs(), diskChecker);
+
+        UncleanShutdownDetection uncleanShutdownDetection = new UncleanShutdownDetectionImpl(ledgerDirsManager);
+        uncleanShutdownDetection.registerStartUp();
+
+        assertTrue(uncleanShutdownDetection.lastShutdownWasUnclean());
+    }
+
+    @Test
+    public void testRegisterStartWithRegisterShutdownEqualsCleanShutdownMultipleDirs() throws IOException {
+        File ledgerDir1 = tempDir.newFolder("l1");
+        File ledgerDir2 = tempDir.newFolder("l2");
+        File ledgerDir3 = tempDir.newFolder("l3");
+        ServerConfiguration conf = TestBKConfiguration.newServerConfiguration()
+                .setLedgerDirNames(new String[] {ledgerDir1.getAbsolutePath(), ledgerDir2.getAbsolutePath(),
+                        ledgerDir3.getAbsolutePath()});
+        DiskChecker diskChecker = new DiskChecker(conf.getDiskUsageThreshold(), conf.getDiskUsageWarnThreshold());
+        LedgerDirsManager ledgerDirsManager = new LedgerDirsManager(
+                conf, conf.getLedgerDirs(), diskChecker);
+
+        UncleanShutdownDetection uncleanShutdownDetection = new UncleanShutdownDetectionImpl(ledgerDirsManager);
+        uncleanShutdownDetection.registerStartUp();
+        uncleanShutdownDetection.registerCleanShutdown();
+
+        assertFalse(uncleanShutdownDetection.lastShutdownWasUnclean());
+    }
+
+    @Test
+    public void testRegisterStartWithPartialRegisterShutdownEqualsUncleanShutdownMultipleDirs() throws IOException {
+        File ledgerDir1 = tempDir.newFolder("l1");
+        File ledgerDir2 = tempDir.newFolder("l2");
+        File ledgerDir3 = tempDir.newFolder("l3");
+        ServerConfiguration conf = TestBKConfiguration.newServerConfiguration()
+                .setLedgerDirNames(new String[] {ledgerDir1.getAbsolutePath(), ledgerDir2.getAbsolutePath(),
+                        ledgerDir3.getAbsolutePath()});
+        DiskChecker diskChecker = new DiskChecker(conf.getDiskUsageThreshold(), conf.getDiskUsageWarnThreshold());
+        LedgerDirsManager ledgerDirsManager = new LedgerDirsManager(
+                conf, conf.getLedgerDirs(), diskChecker);
+
+        UncleanShutdownDetection uncleanShutdownDetection = new UncleanShutdownDetectionImpl(ledgerDirsManager);
+        uncleanShutdownDetection.registerStartUp();
+        uncleanShutdownDetection.registerCleanShutdown();
+        File dirtyFile = new File(ledgerDirsManager.getAllLedgerDirs().get(0),
+                UncleanShutdownDetectionImpl.DIRTY_FILENAME);
+        dirtyFile.createNewFile();
+
+        assertTrue(uncleanShutdownDetection.lastShutdownWasUnclean());
+    }
+
+    @Test(expected = IOException.class)
+    public void testRegisterStartFailsToCreateDirtyFilesAndThrowsIOException() throws IOException {
+        ServerConfiguration conf = TestBKConfiguration.newServerConfiguration();
+        DiskChecker diskChecker = new DiskChecker(conf.getDiskUsageThreshold(), conf.getDiskUsageWarnThreshold());
+        LedgerDirsManager ledgerDirsManager = new MockLedgerDirsManager(conf, conf.getLedgerDirs(), diskChecker);
+
+        UncleanShutdownDetection uncleanShutdownDetection = new UncleanShutdownDetectionImpl(ledgerDirsManager);
+        uncleanShutdownDetection.registerStartUp();
+    }
+
+    private class MockLedgerDirsManager extends LedgerDirsManager {
+        public MockLedgerDirsManager(ServerConfiguration conf, File[] dirs, DiskChecker diskChecker)
+                throws IOException {
+            super(conf, dirs, diskChecker);
+        }
+
+        @Override
+        public List<File> getAllLedgerDirs() {
+            List<File> dirs = new ArrayList<>();
+            dirs.add(new File("does_not_exist"));
+            return dirs;
+        }
+    }
+}
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/datainteg/CookieValidationTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/datainteg/CookieValidationTest.java
new file mode 100644
index 0000000..ea8cf81
--- /dev/null
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/datainteg/CookieValidationTest.java
@@ -0,0 +1,329 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.bookkeeper.bookie.datainteg;
+
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.greaterThan;
+import static org.hamcrest.Matchers.not;
+import static org.hamcrest.Matchers.notNullValue;
+import static org.mockito.Mockito.anyObject;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
+import com.google.common.collect.Lists;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.net.UnknownHostException;
+import java.nio.charset.StandardCharsets;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+
+import org.apache.bookkeeper.bookie.BookieException;
+import org.apache.bookkeeper.bookie.BookieImpl;
+import org.apache.bookkeeper.bookie.Cookie;
+import org.apache.bookkeeper.common.concurrent.FutureUtils;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.discover.MockRegistrationManager;
+import org.apache.bookkeeper.net.BookieId;
+import org.apache.bookkeeper.test.TmpDirs;
+import org.apache.bookkeeper.util.BookKeeperConstants;
+import org.apache.bookkeeper.versioning.Version;
+import org.apache.bookkeeper.versioning.Versioned;
+
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Test;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Test the DataIntegrityCookieValidation implementation of CookieValidation.
+ */
+public class CookieValidationTest {
+    private static Logger log = LoggerFactory.getLogger(CookieValidationTest.class);
+    final TmpDirs tmpDirs = new TmpDirs();
+
+    @After
+    public void cleanup() throws Exception {
+        tmpDirs.cleanup();
+    }
+
+    private File initializedDir() throws Exception {
+        File dir = tmpDirs.createNew("cookie", "validation");
+        BookieImpl.checkDirectoryStructure(BookieImpl.getCurrentDirectory(dir));
+        return dir;
+    }
+
+    private static ServerConfiguration serverConf(boolean stampMissingCookies) {
+        ServerConfiguration conf = new ServerConfiguration();
+        conf.setDataIntegrityStampMissingCookiesEnabled(stampMissingCookies);
+        conf.setAdvertisedAddress("foobar");
+        return conf;
+    }
+
+    private Versioned<byte[]> genCookie(ServerConfiguration conf) throws UnknownHostException {
+        return new Versioned<>(Cookie.generateCookie(conf).build().toString()
+                .getBytes(StandardCharsets.UTF_8), Version.NEW);
+    }
+
+    @Test
+    public void testNoZkCookieAndEmptyDirsStampsNewCookie() throws Exception {
+        List<File> dirs = Lists.newArrayList(initializedDir(),
+                                             initializedDir());
+
+        ServerConfiguration conf = serverConf(false);
+        BookieId bookieId = BookieImpl.getBookieId(conf);
+        MockRegistrationManager regManager = new MockRegistrationManager();
+        DataIntegrityCookieValidation v = new DataIntegrityCookieValidation(
+                conf, regManager, new MockDataIntegrityCheck());
+        v.checkCookies(dirs);
+
+        byte[] cookieBytes = regManager.readCookie(bookieId).getValue();
+        assertThat(cookieBytes, notNullValue());
+        assertThat(cookieBytes.length, greaterThan(0));
+
+        Cookie regManagerCookie = Cookie.parseFromBytes(cookieBytes);
+
+        for (File d : dirs) {
+            assertThat(Cookie.readFromDirectory(d), equalTo(regManagerCookie));
+        }
+    }
+
+    @Test(expected = BookieException.InvalidCookieException.class)
+    public void testZkCookieAndEmptyDirsRaisesErrorWithoutMissingCookieStamping() throws Exception {
+        List<File> dirs = Lists.newArrayList(initializedDir(),
+                initializedDir());
+
+        ServerConfiguration conf = serverConf(false);
+        BookieId bookieId = BookieImpl.getBookieId(conf);
+        MockRegistrationManager regManager = new MockRegistrationManager();
+        regManager.writeCookie(bookieId, genCookie(conf));
+        DataIntegrityCookieValidation v = new DataIntegrityCookieValidation(
+                conf, regManager, new MockDataIntegrityCheck());
+        v.checkCookies(dirs);
+    }
+
+    @Test
+    public void testZkCookieAndEmptyDirsStampsNewCookieWithMissingCookieStamping() throws Exception {
+        List<File> dirs = Lists.newArrayList(initializedDir(),
+                initializedDir());
+
+        ServerConfiguration conf = serverConf(true);
+        BookieId bookieId = BookieImpl.getBookieId(conf);
+        MockRegistrationManager regManager = new MockRegistrationManager();
+        regManager.writeCookie(bookieId, genCookie(conf));
+        DataIntegrityCookieValidation v = new DataIntegrityCookieValidation(
+                conf, regManager, new MockDataIntegrityCheck());
+        v.checkCookies(dirs);
+
+        byte[] cookieBytes = regManager.readCookie(bookieId).getValue();
+        assertThat(cookieBytes, notNullValue());
+        assertThat(cookieBytes.length, greaterThan(0));
+
+        Cookie regManagerCookie = Cookie.parseFromBytes(cookieBytes);
+
+        for (File d : dirs) {
+            assertThat(Cookie.readFromDirectory(d), equalTo(regManagerCookie));
+        }
+    }
+
+    @Test(expected = BookieException.InvalidCookieException.class)
+    public void testMissingZKCookieRaisesError() throws Exception {
+        List<File> dirs = Lists.newArrayList(initializedDir(),
+                initializedDir());
+
+        ServerConfiguration conf = serverConf(true);
+
+        MockRegistrationManager regManager = new MockRegistrationManager();
+        DataIntegrityCookieValidation v1 = new DataIntegrityCookieValidation(
+                conf, regManager, new MockDataIntegrityCheck());
+        v1.checkCookies(dirs);
+
+        MockRegistrationManager blankRegManager = new MockRegistrationManager();
+        DataIntegrityCookieValidation v2 = new DataIntegrityCookieValidation(
+                conf, blankRegManager, new MockDataIntegrityCheck());
+        v2.checkCookies(dirs);
+    }
+
+    @Test
+    public void testMatchingCookiesTakesNoAction() throws Exception {
+        List<File> dirs = Lists.newArrayList(initializedDir(),
+                                             initializedDir());
+
+        ServerConfiguration conf = serverConf(true);
+
+        MockRegistrationManager regManager = new MockRegistrationManager();
+        DataIntegrityCookieValidation v1 = new DataIntegrityCookieValidation(
+                conf, regManager, new MockDataIntegrityCheck());
+        v1.checkCookies(dirs); // stamp original cookies
+
+        DataIntegrityCookieValidation v2 = new DataIntegrityCookieValidation(
+                conf, regManager, new MockDataIntegrityCheck());
+        v2.checkCookies(dirs); // should find cookies and return successfully
+    }
+
+    @Test
+    public void testEmptyDirectoryTriggersIntegrityCheck() throws Exception {
+        List<File> dirs = Lists.newArrayList(initializedDir(),
+                                             initializedDir());
+        ServerConfiguration conf = serverConf(true);
+
+        MockRegistrationManager regManager = new MockRegistrationManager();
+        MockDataIntegrityCheck dataIntegCheck = spy(new MockDataIntegrityCheck());
+        DataIntegrityCookieValidation v1 = new DataIntegrityCookieValidation(
+                conf, regManager, dataIntegCheck);
+        v1.checkCookies(dirs); // stamp original cookies
+        verify(dataIntegCheck, times(0)).runPreBootCheck("INVALID_COOKIE");
+
+        dirs.add(initializedDir());
+        v1.checkCookies(dirs); // stamp original cookies
+        verify(dataIntegCheck, times(1)).runPreBootCheck("INVALID_COOKIE");
+
+        v1.checkCookies(dirs); // stamp original cookies
+        verify(dataIntegCheck, times(1)).runPreBootCheck("INVALID_COOKIE");
+    }
+
+    @Test
+    public void testErrorInIntegrityCheckPreventsStamping() throws Exception {
+        List<File> dirs = Lists.newArrayList(initializedDir(),
+                                             initializedDir());
+
+        ServerConfiguration conf = serverConf(true);
+
+        MockRegistrationManager regManager = spy(new MockRegistrationManager());
+        MockDataIntegrityCheck dataIntegCheck = spy(new MockDataIntegrityCheck() {
+                @Override
+                public CompletableFuture<Void> runPreBootCheck(String reason) {
+                    return FutureUtils.exception(new BookieException.InvalidCookieException("blah"));
+                }
+            });
+
+        DataIntegrityCookieValidation v1 = new DataIntegrityCookieValidation(
+                conf, regManager, dataIntegCheck);
+
+        v1.checkCookies(dirs); // stamp original cookies
+        verify(dataIntegCheck, times(0)).runPreBootCheck("INVALID_COOKIE");
+        verify(regManager, times(1)).writeCookie(anyObject(), anyObject());
+
+        // add a directory to trigger data integrity check
+        dirs.add(initializedDir());
+        try {
+            v1.checkCookies(dirs); // stamp original cookies
+            Assert.fail("failure of data integrity should fail cookie check");
+        } catch (BookieException.InvalidCookieException e) {
+            // expected
+        }
+        verify(dataIntegCheck, times(1)).runPreBootCheck("INVALID_COOKIE");
+        verify(regManager, times(1)).writeCookie(anyObject(), anyObject());
+
+        // running the check again should run data integrity again, as stamping didn't happen
+        try {
+            v1.checkCookies(dirs); // stamp original cookies
+            Assert.fail("failure of data integrity should fail cookie check");
+        } catch (BookieException.InvalidCookieException e) {
+            // expected
+        }
+        verify(dataIntegCheck, times(2)).runPreBootCheck("INVALID_COOKIE");
+        verify(regManager, times(1)).writeCookie(anyObject(), anyObject());
+    }
+
+    @Test
+    public void testChangingBookieIdRaisesError() throws Exception {
+        List<File> dirs = Lists.newArrayList(initializedDir(),
+                                             initializedDir());
+        ServerConfiguration conf = serverConf(true);
+        MockRegistrationManager regManager = new MockRegistrationManager();
+        DataIntegrityCookieValidation v1 = new DataIntegrityCookieValidation(
+                conf, regManager, new MockDataIntegrityCheck());
+        v1.checkCookies(dirs); // stamp original cookies
+
+        conf.setAdvertisedAddress("barfoo");
+        DataIntegrityCookieValidation v2 = new DataIntegrityCookieValidation(
+                conf, regManager, new MockDataIntegrityCheck());
+        try {
+            v2.checkCookies(dirs); // should fail as cookie not found in ZK, but exists in dirs
+            Assert.fail("Check shouldn't have succeeded with new bookieId");
+        } catch (BookieException.InvalidCookieException ice) {
+            // expected
+        }
+
+        conf.setAdvertisedAddress("foobar");
+        DataIntegrityCookieValidation v3 = new DataIntegrityCookieValidation(
+                conf, regManager, new MockDataIntegrityCheck());
+        v3.checkCookies(dirs); // should succeed as the cookie is same as before
+    }
+
+    @Test
+    public void testMismatchLocalCookie() throws Exception {
+        List<File> dirs = Lists.newArrayList(initializedDir(),
+                                             initializedDir());
+
+        ServerConfiguration conf = serverConf(true);
+
+        MockDataIntegrityCheck dataIntegCheck = spy(new MockDataIntegrityCheck());
+        MockRegistrationManager regManager = spy(new MockRegistrationManager());
+        DataIntegrityCookieValidation v1 = new DataIntegrityCookieValidation(
+                conf, regManager, dataIntegCheck);
+        v1.checkCookies(dirs); // stamp original cookies
+
+        verify(dataIntegCheck, times(0)).runPreBootCheck("INVALID_COOKIE");
+        verify(regManager, times(1)).writeCookie(anyObject(), anyObject());
+
+        Cookie current = Cookie.readFromDirectory(dirs.get(0));
+        Cookie mismatch = Cookie.newBuilder(current).setBookieId("mismatch:3181").build();
+        mismatch.writeToDirectory(dirs.get(0));
+        assertThat(current, not(Cookie.readFromDirectory(dirs.get(0))));
+
+        v1.checkCookies(dirs);
+        verify(dataIntegCheck, times(1)).runPreBootCheck("INVALID_COOKIE");
+        verify(regManager, times(2)).writeCookie(anyObject(), anyObject());
+
+        Cookie afterCheck = Cookie.readFromDirectory(dirs.get(0));
+        assertThat(afterCheck, equalTo(current));
+    }
+
+    @Test(expected = BookieException.InvalidCookieException.class)
+    public void testCorruptLocalCookie() throws Exception {
+        List<File> dirs = Lists.newArrayList(initializedDir(),
+                                             initializedDir());
+
+        ServerConfiguration conf = serverConf(true);
+
+        MockDataIntegrityCheck dataIntegCheck = spy(new MockDataIntegrityCheck());
+        MockRegistrationManager regManager = spy(new MockRegistrationManager());
+        DataIntegrityCookieValidation v1 = new DataIntegrityCookieValidation(
+                conf, regManager, dataIntegCheck);
+        v1.checkCookies(dirs); // stamp original cookies
+
+        verify(dataIntegCheck, times(0)).runPreBootCheck("INVALID_COOKIE");
+        verify(regManager, times(1)).writeCookie(anyObject(), anyObject());
+
+        File cookieFile = new File(dirs.get(0), BookKeeperConstants.VERSION_FILENAME);
+        try (FileOutputStream out = new FileOutputStream(cookieFile)) {
+            out.write(0xdeadbeef);
+        }
+        v1.checkCookies(dirs); // should throw
+    }
+}
+
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/datainteg/DataIntegrityCheckTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/datainteg/DataIntegrityCheckTest.java
new file mode 100644
index 0000000..eb58e33
--- /dev/null
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/datainteg/DataIntegrityCheckTest.java
@@ -0,0 +1,1544 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.bookkeeper.bookie.datainteg;
+
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.instanceOf;
+import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.isIn;
+import static org.hamcrest.Matchers.not;
+import static org.mockito.Mockito.anyInt;
+import static org.mockito.Mockito.anyLong;
+import static org.mockito.Mockito.anyObject;
+import static org.mockito.Mockito.eq;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+
+import io.netty.buffer.ByteBuf;
+
+import io.reactivex.rxjava3.core.Single;
+import io.reactivex.rxjava3.exceptions.CompositeException;
+import io.reactivex.rxjava3.observers.TestObserver;
+import io.reactivex.rxjava3.schedulers.Schedulers;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+import java.util.stream.LongStream;
+
+import org.apache.bookkeeper.bookie.BookieException;
+import org.apache.bookkeeper.bookie.BookieImpl;
+import org.apache.bookkeeper.bookie.LedgerStorage.StorageState;
+import org.apache.bookkeeper.bookie.MockLedgerStorage;
+import org.apache.bookkeeper.client.BKException;
+import org.apache.bookkeeper.client.BookKeeperAdmin;
+import org.apache.bookkeeper.client.LedgerMetadataBuilder;
+import org.apache.bookkeeper.client.api.DigestType;
+import org.apache.bookkeeper.client.api.LedgerMetadata;
+import org.apache.bookkeeper.common.concurrent.FutureUtils;
+import org.apache.bookkeeper.common.util.MockTicker;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.meta.MockLedgerManager;
+import org.apache.bookkeeper.net.BookieId;
+import org.apache.bookkeeper.proto.MockBookieClient;
+
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Test of DataIntegrityCheckImpl.
+ */
+public class DataIntegrityCheckTest {
+    private static final byte[] PASSWD = new byte[0];
+
+    private final BookieId bookie1 = BookieId.parse("bookie1:3181");
+    private final BookieId bookie2 = BookieId.parse("bookie2:3181");
+    private final BookieId bookie3 = BookieId.parse("bookie3:3181");
+    private final BookieId bookie4 = BookieId.parse("bookie4:3181");
+    private final BookieId bookie5 = BookieId.parse("bookie5:3181");
+
+    private OrderedExecutor executor = null;
+
+    @Before
+    public void setup() throws Exception {
+        executor = OrderedExecutor.newBuilder().numThreads(1).name("test").build();
+    }
+
+    @After
+    public void teardown() throws Exception {
+        if (executor != null) {
+            executor.shutdownNow();
+        }
+    }
+
+    private static ServerConfiguration serverConf() {
+        ServerConfiguration conf = new ServerConfiguration();
+        conf.setAdvertisedAddress("foobar");
+        return conf;
+    }
+
+    private LedgerMetadataBuilder newMetadataWithEnsemble(
+            long ledgerId,
+            BookieId... bookies) {
+        return LedgerMetadataBuilder.create()
+            .withId(ledgerId)
+            .withPassword(new byte[0])
+            .withDigestType(DigestType.CRC32C)
+            .withEnsembleSize(bookies.length)
+            .withWriteQuorumSize(bookies.length)
+            .withAckQuorumSize(bookies.length)
+            .newEnsembleEntry(0, Lists.newArrayList(bookies));
+    }
+
+    private LedgerMetadataBuilder newClosedMetadataWithEnsemble(long ledgerId,
+                                                                long numEntries,
+                                                                BookieId... bookies) {
+        return LedgerMetadataBuilder.create()
+            .withId(ledgerId)
+            .withPassword(new byte[0])
+            .withDigestType(DigestType.CRC32C)
+            .withEnsembleSize(bookies.length)
+            .withWriteQuorumSize(bookies.length)
+            .withAckQuorumSize(bookies.length)
+            .newEnsembleEntry(0, Lists.newArrayList(bookies))
+            .withLastEntryId(numEntries - 1)
+            .withLength(128 * numEntries)
+            .withClosedState();
+    }
+
+    @Test
+    public void testPrebootBookieIdInOpenSegmentMarkedInLimbo() throws Exception {
+        MockLedgerManager lm = new MockLedgerManager();
+
+        ServerConfiguration conf = serverConf();
+        BookieId bookieId = BookieImpl.getBookieId(conf);
+        lm.createLedgerMetadata(0xbeefL, newMetadataWithEnsemble(0xbeefL, bookieId).build()).get();
+
+        MockLedgerStorage storage = new MockLedgerStorage();
+        assertThat(storage.ledgerExists(0xbeefL), is(false));
+        DataIntegrityCheckImpl impl = new DataIntegrityCheckImpl(BookieImpl.getBookieId(conf), lm, storage,
+                                                                 mock(EntryCopier.class),
+                                                                 mock(BookKeeperAdmin.class),
+                                                                 Schedulers.io());
+        impl.runPreBootCheck("test").get();
+
+        assertThat(storage.hasLimboState(0xbeefL), is(true));
+        assertThat(storage.isFenced(0xbeefL), is(true));
+    }
+
+    @Test
+    public void testPrebootFencedMarkedInLimbo() throws Exception {
+        MockLedgerManager lm = new MockLedgerManager();
+
+        ServerConfiguration conf = serverConf();
+        BookieId bookieId = BookieImpl.getBookieId(conf);
+        lm.createLedgerMetadata(0xbeefL,
+                newMetadataWithEnsemble(0xbeefL,
+                        BookieImpl.getBookieId(conf)).withInRecoveryState().build()).get();
+
+        MockLedgerStorage storage = new MockLedgerStorage();
+        assertThat(storage.ledgerExists(0xbeefL), is(false));
+
+        DataIntegrityCheckImpl impl = new DataIntegrityCheckImpl(bookieId, lm, storage,
+                                                                 mock(EntryCopier.class),
+                                                                 mock(BookKeeperAdmin.class),
+                                                                 Schedulers.io());
+        impl.runPreBootCheck("test").get();
+
+        assertThat(storage.hasLimboState(0xbeefL), is(true));
+        assertThat(storage.isFenced(0xbeefL), is(true));
+    }
+
+    @Test
+    public void testPrebootClosedNotMarkedInLimbo() throws Exception {
+        MockLedgerManager lm = new MockLedgerManager();
+
+        ServerConfiguration conf = serverConf();
+        BookieId bookieId = BookieImpl.getBookieId(conf);
+        lm.createLedgerMetadata(0xbeefL,
+                newMetadataWithEnsemble(0xbeefL, BookieImpl.getBookieId(conf)).withClosedState()
+                .withLength(100).withLastEntryId(1).build()).get();
+
+        MockLedgerStorage storage = new MockLedgerStorage();
+        assertThat(storage.ledgerExists(0xbeefL), is(false));
+
+        DataIntegrityCheckImpl impl = new DataIntegrityCheckImpl(bookieId, lm, storage,
+                                                                 mock(EntryCopier.class),
+                                                                 mock(BookKeeperAdmin.class),
+                                                                 Schedulers.io());
+        impl.runPreBootCheck("test").get();
+
+        assertThat(storage.hasLimboState(0xbeefL), is(false));
+        assertThat(storage.isFenced(0xbeefL), is(false));
+    }
+
+    @Test
+    public void testPrebootFlushCalled() throws Exception {
+        MockLedgerManager lm = new MockLedgerManager();
+
+        ServerConfiguration conf = serverConf();
+        BookieId bookieId = BookieImpl.getBookieId(conf);
+        lm.createLedgerMetadata(0xbeefL, newMetadataWithEnsemble(
+                0xbeefL, BookieImpl.getBookieId(conf)).build()).get();
+
+        MockLedgerStorage storage = spy(new MockLedgerStorage());
+        assertThat(storage.ledgerExists(0xbeefL), is(false));
+
+        DataIntegrityCheckImpl impl = new DataIntegrityCheckImpl(bookieId, lm, storage,
+                                                                 mock(EntryCopier.class),
+                                                                 mock(BookKeeperAdmin.class),
+                                                                 Schedulers.io());
+        verify(storage, times(0)).flush();
+        impl.runPreBootCheck("test").get();
+        verify(storage, times(1)).flush();
+
+        assertThat(storage.hasLimboState(0xbeefL), is(true));
+        assertThat(storage.isFenced(0xbeefL), is(true));
+    }
+
+    @Test(expected = ExecutionException.class)
+    public void testFailureInPrebootMarkFailsAll() throws Exception {
+        MockLedgerManager lm = new MockLedgerManager();
+
+        ServerConfiguration conf = serverConf();
+        BookieId bookieId = BookieImpl.getBookieId(conf);
+        lm.createLedgerMetadata(0xbeedL, newMetadataWithEnsemble(0xbeedL, bookieId).build()).get();
+        lm.createLedgerMetadata(0xbeefL, newMetadataWithEnsemble(0xbeefL, bookieId).build()).get();
+        lm.createLedgerMetadata(0xbee0L, newMetadataWithEnsemble(0xbee0L, bookieId).build()).get();
+
+        MockLedgerStorage storage = new MockLedgerStorage() {
+                @Override
+                public void setLimboState(long ledgerId) throws IOException {
+                    if (ledgerId == 0xbeefL) {
+                        throw new IOException("boom!");
+                    } else {
+                        super.setLimboState(ledgerId);
+                    }
+                }
+            };
+
+        DataIntegrityCheckImpl impl = new DataIntegrityCheckImpl(bookieId, lm, storage,
+                                                                 mock(EntryCopier.class),
+                                                                 mock(BookKeeperAdmin.class),
+                                                                 Schedulers.io());
+        impl.runPreBootCheck("test").get();
+    }
+
+    @Test
+    public void testRecoverLimboOpensAndClears() throws Exception {
+        MockLedgerManager lm = new MockLedgerManager();
+        ServerConfiguration conf = serverConf();
+        BookieId bookieId = BookieImpl.getBookieId(conf);
+        MockLedgerStorage storage = spy(new MockLedgerStorage());
+        DataIntegrityCheckImpl impl = new DataIntegrityCheckImpl(bookieId, lm, storage,
+                                                                 mock(EntryCopier.class),
+                                                                 mock(BookKeeperAdmin.class),
+                                                                 Schedulers.io()) {
+                @Override
+                Single<LedgerMetadata> recoverLedger(long ledgerId, String runId) {
+                    return Single.just(newClosedMetadataWithEnsemble(ledgerId, -1, bookieId, bookie1).build());
+                }
+            };
+
+        Map<Long, LedgerMetadata> ledgers = new HashMap<>();
+        ledgers.put(0xf00L, newMetadataWithEnsemble(0xf00L, bookieId, bookie1).build());
+        storage.setMasterKey(0xf00L, PASSWD);
+        storage.setLimboState(0xf00L);
+        ledgers.put(0xdeadL, newMetadataWithEnsemble(0xdeadL, bookieId, bookie1).build());
+        storage.setMasterKey(0xdeadL, PASSWD);
+        storage.setLimboState(0xdeadL);
+
+        Set<DataIntegrityCheckImpl.LedgerResult> results = impl.checkAndRecoverLedgers(
+                ledgers, "test").get();
+
+        assertThat(results.stream().filter(r -> r.isOK()).count(), equalTo(2L));
+        verify(storage, times(1)).clearLimboState(0xf00L);
+        verify(storage, times(1)).clearLimboState(0xdeadL);
+    }
+
+    @Test
+    public void testRecoverLimboErrorOnOpenOnlyAffectsThatOne() throws Exception {
+        MockLedgerManager lm = new MockLedgerManager();
+        ServerConfiguration conf = serverConf();
+        BookieId bookieId = BookieImpl.getBookieId(conf);
+        MockLedgerStorage storage = spy(new MockLedgerStorage());
+        DataIntegrityCheckImpl impl = new DataIntegrityCheckImpl(bookieId, lm, storage,
+                                                                 mock(EntryCopier.class),
+                                                                 mock(BookKeeperAdmin.class),
+                                                                 Schedulers.io()) {
+                @Override
+                Single<LedgerMetadata> recoverLedger(long ledgerId, String runId) {
+                    if (ledgerId == 0xf00L) {
+                        return Single.error(new BKException.BKReadException());
+                    } else {
+                        return Single.just(newClosedMetadataWithEnsemble(ledgerId, 0, bookieId, bookie1).build());
+                    }
+                }
+            };
+
+        Map<Long, LedgerMetadata> ledgers = new HashMap<>();
+        ledgers.put(0xf00L, newMetadataWithEnsemble(0xf00L, bookieId, bookie1).build());
+        storage.setMasterKey(0xf00L, PASSWD);
+        storage.setLimboState(0xf00L);
+        ledgers.put(0xdeadL, newMetadataWithEnsemble(0xdeadL, bookieId, bookie1).build());
+        storage.setMasterKey(0xdeadL, PASSWD);
+        storage.setLimboState(0xdeadL);
+
+        Set<DataIntegrityCheckImpl.LedgerResult> results = impl.checkAndRecoverLedgers(ledgers, "test").get();
+
+        assertThat(results.stream().filter(r -> r.isOK()).count(), equalTo(1L));
+        assertThat(results.stream().filter(r -> r.isOK()).map(r -> r.getLedgerId()).findFirst().get(),
+                   equalTo(0xdeadL));
+        assertThat(results.stream().filter(r -> r.isError()).count(), equalTo(1L));
+        assertThat(results.stream().filter(r -> r.isError()).map(r -> r.getLedgerId()).findFirst().get(),
+                   equalTo(0xf00L));
+
+        verify(storage, times(0)).clearLimboState(0xf00L);
+        verify(storage, times(1)).clearLimboState(0xdeadL);
+    }
+
+    @Test
+    public void testRecoverLimboNoSuchLedger() throws Exception {
+        MockLedgerManager lm = new MockLedgerManager();
+        ServerConfiguration conf = serverConf();
+        BookieId bookieId = BookieImpl.getBookieId(conf);
+        MockLedgerStorage storage = spy(new MockLedgerStorage());
+
+        DataIntegrityCheckImpl impl = new DataIntegrityCheckImpl(bookieId, lm, storage,
+                                                                 mock(EntryCopier.class),
+                                                                 mock(BookKeeperAdmin.class),
+                                                                 Schedulers.io()) {
+                @Override
+                Single<LedgerMetadata> recoverLedger(long ledgerId, String runId) {
+                    if (ledgerId == 0xdeadL) {
+                        return Single.error(
+                                new BKException.BKNoSuchLedgerExistsOnMetadataServerException());
+                    } else {
+                        return Single.just(newClosedMetadataWithEnsemble(ledgerId, -1, bookieId, bookie1).build());
+                    }
+                }
+            };
+
+        Map<Long, LedgerMetadata> ledgers = new HashMap<>();
+        ledgers.put(0xf00L, newMetadataWithEnsemble(0xf00L, bookieId, bookie1).build());
+        storage.setMasterKey(0xf00L, PASSWD);
+        storage.setLimboState(0xf00L);
+        ledgers.put(0xdeadL, newMetadataWithEnsemble(0xdeadL, bookieId, bookie1).build());
+        storage.setMasterKey(0xdeadL, PASSWD);
+        storage.setLimboState(0xdeadL);
+
+        Set<DataIntegrityCheckImpl.LedgerResult> results = impl.checkAndRecoverLedgers(ledgers, "test").get();
+
+        assertThat(results.stream().filter(r -> r.isOK()).count(), equalTo(1L));
+        assertThat(results.stream().filter(r -> r.isOK()).map(r -> r.getLedgerId()).findFirst().get(),
+                   equalTo(0xf00L));
+        assertThat(results.stream().filter(r -> r.isMissing()).count(), equalTo(1L));
+        assertThat(results.stream().filter(r -> r.isMissing()).map(r -> r.getLedgerId()).findFirst().get(),
+                   equalTo(0xdeadL));
+
+        verify(storage, times(1)).clearLimboState(0xf00L);
+        verify(storage, times(0)).clearLimboState(0xdeadL);
+    }
+
+    @Test
+    public void testRecoverLimboClearStateFailure() throws Exception {
+        MockLedgerManager lm = new MockLedgerManager();
+        ServerConfiguration conf = serverConf();
+        BookieId bookieId = BookieImpl.getBookieId(conf);
+        MockLedgerStorage storage = spy(new MockLedgerStorage() {
+                @Override
+                public void clearLimboState(long ledgerId) throws IOException {
+                    if (ledgerId == 0xf00L) {
+                        throw new IOException("foobar");
+                    }
+                }
+            });
+        DataIntegrityCheckImpl impl = new DataIntegrityCheckImpl(bookieId, lm, storage,
+                                                                 mock(EntryCopier.class),
+                                                                 mock(BookKeeperAdmin.class),
+                                                                 Schedulers.io()) {
+                @Override
+                Single<LedgerMetadata> recoverLedger(long ledgerId, String runId) {
+                    return Single.just(newClosedMetadataWithEnsemble(ledgerId, -1, bookieId, bookie1).build());
+                }
+            };
+        Map<Long, LedgerMetadata> ledgers = new HashMap<>();
+        ledgers.put(0xf00L, newMetadataWithEnsemble(0xf00L, bookieId, bookie1).build());
+        storage.setMasterKey(0xf00L, PASSWD);
+        storage.setLimboState(0xf00L);
+        ledgers.put(0xdeadL, newMetadataWithEnsemble(0xdeadL, bookieId, bookie1).build());
+        storage.setMasterKey(0xdeadL, PASSWD);
+        storage.setLimboState(0xdeadL);
+
+        Set<DataIntegrityCheckImpl.LedgerResult> results = impl.checkAndRecoverLedgers(ledgers, "test").get();
+
+        verify(storage, times(0)).flush();
+    }
+
+    // TODO: what is this test?
+//    @Test
+//    public void testRecoverLimboFlushFailure() throws Exception {
+//        MockLedgerManager lm = new MockLedgerManager();
+//        ServerConfiguration conf = serverConf();
+//        BookieId bookieId = BookieImpl.getBookieId(conf);
+//        MockLedgerStorage storage = spy(new MockLedgerStorage() {
+//                @Override
+//                public void flush() throws IOException {
+//                    throw new IOException("foobar");
+//                }
+//            });
+//        DataIntegrityCheckImpl impl = new DataIntegrityCheckImpl(bookieId, lm, storage,
+//                                                                 mock(EntryCopier.class),
+//                                                                 mock(BookKeeperAdmin.class),
+//                                                                 Schedulers.io()) {
+//                @Override
+//                CompletableFuture<Long> recoverLedgerIgnoreMissing(long ledgerId) {
+//                    return CompletableFuture.completedFuture(ledgerId);
+//                }
+//            };
+//        Set<Long> ledgers = Sets.newHashSet(0xf00L, 0xdeadL);
+//
+//        try {
+//            impl.recoverLedgersInLimbo(ledgers).get();
+//            Assert.fail("Shouldn't continue on an IOException");
+//        } catch (ExecutionException ee) {
+//            assertThat(ee.getCause(), instanceOf(IOException.class));
+//        }
+//        assertThat(results.stream().filter(r -> r.isOK()).count(), equalTo(1L));
+//        assertThat(results.stream().filter(r -> r.isOK()).map(r -> r.getLedgerId()).findFirst().get(),
+//                   equalTo(0xdeadL));
+//        assertThat(results.stream().filter(r -> r.isError()).count(), equalTo(1L));
+//        assertThat(results.stream().filter(r -> r.isError()).map(r -> r.getLedgerId()).findFirst().get(),
+//                   equalTo(0xf00L));
+//
+//        verify(storage, times(1)).clearLimboState(0xf00L);
+//        verify(storage, times(1)).clearLimboState(0xdeadL);
+//    }
+
+    @Test
+    public void testRecoverLimboManyLedgers() throws Exception {
+        MockLedgerManager lm = new MockLedgerManager();
+        ServerConfiguration conf = serverConf();
+        BookieId bookieId = BookieImpl.getBookieId(conf);
+        List<Long> cleared = new ArrayList<>();
+        MockLedgerStorage storage = spy(new MockLedgerStorage() {
+                @Override
+                public void clearLimboState(long ledgerId) {
+                    // not using spy for this because it takes 10ms per ledger to verify
+                    cleared.add(ledgerId);
+                }
+            });
+        DataIntegrityCheckImpl impl = new DataIntegrityCheckImpl(bookieId, lm, storage,
+                                                                 mock(EntryCopier.class),
+                                                                 mock(BookKeeperAdmin.class),
+                                                                 Schedulers.io()) {
+                @Override
+                Single<LedgerMetadata> recoverLedger(long ledgerId, String runId) {
+                    return Single.just(newClosedMetadataWithEnsemble(ledgerId, -1, bookieId, bookie1).build());
+                }
+            };
+        final long numLedgers = 10000;
+        long first = 1;
+        long last = first + numLedgers;
+
+        Map<Long, LedgerMetadata> ledgers = new HashMap<>();
+        for (long i = first; i < last; i++) {
+            LedgerMetadata metadata = newMetadataWithEnsemble(i, bookieId, bookie1).build();
+            ledgers.put(i, metadata);
+            storage.setMasterKey(i, metadata.getPassword());
+            storage.setLimboState(i);
+        }
+        assertThat(ledgers.size(), equalTo((int) numLedgers));
+
+        Set<DataIntegrityCheckImpl.LedgerResult> results = impl.checkAndRecoverLedgers(ledgers, "test").get();
+        assertThat(results.size(), equalTo((int) numLedgers));
+        assertThat(results.stream().filter(r -> r.isOK()).count(), equalTo(numLedgers));
+        for (DataIntegrityCheckImpl.LedgerResult r : results) {
+            assertThat(r.isOK(), equalTo(true));
+            ledgers.remove(r.getLedgerId());
+        }
+        assertThat(ledgers.isEmpty(), equalTo(true));
+
+        Set<Long> clearedSet = Sets.newHashSet(cleared);
+        assertThat(clearedSet.size(), equalTo(cleared.size()));
+        for (long l : LongStream.range(first, last).toArray()) {
+            assertThat(l, isIn(clearedSet));
+        }
+        verify(storage, times(10000)).clearLimboState(anyLong());
+    }
+
+    @Test
+    public void testRecoverLimboManyLedgersErrorOnFirst() throws Exception {
+        MockLedgerManager lm = new MockLedgerManager();
+        ServerConfiguration conf = serverConf();
+        BookieId bookieId = BookieImpl.getBookieId(conf);
+        List<Long> cleared = new ArrayList<>();
+        MockLedgerStorage storage = spy(new MockLedgerStorage() {
+                @Override
+                public void clearLimboState(long ledgerId) {
+                    // not using spy for this because it takes 10ms per ledger to verify
+                    cleared.add(ledgerId);
+                }
+            });
+
+        final long numLedgers = 100;
+        long first = 1;
+        long last = first + numLedgers;
+
+        DataIntegrityCheckImpl impl = new DataIntegrityCheckImpl(bookieId, lm, storage,
+                                                                 mock(EntryCopier.class),
+                                                                 mock(BookKeeperAdmin.class),
+                                                                 Schedulers.io()) {
+                @Override
+                Single<LedgerMetadata> recoverLedger(long ledgerId, String runId) {
+                    if (ledgerId == first) {
+                        return Single.error(
+                                new BKException.BKBookieHandleNotAvailableException());
+                    } else {
+                        return Single.just(newClosedMetadataWithEnsemble(ledgerId, -1, bookieId, bookie1).build());
+                    }
+                }
+            };
+        Map<Long, LedgerMetadata> ledgers = new HashMap<>();
+        for (long i = first; i < last; i++) {
+            LedgerMetadata metadata = newMetadataWithEnsemble(i, bookieId, bookie1).build();
+            ledgers.put(i, metadata);
+            storage.setMasterKey(i, metadata.getPassword());
+            storage.setLimboState(i);
+        }
+        assertThat(ledgers.size(), equalTo((int) numLedgers));
+
+        Set<DataIntegrityCheckImpl.LedgerResult> results = impl.checkAndRecoverLedgers(ledgers, "test").get();
+        assertThat(results.size(), equalTo((int) numLedgers));
+        assertThat(results.stream().filter(r -> r.isOK()).count(), equalTo(numLedgers - 1));
+        assertThat(results.stream().filter(r -> r.isError()).count(), equalTo(1L));
+        assertThat(results.stream().filter(r -> r.isError()).map(r -> r.getLedgerId()).findFirst().get(),
+                   equalTo(first));
+        Set<Long> clearedSet = Sets.newHashSet(cleared);
+        assertThat(clearedSet.size(), equalTo(cleared.size()));
+        for (long l : LongStream.range(first, last).toArray()) {
+            if (l == first) {
+                assertThat(l, not(isIn(clearedSet)));
+            } else {
+                assertThat(l, isIn(clearedSet));
+            }
+        }
+        verify(storage, times((int) numLedgers - 1)).clearLimboState(anyLong());
+    }
+
+    @Test
+    public void testRecoverLimboNoLedgers() throws Exception {
+        MockLedgerManager lm = new MockLedgerManager();
+        ServerConfiguration conf = serverConf();
+        BookieId bookieId = BookieImpl.getBookieId(conf);
+        List<Long> cleared = new ArrayList<>();
+        MockLedgerStorage storage = spy(new MockLedgerStorage());
+        DataIntegrityCheckImpl impl = new DataIntegrityCheckImpl(bookieId, lm, storage,
+                                                                 mock(EntryCopier.class),
+                                                                 mock(BookKeeperAdmin.class),
+                                                                 Schedulers.io()) {
+                @Override
+                Single<LedgerMetadata> recoverLedger(long ledgerId, String runId) {
+                    return Single.just(newClosedMetadataWithEnsemble(ledgerId, -1, bookieId, bookie1).build());
+                }
+            };
+        ImmutableMap<Long, LedgerMetadata> ledgers = ImmutableMap.of();
+        Set<DataIntegrityCheckImpl.LedgerResult> resolved =
+            impl.checkAndRecoverLedgers(ledgers, "test").get(10, TimeUnit.SECONDS);
+        assertThat(resolved.isEmpty(), equalTo(true));
+        verify(storage, times(0)).clearLimboState(anyLong());
+    }
+
+
+    @Test
+    public void testRecoverSingleLedgerEntriesOnLedgerIDontHave() throws Exception {
+        MockBookieClient bookieClient = spy(new MockBookieClient(executor));
+        MockLedgerManager lm = new MockLedgerManager();
+        ServerConfiguration conf = serverConf();
+        MockLedgerStorage storage = spy(new MockLedgerStorage());
+
+        EntryCopier copier = new EntryCopierImpl(bookie1, bookieClient, storage, new MockTicker());
+        DataIntegrityCheckImpl impl = new DataIntegrityCheckImpl(bookie1, lm, storage,
+                                                                 copier,
+                                                                 mock(BookKeeperAdmin.class),
+                                                                 Schedulers.io());
+        long id1 = 0xdeadL;
+        LedgerMetadata metadata1 = newClosedMetadataWithEnsemble(id1, 1000, bookie3, bookie2).build();
+        bookieClient.getMockBookies().seedLedger(id1, metadata1);
+
+        assertThat(storage.ledgerExists(id1), equalTo(false));
+
+        TestObserver<Long> observer = TestObserver.create();
+        impl.checkAndRecoverLedgerEntries(id1, metadata1, "test").subscribe(observer);
+        observer.await().assertNoErrors();
+
+        assertThat(storage.ledgerExists(id1), equalTo(true)); // because we passed it in
+        for (long i = 0; i <= metadata1.getLastEntryId(); i++) {
+            assertThat(storage.entryExists(id1, i), equalTo(false));
+        }
+    }
+
+    @Test
+    public void testRecoverSingleLedgerNotClosedOneEnsemble() throws Exception {
+        MockBookieClient bookieClient = spy(new MockBookieClient(executor));
+        MockLedgerManager lm = new MockLedgerManager();
+        ServerConfiguration conf = serverConf();
+        MockLedgerStorage storage = spy(new MockLedgerStorage());
+
+        EntryCopier copier = new EntryCopierImpl(bookie1, bookieClient, storage, new MockTicker());
+        DataIntegrityCheckImpl impl = new DataIntegrityCheckImpl(bookie1, lm, storage,
+                                                                 copier,
+                                                                 mock(BookKeeperAdmin.class),
+                                                                 Schedulers.io());
+        long id1 = 0xdeadL;
+        LedgerMetadata metadata1 = newMetadataWithEnsemble(id1, bookie1, bookie2).build();
+        bookieClient.getMockBookies().seedLedgerForBookie(bookie2, id1, metadata1);
+
+        assertThat(storage.ledgerExists(id1), equalTo(false));
+
+        TestObserver<Long> observer = TestObserver.create();
+        impl.checkAndRecoverLedgerEntries(id1, metadata1, "test").subscribe(observer);
+        observer.await().assertNoErrors();
+
+        LedgerMetadata md1 = newMetadataWithEnsemble(id1, bookie1).build();
+        assertThat(storage.ledgerExists(id1), equalTo(false));
+    }
+
+    @Test
+    public void testRecoverSingleLedgerNoClosedMultiEnsembleBookieInClosed() throws Exception {
+        MockBookieClient bookieClient = spy(new MockBookieClient(executor));
+        MockLedgerManager lm = new MockLedgerManager();
+        ServerConfiguration conf = serverConf();
+        MockLedgerStorage storage = spy(new MockLedgerStorage());
+
+        EntryCopier copier = new EntryCopierImpl(bookie1, bookieClient, storage, new MockTicker());
+        DataIntegrityCheckImpl impl = new DataIntegrityCheckImpl(bookie1, lm, storage,
+                                                                 copier,
+                                                                 mock(BookKeeperAdmin.class),
+                                                                 Schedulers.io());
+        long id1 = 0xdeadL;
+        LedgerMetadata metadata1 = newMetadataWithEnsemble(id1, bookie1, bookie2)
+            .newEnsembleEntry(10L, Lists.newArrayList(bookie3, bookie2)).build();
+        bookieClient.getMockBookies().seedLedgerForBookie(bookie2, id1, metadata1);
+        bookieClient.getMockBookies().seedLedgerForBookie(bookie3, id1, metadata1);
+
+        assertThat(storage.ledgerExists(id1), equalTo(false));
+
+        TestObserver<Long> observer = TestObserver.create();
+        impl.checkAndRecoverLedgerEntries(id1, metadata1, "test").subscribe(observer);
+        observer.await().assertNoErrors();
+
+        assertThat(storage.ledgerExists(id1), equalTo(true));
+        for (long e = 0; e < 10; e++) {
+            assertThat(storage.entryExists(id1, e), equalTo(true));
+        }
+        assertThat(storage.entryExists(id1, 10), equalTo(false));
+    }
+
+    @Test
+    public void testRecoverSingleLedgerNotClosedMultiEnsembleBookieInFinal() throws Exception {
+        MockBookieClient bookieClient = spy(new MockBookieClient(executor));
+        MockLedgerManager lm = new MockLedgerManager();
+        ServerConfiguration conf = serverConf();
+        MockLedgerStorage storage = spy(new MockLedgerStorage());
+
+        EntryCopier copier = new EntryCopierImpl(bookie1, bookieClient, storage, new MockTicker());
+        DataIntegrityCheckImpl impl = new DataIntegrityCheckImpl(bookie1, lm, storage,
+                                                                 copier,
+                                                                 mock(BookKeeperAdmin.class),
+                                                                 Schedulers.io());
+        long id1 = 0xdeadL;
+        LedgerMetadata metadata1 = newMetadataWithEnsemble(id1, bookie3, bookie2)
+            .newEnsembleEntry(10L, Lists.newArrayList(bookie1, bookie2)).build();
+        bookieClient.getMockBookies().seedLedgerForBookie(bookie2, id1, metadata1);
+        bookieClient.getMockBookies().seedLedgerForBookie(bookie3, id1, metadata1);
+
+        assertThat(storage.ledgerExists(id1), equalTo(false));
+
+        TestObserver<Long> observer = TestObserver.create();
+        impl.checkAndRecoverLedgerEntries(id1, metadata1, "test").subscribe(observer);
+        observer.await().assertNoErrors();
+
+        assertThat(storage.ledgerExists(id1), equalTo(true));
+    }
+
+    @Test
+    public void testRecoverSingleLedgerLargeEnsembleStriped() throws Exception {
+
+        MockBookieClient bookieClient = spy(new MockBookieClient(executor));
+        MockLedgerManager lm = new MockLedgerManager();
+        ServerConfiguration conf = serverConf();
+        MockLedgerStorage storage = spy(new MockLedgerStorage());
+
+        EntryCopier copier = new EntryCopierImpl(bookie4, bookieClient, storage, new MockTicker());
+        DataIntegrityCheckImpl impl = new DataIntegrityCheckImpl(bookie4, lm, storage,
+                                                                 copier,
+                                                                 mock(BookKeeperAdmin.class),
+                                                                 Schedulers.io());
+        long id1 = 0xdeadL;
+        LedgerMetadata metadata1 = LedgerMetadataBuilder.create()
+            .withId(id1)
+            .withPassword(new byte[0])
+            .withDigestType(DigestType.CRC32C)
+            .withEnsembleSize(5)
+            .withWriteQuorumSize(2)
+            .withAckQuorumSize(2)
+            .newEnsembleEntry(0, Lists.newArrayList(bookie1, bookie2, bookie3, bookie4, bookie5))
+            .withClosedState().withLastEntryId(10).withLength(1000)
+            .build();
+        bookieClient.getMockBookies().seedLedgerForBookie(bookie1, id1, metadata1);
+        bookieClient.getMockBookies().seedLedgerForBookie(bookie2, id1, metadata1);
+        bookieClient.getMockBookies().seedLedgerForBookie(bookie3, id1, metadata1);
+        bookieClient.getMockBookies().seedLedgerForBookie(bookie5, id1, metadata1);
+
+        assertThat(storage.ledgerExists(id1), equalTo(false));
+
+        TestObserver<Long> observer = TestObserver.create();
+        impl.checkAndRecoverLedgerEntries(id1, metadata1, "test").subscribe(observer);
+        observer.await().assertNoErrors();
+
+        assertThat(storage.ledgerExists(id1), equalTo(true));
+        assertThat(storage.entryExists(id1, 0), equalTo(false));
+        assertThat(storage.entryExists(id1, 1), equalTo(false));
+        assertThat(storage.entryExists(id1, 2), equalTo(true));
+        assertThat(storage.entryExists(id1, 3), equalTo(true));
+        assertThat(storage.entryExists(id1, 4), equalTo(false));
+        assertThat(storage.entryExists(id1, 5), equalTo(false));
+        assertThat(storage.entryExists(id1, 6), equalTo(false));
+        assertThat(storage.entryExists(id1, 7), equalTo(true));
+        assertThat(storage.entryExists(id1, 8), equalTo(true));
+        assertThat(storage.entryExists(id1, 9), equalTo(false));
+        assertThat(storage.entryExists(id1, 10), equalTo(false));
+    }
+
+    @Test
+    public void testRecoverSingleLedgerEntriesOnlyEntriesNeeded() throws Exception {
+        MockBookieClient bookieClient = spy(new MockBookieClient(executor));
+        MockLedgerManager lm = new MockLedgerManager();
+        ServerConfiguration conf = serverConf();
+        MockLedgerStorage storage = spy(new MockLedgerStorage());
+
+        EntryCopier copier = new EntryCopierImpl(bookie1, bookieClient, storage, new MockTicker());
+        DataIntegrityCheckImpl impl = new DataIntegrityCheckImpl(bookie1, lm, storage,
+                                                                 copier,
+                                                                 mock(BookKeeperAdmin.class),
+                                                                 Schedulers.io());
+        long id1 = 0xdeadL;
+        LedgerMetadata metadata1 = newClosedMetadataWithEnsemble(id1, 1000, bookie3, bookie2)
+            .newEnsembleEntry(10, Lists.newArrayList(bookie1, bookie2))
+            .newEnsembleEntry(100, Lists.newArrayList(bookie3, bookie2)).build();
+        bookieClient.getMockBookies().seedLedgerForBookie(bookie2, id1, metadata1);
+        bookieClient.getMockBookies().seedLedgerForBookie(bookie3, id1, metadata1);
+
+        assertThat(storage.ledgerExists(id1), equalTo(false));
+
+        TestObserver<Long> observer = TestObserver.create();
+        impl.checkAndRecoverLedgerEntries(id1, metadata1, "test").subscribe(observer);
+        observer.await().assertNoErrors();
+
+        assertThat(storage.ledgerExists(id1), equalTo(true));
+        assertThat(storage.entryExists(id1, 9), equalTo(false));
+        for (long e = 10; e < 100; e++) {
+            assertThat(storage.entryExists(id1, e), equalTo(true));
+        }
+        assertThat(storage.entryExists(id1, 100), equalTo(false));
+    }
+
+    @Test
+    public void testRecoverSingleLedgerEntriesOnlyEntriesNeededEverySecond() throws Exception {
+        MockBookieClient bookieClient = spy(new MockBookieClient(executor));
+        MockLedgerManager lm = new MockLedgerManager();
+        ServerConfiguration conf = serverConf();
+        MockLedgerStorage storage = spy(new MockLedgerStorage());
+
+        EntryCopier copier = new EntryCopierImpl(bookie1, bookieClient, storage, new MockTicker());
+        DataIntegrityCheckImpl impl = new DataIntegrityCheckImpl(bookie1, lm, storage,
+                                                                 copier,
+                                                                 mock(BookKeeperAdmin.class),
+                                                                 Schedulers.io());
+        long id1 = 0xdeadL;
+        LedgerMetadata metadata1 = newClosedMetadataWithEnsemble(id1, 1000, bookie1, bookie2).build();
+        bookieClient.getMockBookies().seedLedgerForBookie(bookie2, id1, metadata1);
+        long added = 0;
+        storage.setMasterKey(id1, PASSWD);
+        for (long e = 0; e <= metadata1.getLastEntryId(); e++) {
+            if (e % 2 == 0) {
+                storage.addEntry(bookieClient.getMockBookies().generateEntry(id1, e, e - 1));
+                added++;
+            }
+        }
+        assertThat(storage.ledgerExists(id1), equalTo(true));
+
+        TestObserver<Long> observer = TestObserver.create();
+        impl.checkAndRecoverLedgerEntries(id1, metadata1, "test").subscribe(observer);
+        observer.await().assertNoErrors();
+
+        for (long e = 0; e <= metadata1.getLastEntryId(); e++) {
+            if (e % 2 == 0) {
+                verify(bookieClient, times(0)).readEntry(anyObject(), eq(id1), eq(e),
+                                                         anyObject(), anyObject(), anyInt());
+            }
+            if (e % 2 == 1) {
+                verify(bookieClient, times(1)).readEntry(anyObject(), eq(id1), eq(e),
+                                                         anyObject(), anyObject(), anyInt());
+            }
+
+            assertThat(storage.entryExists(id1, e), equalTo(true));
+        }
+    }
+
+    @Test
+    public void testRecoverSingleLedgerErrorAtStart() throws Exception {
+        MockBookieClient bookieClient = spy(new MockBookieClient(executor));
+        MockLedgerManager lm = new MockLedgerManager();
+        ServerConfiguration conf = serverConf();
+        MockLedgerStorage storage = spy(new MockLedgerStorage());
+
+        EntryCopier copier = new EntryCopierImpl(bookie1, bookieClient, storage, new MockTicker());
+        DataIntegrityCheckImpl impl = new DataIntegrityCheckImpl(bookie1, lm, storage,
+                                                                 copier,
+                                                                 mock(BookKeeperAdmin.class),
+                                                                 Schedulers.io());
+        long id1 = 0xdeadL;
+        LedgerMetadata metadata1 = newClosedMetadataWithEnsemble(id1, 1000, bookie1, bookie2).build();
+
+        // only seed for ledger1 & ledger3
+        bookieClient.getMockBookies().seedLedgerForBookie(bookie2, id1, metadata1);
+        bookieClient.setPreReadHook((bookie, ledger, entry) -> {
+                if (entry == 0L) {
+                    return FutureUtils.exception(new BKException.BKReadException());
+                } else {
+                    return CompletableFuture.completedFuture(null);
+                }
+            });
+
+        TestObserver<Long> observer = TestObserver.create();
+        impl.checkAndRecoverLedgerEntries(id1, metadata1, "test").subscribe(observer);
+        observer.await().assertError((t) -> {
+                return t instanceof BKException.BKReadException;
+            });
+        assertThat(storage.entryExists(id1, 0), equalTo(false));
+        for (long e = 1; e <= metadata1.getLastEntryId(); e++) {
+            assertThat(storage.entryExists(id1, e), equalTo(true));
+        }
+    }
+
+    @Test
+    public void testRecoverSingleLedgerErrorEverySecond() throws Exception {
+        MockBookieClient bookieClient = spy(new MockBookieClient(executor));
+        MockLedgerManager lm = new MockLedgerManager();
+        ServerConfiguration conf = serverConf();
+        MockLedgerStorage storage = spy(new MockLedgerStorage());
+
+        EntryCopier copier = new EntryCopierImpl(bookie1, bookieClient, storage, new MockTicker());
+        DataIntegrityCheckImpl impl = new DataIntegrityCheckImpl(bookie1, lm, storage,
+                                                                 copier,
+                                                                 mock(BookKeeperAdmin.class),
+                                                                 Schedulers.io());
+        long id1 = 0xdeadL;
+        LedgerMetadata metadata1 = newClosedMetadataWithEnsemble(id1, 1000, bookie1, bookie2).build();
+
+        // only seed for ledger1 & ledger3
+        bookieClient.getMockBookies().seedLedgerForBookie(bookie2, id1, metadata1);
+        bookieClient.setPreReadHook((bookie, ledger, entry) -> {
+                if (entry % 2 == 0) {
+                    return FutureUtils.exception(new BKException.BKReadException());
+                } else {
+                    return CompletableFuture.completedFuture(null);
+                }
+            });
+
+        TestObserver<Long> observer = TestObserver.create();
+        impl.checkAndRecoverLedgerEntries(id1, metadata1, "test").subscribe(observer);
+        observer.await().assertError((t) -> {
+                if (t instanceof CompositeException) {
+                    CompositeException e = (CompositeException) t;
+                    for (Throwable t2 : e.getExceptions()) {
+                        if (!(t2 instanceof BKException.BKReadException)) {
+                            return false;
+                        }
+                    }
+                    return e.getExceptions().size() == 500;
+                } else {
+                    return false;
+                }
+            });
+        for (long e = 0; e <= metadata1.getLastEntryId(); e++) {
+            if (e % 2 == 0) {
+                assertThat(storage.entryExists(id1, e), equalTo(false));
+            } else {
+                assertThat(storage.entryExists(id1, e), equalTo(true));
+            }
+        }
+    }
+
+    @Test
+    public void testRecoverSingleLedgerErrorOneOnStore() throws Exception {
+        MockBookieClient bookieClient = spy(new MockBookieClient(executor));
+        MockLedgerManager lm = new MockLedgerManager();
+        ServerConfiguration conf = serverConf();
+        MockLedgerStorage storage = spy(new MockLedgerStorage() {
+                @Override
+                public long addEntry(ByteBuf entry) throws IOException, BookieException {
+                    long entryId = extractEntryId(entry);
+                    if (entryId > 10 && entryId <= 100) {
+                        throw new IOException("Don't feel like storing these");
+                    }
+                    return super.addEntry(entry);
+                }
+            });
+
+        EntryCopier copier = new EntryCopierImpl(bookie1, bookieClient, storage, new MockTicker());
+        DataIntegrityCheckImpl impl = new DataIntegrityCheckImpl(bookie1, lm, storage,
+                                                                 copier,
+                                                                 mock(BookKeeperAdmin.class),
+                                                                 Schedulers.io());
+        long id1 = 0xdeadL;
+        LedgerMetadata metadata1 = newClosedMetadataWithEnsemble(id1, 1000, bookie1, bookie2).build();
+        // only seed for ledger1 & ledger3
+        bookieClient.getMockBookies().seedLedgerForBookie(bookie2, id1, metadata1);
+
+        TestObserver<Long> observer = TestObserver.create();
+        impl.checkAndRecoverLedgerEntries(id1, metadata1, "test").subscribe(observer);
+        observer.await().assertError((t) -> {
+                if (t instanceof CompositeException) {
+                    CompositeException e = (CompositeException) t;
+                    for (Throwable t2 : e.getExceptions()) {
+                        boolean failStore = t2 instanceof IOException;
+                        if (!failStore) {
+                            return false;
+                        }
+                    }
+                    return e.getExceptions().size() == 90;
+                } else {
+                    return false;
+                }
+            });
+        for (long e = 0; e <= 10; e++) {
+            assertThat(storage.entryExists(id1, e), equalTo(true));
+        }
+        for (long e = 11; e <= 100; e++) {
+            assertThat(storage.entryExists(id1, e), equalTo(false));
+        }
+        for (long e = 101; e <= metadata1.getLastEntryId(); e++) {
+            assertThat(storage.entryExists(id1, e), equalTo(true));
+        }
+    }
+
+    @Test
+    public void testRecoverMultiLedgers() throws Exception {
+        MockBookieClient bookieClient = spy(new MockBookieClient(executor));
+        MockLedgerManager lm = new MockLedgerManager();
+        ServerConfiguration conf = serverConf();
+        MockLedgerStorage storage = spy(new MockLedgerStorage());
+
+        EntryCopier copier = new EntryCopierImpl(bookie1, bookieClient, storage, new MockTicker());
+        DataIntegrityCheckImpl impl = new DataIntegrityCheckImpl(bookie1, lm, storage,
+                                                                 copier,
+                                                                 mock(BookKeeperAdmin.class),
+                                                                 Schedulers.io());
+        long id1 = 0xdeadL;
+        long id2 = 0xbedeL;
+        long id3 = 0xbebeL;
+        LedgerMetadata metadata1 = newClosedMetadataWithEnsemble(id1, 1000, bookie1, bookie2).build();
+        LedgerMetadata metadata2 = newClosedMetadataWithEnsemble(id2, 1000, bookie1, bookie3).build();
+        LedgerMetadata metadata3 = newClosedMetadataWithEnsemble(id3, 1000, bookie1, bookie3).build();
+
+        bookieClient.getMockBookies().seedLedgerForBookie(bookie2, id1, metadata1);
+        bookieClient.getMockBookies().seedLedgerForBookie(bookie3, id2, metadata2);
+        bookieClient.getMockBookies().seedLedgerForBookie(bookie3, id3, metadata3);
+
+        assertThat(storage.ledgerExists(id1), equalTo(false));
+        assertThat(storage.ledgerExists(id2), equalTo(false));
+        assertThat(storage.ledgerExists(id3), equalTo(false));
+        Map<Long, LedgerMetadata> ledgers = ImmutableMap.of(
+                id1, metadata1, id2, metadata2, id3, metadata3);
+        Set<DataIntegrityCheckImpl.LedgerResult> resolved =
+            impl.checkAndRecoverLedgers(ledgers, "test").get(10, TimeUnit.SECONDS);
+        assertThat(resolved.stream().filter(r -> r.isOK()).count(), equalTo(3L));
+        assertThat(resolved.stream().filter(r -> r.isOK()).map(r -> r.getLedgerId())
+                   .collect(Collectors.toSet()), containsInAnyOrder(id1, id2, id3));
+        for (long e = 0; e <= metadata1.getLastEntryId(); e++) {
+            assertThat(storage.entryExists(id1, e), equalTo(true));
+            assertThat(storage.entryExists(id2, e), equalTo(true));
+            assertThat(storage.entryExists(id3, e), equalTo(true));
+        }
+    }
+
+    @Test
+    public void testRecoverMultiLedgersOneUnavailable() throws Exception {
+        MockBookieClient bookieClient = spy(new MockBookieClient(executor));
+        MockLedgerManager lm = new MockLedgerManager();
+        ServerConfiguration conf = serverConf();
+        MockLedgerStorage storage = spy(new MockLedgerStorage());
+
+        EntryCopier copier = new EntryCopierImpl(bookie1, bookieClient, storage, new MockTicker());
+        DataIntegrityCheckImpl impl = new DataIntegrityCheckImpl(bookie1, lm, storage,
+                                                                 copier,
+                                                                 mock(BookKeeperAdmin.class),
+                                                                 Schedulers.io());
+        long id1 = 0xdeadL;
+        long id2 = 0xbedeL;
+        long id3 = 0xbebeL;
+        LedgerMetadata metadata1 = newClosedMetadataWithEnsemble(id1, 1000, bookie1, bookie2).build();
+        LedgerMetadata metadata2 = newClosedMetadataWithEnsemble(id2, 1000, bookie1, bookie3).build();
+        LedgerMetadata metadata3 = newClosedMetadataWithEnsemble(id3, 1000, bookie1, bookie3).build();
+
+        // id2 will be unavailable because there's no entries
+        bookieClient.getMockBookies().seedLedgerForBookie(bookie2, id1, metadata1);
+        bookieClient.getMockBookies().seedLedgerForBookie(bookie3, id3, metadata3);
+
+        assertThat(storage.ledgerExists(id1), equalTo(false));
+        assertThat(storage.ledgerExists(id2), equalTo(false));
+        assertThat(storage.ledgerExists(id3), equalTo(false));
+
+        Map<Long, LedgerMetadata> ledgers = ImmutableMap.of(
+                id1, metadata1, id2, metadata2, id3, metadata3);
+        Set<DataIntegrityCheckImpl.LedgerResult> resolved =
+            impl.checkAndRecoverLedgers(ledgers, "test").get(10, TimeUnit.SECONDS);
+        assertThat(resolved.stream().filter(r -> r.isOK()).count(), equalTo(2L));
+        assertThat(resolved.stream().filter(r -> r.isError()).count(), equalTo(1L));
+        assertThat(resolved.stream().filter(r -> r.isOK()).map(r -> r.getLedgerId())
+                   .collect(Collectors.toSet()), containsInAnyOrder(id1, id3));
+        for (long e = 0; e <= metadata1.getLastEntryId(); e++) {
+            assertThat(storage.entryExists(id1, e), equalTo(true));
+            assertThat(storage.entryExists(id3, e), equalTo(true));
+        }
+    }
+
+    @Test
+    public void testRecoverMultiLedgersOneFailsToWriteLocally() throws Exception {
+        long id1 = 0xdeadL;
+        long id2 = 0xbedeL;
+        long id3 = 0xbebeL;
+
+        MockBookieClient bookieClient = spy(new MockBookieClient(executor));
+        MockLedgerManager lm = new MockLedgerManager();
+        ServerConfiguration conf = serverConf();
+        MockLedgerStorage storage = spy(new MockLedgerStorage() {
+                @Override
+                public long addEntry(ByteBuf entry) throws IOException, BookieException {
+                    if (extractLedgerId(entry) == id1
+                        && extractEntryId(entry) == 3) {
+                        throw new IOException("Don't feel like storing this");
+                    }
+                    return super.addEntry(entry);
+                }
+            });
+
+        EntryCopier copier = new EntryCopierImpl(bookie1, bookieClient, storage, new MockTicker());
+        DataIntegrityCheckImpl impl = new DataIntegrityCheckImpl(bookie1, lm, storage,
+                                                                 copier,
+                                                                 mock(BookKeeperAdmin.class),
+                                                                 Schedulers.io());
+        LedgerMetadata metadata1 = newClosedMetadataWithEnsemble(id1, 1000, bookie1, bookie2).build();
+        LedgerMetadata metadata2 = newClosedMetadataWithEnsemble(id2, 1000, bookie1, bookie3).build();
+        LedgerMetadata metadata3 = newClosedMetadataWithEnsemble(id3, 1000, bookie1, bookie3).build();
+
+        bookieClient.getMockBookies().seedLedgerForBookie(bookie2, id1, metadata1);
+        bookieClient.getMockBookies().seedLedgerForBookie(bookie3, id2, metadata2);
+        bookieClient.getMockBookies().seedLedgerForBookie(bookie3, id3, metadata3);
+
+        assertThat(storage.ledgerExists(id1), equalTo(false));
+        assertThat(storage.ledgerExists(id2), equalTo(false));
+        assertThat(storage.ledgerExists(id3), equalTo(false));
+
+        Map<Long, LedgerMetadata> ledgers = ImmutableMap.of(
+                id1, metadata1, id2, metadata2, id3, metadata3);
+
+        Set<DataIntegrityCheckImpl.LedgerResult> resolved =
+            impl.checkAndRecoverLedgers(ledgers, "test").get(10, TimeUnit.SECONDS);
+        assertThat(resolved.stream().filter(r -> r.isOK()).count(), equalTo(2L));
+        assertThat(resolved.stream().filter(r -> r.isOK())
+                   .map(r -> r.getLedgerId()).collect(Collectors.toSet()),
+                   containsInAnyOrder(id2, id3));
+        assertThat(resolved.stream().filter(r -> r.isError())
+                   .map(r -> r.getLedgerId()).collect(Collectors.toSet()),
+                   containsInAnyOrder(id1));
+
+        for (long e = 0; e <= metadata1.getLastEntryId(); e++) {
+            assertThat(storage.entryExists(id1, e), equalTo(e != 3));
+            assertThat(storage.entryExists(id2, e), equalTo(true));
+            assertThat(storage.entryExists(id3, e), equalTo(true));
+        }
+    }
+
+    @Test
+    public void testRecoverMultiLedgersAllUnavailable() throws Exception {
+        MockBookieClient bookieClient = spy(new MockBookieClient(executor));
+        MockLedgerManager lm = new MockLedgerManager();
+        ServerConfiguration conf = serverConf();
+        MockLedgerStorage storage = spy(new MockLedgerStorage());
+
+        EntryCopier copier = new EntryCopierImpl(bookie1, bookieClient, storage, new MockTicker());
+        DataIntegrityCheckImpl impl = new DataIntegrityCheckImpl(bookie1, lm, storage,
+                                                                 copier,
+                                                                 mock(BookKeeperAdmin.class),
+                                                                 Schedulers.io());
+        long id1 = 0xdeadL;
+        long id2 = 0xbedeL;
+        long id3 = 0xbebeL;
+        LedgerMetadata metadata1 = newClosedMetadataWithEnsemble(id1, 1000, bookie1, bookie2).build();
+        LedgerMetadata metadata2 = newClosedMetadataWithEnsemble(id2, 1000, bookie1, bookie3).build();
+        LedgerMetadata metadata3 = newClosedMetadataWithEnsemble(id3, 1000, bookie1, bookie3).build();
+
+        assertThat(storage.ledgerExists(id1), equalTo(false));
+        assertThat(storage.ledgerExists(id2), equalTo(false));
+        assertThat(storage.ledgerExists(id3), equalTo(false));
+
+        Map<Long, LedgerMetadata> ledgers = ImmutableMap.of(
+                id1, metadata1, id2, metadata2, id3, metadata3);
+
+        Set<DataIntegrityCheckImpl.LedgerResult> resolved =
+            impl.checkAndRecoverLedgers(ledgers, "test").get(10, TimeUnit.SECONDS);
+        assertThat(resolved.stream().filter(r -> r.isOK()).count(), equalTo(0L));
+        assertThat(resolved.stream().filter(r -> r.isError()).count(), equalTo(3L));
+        assertThat(storage.ledgerExists(id1), equalTo(true));
+        assertThat(storage.entryExists(id1, 0), equalTo(false));
+        assertThat(storage.ledgerExists(id2), equalTo(true));
+        assertThat(storage.entryExists(id2, 0), equalTo(false));
+        assertThat(storage.ledgerExists(id3), equalTo(true));
+        assertThat(storage.entryExists(id3, 0), equalTo(false));
+    }
+
+    @Test
+    public void testEnsemblesContainBookie() throws Exception {
+        LedgerMetadata md1 = newMetadataWithEnsemble(1, bookie1).build();
+        assertThat(DataIntegrityCheckImpl.ensemblesContainBookie(md1, bookie1), equalTo(true));
+        assertThat(DataIntegrityCheckImpl.ensemblesContainBookie(md1, bookie2), equalTo(false));
+        assertThat(DataIntegrityCheckImpl.ensemblesContainBookie(md1, bookie3), equalTo(false));
+
+        LedgerMetadata md2 = newMetadataWithEnsemble(2, bookie1, bookie2)
+            .newEnsembleEntry(1, Lists.newArrayList(bookie2, bookie3)).build();
+        assertThat(DataIntegrityCheckImpl.ensemblesContainBookie(md2, bookie1), equalTo(true));
+        assertThat(DataIntegrityCheckImpl.ensemblesContainBookie(md2, bookie2), equalTo(true));
+        assertThat(DataIntegrityCheckImpl.ensemblesContainBookie(md2, bookie3), equalTo(true));
+
+        LedgerMetadata md3 = newMetadataWithEnsemble(3, bookie1, bookie2)
+            .newEnsembleEntry(1, Lists.newArrayList(bookie2, bookie1)).build();
+        assertThat(DataIntegrityCheckImpl.ensemblesContainBookie(md3, bookie1), equalTo(true));
+        assertThat(DataIntegrityCheckImpl.ensemblesContainBookie(md3, bookie2), equalTo(true));
+        assertThat(DataIntegrityCheckImpl.ensemblesContainBookie(md3, bookie3), equalTo(false));
+    }
+
+    @Test
+    public void testMetadataCacheLoad() throws Exception {
+        MockBookieClient bookieClient = spy(new MockBookieClient(executor));
+        MockLedgerManager lm = new MockLedgerManager();
+        ServerConfiguration conf = serverConf();
+        MockLedgerStorage storage = spy(new MockLedgerStorage());
+
+        EntryCopier copier = new EntryCopierImpl(bookie1, bookieClient, storage, new MockTicker());
+        DataIntegrityCheckImpl impl = new DataIntegrityCheckImpl(bookie1, lm, storage,
+                                                                 copier,
+                                                                 mock(BookKeeperAdmin.class),
+                                                                 Schedulers.io());
+        long id1 = 0xdeadL;
+        long id2 = 0xbedeL;
+        long id3 = 0xbebeL;
+        LedgerMetadata metadata1 = newClosedMetadataWithEnsemble(id1, 1000, bookie1, bookie2).build();
+        LedgerMetadata metadata2 = newClosedMetadataWithEnsemble(id2, 1000, bookie1, bookie3).build();
+        LedgerMetadata metadata3 = newClosedMetadataWithEnsemble(id3, 1000, bookie1, bookie3).build();
+
+        lm.createLedgerMetadata(id1, metadata1).get();
+        lm.createLedgerMetadata(id2, metadata2).get();
+        lm.createLedgerMetadata(id3, metadata3).get();
+
+        Map<Long, LedgerMetadata> ledgers = impl.getCachedOrReadMetadata("test").get();
+        assertThat(ledgers.keySet(), containsInAnyOrder(id1, id2, id3));
+    }
+
+    @Test
+    public void testFullCheckCacheLoadAndProcessIfEmpty() throws Exception {
+        MockBookieClient bookieClient = spy(new MockBookieClient(executor));
+        MockLedgerManager lm = new MockLedgerManager();
+        ServerConfiguration conf = serverConf();
+        MockLedgerStorage storage = spy(new MockLedgerStorage());
+
+        EntryCopier copier = new EntryCopierImpl(bookie1, bookieClient, storage, new MockTicker());
+        DataIntegrityCheckImpl impl = new DataIntegrityCheckImpl(bookie1, lm, storage,
+                                                                 copier,
+                                                                 mock(BookKeeperAdmin.class),
+                                                                 Schedulers.io());
+        long id1 = 0xdeadL;
+        long id2 = 0xbedeL;
+        long id3 = 0xbebeL;
+        LedgerMetadata metadata1 = newClosedMetadataWithEnsemble(id1, 1000, bookie1, bookie2).build();
+        LedgerMetadata metadata2 = newClosedMetadataWithEnsemble(id2, 1000, bookie1, bookie3).build();
+        LedgerMetadata metadata3 = newClosedMetadataWithEnsemble(id3, 1000, bookie1, bookie3).build();
+
+        bookieClient.getMockBookies().seedLedgerForBookie(bookie2, id1, metadata1);
+        bookieClient.getMockBookies().seedLedgerForBookie(bookie3, id2, metadata2);
+        bookieClient.getMockBookies().seedLedgerForBookie(bookie3, id3, metadata3);
+
+        lm.createLedgerMetadata(id1, metadata1).get();
+        lm.createLedgerMetadata(id2, metadata2).get();
+        lm.createLedgerMetadata(id3, metadata3).get();
+
+        assertThat(storage.ledgerExists(id1), equalTo(false));
+        assertThat(storage.ledgerExists(id2), equalTo(false));
+        assertThat(storage.ledgerExists(id3), equalTo(false));
+
+        impl.runFullCheck().get();
+
+        assertThat(storage.ledgerExists(id1), equalTo(true));
+        assertThat(storage.ledgerExists(id2), equalTo(true));
+        assertThat(storage.ledgerExists(id3), equalTo(true));
+    }
+
+    @Test
+    public void testFullCheckCacheLoadAndProcessSomeInLimbo() throws Exception {
+        MockBookieClient bookieClient = spy(new MockBookieClient(executor));
+        MockLedgerManager lm = new MockLedgerManager();
+        ServerConfiguration conf = serverConf();
+        MockLedgerStorage storage = spy(new MockLedgerStorage());
+
+        EntryCopier copier = new EntryCopierImpl(bookie1, bookieClient, storage, new MockTicker());
+        long id1 = 0xdeadL;
+        long id2 = 0xbedeL;
+        long id3 = 0xbebeL;
+        LedgerMetadata metadata1 = newClosedMetadataWithEnsemble(id1, 1000, bookie1, bookie2).build();
+        LedgerMetadata metadata2 = newClosedMetadataWithEnsemble(id2, 1000, bookie1, bookie3).build();
+        LedgerMetadata metadata3 = newMetadataWithEnsemble(id3, bookie1, bookie3).build();
+        LedgerMetadata metadata3closed = newClosedMetadataWithEnsemble(id3, 1000, bookie1, bookie3).build();
+
+        DataIntegrityCheckImpl impl = new DataIntegrityCheckImpl(bookie1, lm, storage,
+                                                                 copier,
+                                                                 mock(BookKeeperAdmin.class),
+                                                                 Schedulers.io()) {
+                @Override
+                Single<LedgerMetadata> recoverLedger(long ledgerId, String runId) {
+                    return Single.just(metadata3closed);
+                }
+            };
+
+        bookieClient.getMockBookies().seedLedgerForBookie(bookie2, id1, metadata1);
+        bookieClient.getMockBookies().seedLedgerForBookie(bookie3, id2, metadata2);
+        bookieClient.getMockBookies().seedLedgerForBookie(bookie3, id3, metadata3closed);
+
+        lm.createLedgerMetadata(id1, metadata1).get();
+        lm.createLedgerMetadata(id2, metadata2).get();
+        lm.createLedgerMetadata(id3, metadata3).get();
+
+        assertThat(storage.ledgerExists(id1), equalTo(false));
+        assertThat(storage.ledgerExists(id2), equalTo(false));
+        assertThat(storage.ledgerExists(id3), equalTo(false));
+        storage.setMasterKey(id3, PASSWD);
+        storage.setLimboState(id3);
+        assertThat(storage.hasLimboState(id3), equalTo(true));
+
+        storage.setStorageStateFlag(StorageState.NEEDS_INTEGRITY_CHECK);
+        assertThat(StorageState.NEEDS_INTEGRITY_CHECK,
+                   isIn(storage.getStorageStateFlags()));
+
+        impl.runFullCheck().get();
+
+        assertThat(StorageState.NEEDS_INTEGRITY_CHECK,
+                   not(isIn(storage.getStorageStateFlags())));
+
+        assertThat(storage.ledgerExists(id1), equalTo(true));
+        assertThat(storage.ledgerExists(id2), equalTo(true));
+        assertThat(storage.ledgerExists(id3), equalTo(true));
+        assertThat(storage.hasLimboState(id3), equalTo(false));
+    }
+
+    @Test
+    public void testFullCheckInLimboRecoveryFailsFirstTime() throws Exception {
+        MockBookieClient bookieClient = spy(new MockBookieClient(executor));
+        MockLedgerManager lm = new MockLedgerManager();
+        ServerConfiguration conf = serverConf();
+        MockLedgerStorage storage = spy(new MockLedgerStorage());
+
+        EntryCopier copier = new EntryCopierImpl(bookie1, bookieClient, storage, new MockTicker());
+        long id1 = 0xdeadL;
+        long id2 = 0xbedeL;
+        long id3 = 0xbebeL;
+        LedgerMetadata metadata1 = newClosedMetadataWithEnsemble(id1, 1000, bookie1, bookie2).build();
+        LedgerMetadata metadata2 = newClosedMetadataWithEnsemble(id2, 1000, bookie1, bookie3).build();
+        LedgerMetadata metadata3 = newMetadataWithEnsemble(id3, bookie1, bookie3).build();
+        LedgerMetadata metadata3closed = newClosedMetadataWithEnsemble(id3, 1000, bookie1, bookie3).build();
+
+        AtomicInteger callCount = new AtomicInteger(0);
+        DataIntegrityCheckImpl impl = new DataIntegrityCheckImpl(bookie1, lm, storage,
+                                                                 copier,
+                                                                 mock(BookKeeperAdmin.class),
+                                                                 Schedulers.io()) {
+                @Override
+                Single<LedgerMetadata> recoverLedger(long ledgerId, String runId) {
+                    if (callCount.getAndIncrement() == 0) {
+                        return Single.error(new BKException.BKReadException());
+                    } else {
+                        return Single.just(metadata3closed);
+                    }
+                }
+            };
+
+        bookieClient.getMockBookies().seedLedgerForBookie(bookie2, id1, metadata1);
+        bookieClient.getMockBookies().seedLedgerForBookie(bookie3, id2, metadata2);
+        bookieClient.getMockBookies().seedLedgerForBookie(bookie3, id3, metadata3closed);
+
+        lm.createLedgerMetadata(id1, metadata1).get();
+        lm.createLedgerMetadata(id2, metadata2).get();
+        lm.createLedgerMetadata(id3, metadata3).get();
+
+        assertThat(storage.ledgerExists(id1), equalTo(false));
+        assertThat(storage.ledgerExists(id2), equalTo(false));
+        assertThat(storage.ledgerExists(id3), equalTo(false));
+        storage.setMasterKey(id3, PASSWD);
+        storage.setLimboState(id3);
+        assertThat(storage.hasLimboState(id3), equalTo(true));
+
+        storage.setStorageStateFlag(StorageState.NEEDS_INTEGRITY_CHECK);
+        assertThat(StorageState.NEEDS_INTEGRITY_CHECK,
+                   isIn(storage.getStorageStateFlags()));
+
+        impl.runFullCheck().get();
+
+        assertThat(StorageState.NEEDS_INTEGRITY_CHECK,
+                   isIn(storage.getStorageStateFlags()));
+        verify(storage, times(1)).flush();
+
+        assertThat(storage.ledgerExists(id1), equalTo(true));
+        assertThat(storage.ledgerExists(id2), equalTo(true));
+        assertThat(storage.ledgerExists(id3), equalTo(true));
+        assertThat(storage.entryExists(id3, 0), equalTo(false));
+        assertThat(storage.hasLimboState(id3), equalTo(true));
+
+        // run again, second time shouldn't error
+        impl.runFullCheck().get();
+
+        assertThat(StorageState.NEEDS_INTEGRITY_CHECK,
+                   not(isIn(storage.getStorageStateFlags())));
+        verify(storage, times(2)).flush();
+
+        assertThat(storage.ledgerExists(id3), equalTo(true));
+        assertThat(storage.entryExists(id3, 0), equalTo(true));
+        assertThat(storage.hasLimboState(id3), equalTo(false));
+    }
+
+    @Test
+    public void testFullCheckInEntryCopyFailsFirstTime() throws Exception {
+        MockBookieClient bookieClient = spy(new MockBookieClient(executor));
+        MockLedgerManager lm = new MockLedgerManager();
+        ServerConfiguration conf = serverConf();
+        MockLedgerStorage storage = spy(new MockLedgerStorage());
+
+        EntryCopier copier = new EntryCopierImpl(bookie1, bookieClient, storage, new MockTicker());
+        long id1 = 0xdeadL;
+        long id2 = 0xbedeL;
+        long id3 = 0xbebeL;
+        LedgerMetadata metadata1 = newClosedMetadataWithEnsemble(id1, 100, bookie1, bookie2).build();
+        LedgerMetadata metadata2 = newClosedMetadataWithEnsemble(id2, 100, bookie1, bookie3).build();
+        LedgerMetadata metadata3 = newMetadataWithEnsemble(id3, bookie1, bookie3).build();
+        LedgerMetadata metadata3closed = newClosedMetadataWithEnsemble(id3, 100, bookie1, bookie3).build();
+
+        DataIntegrityCheckImpl impl = new DataIntegrityCheckImpl(bookie1, lm, storage,
+                                                                 copier,
+                                                                 mock(BookKeeperAdmin.class),
+                                                                 Schedulers.io()) {
+                @Override
+                Single<LedgerMetadata> recoverLedger(long ledgerId, String runId) {
+                    return Single.just(metadata3closed);
+                }
+            };
+
+        bookieClient.getMockBookies().seedLedgerForBookie(bookie2, id1, metadata1);
+        bookieClient.getMockBookies().seedLedgerForBookie(bookie3, id2, metadata2);
+
+        lm.createLedgerMetadata(id1, metadata1).get();
+        lm.createLedgerMetadata(id2, metadata2).get();
+        lm.createLedgerMetadata(id3, metadata3).get();
+
+        assertThat(storage.ledgerExists(id1), equalTo(false));
+        assertThat(storage.ledgerExists(id2), equalTo(false));
+        assertThat(storage.ledgerExists(id3), equalTo(false));
+        storage.setMasterKey(id3, PASSWD);
+        storage.setLimboState(id3);
+        assertThat(storage.hasLimboState(id3), equalTo(true));
+
+        storage.setStorageStateFlag(StorageState.NEEDS_INTEGRITY_CHECK);
+        assertThat(StorageState.NEEDS_INTEGRITY_CHECK,
+                   isIn(storage.getStorageStateFlags()));
+
+        impl.runFullCheck().get();
+
+        assertThat(StorageState.NEEDS_INTEGRITY_CHECK,
+                   isIn(storage.getStorageStateFlags()));
+        verify(storage, times(1)).flush();
+
+        assertThat(storage.ledgerExists(id1), equalTo(true));
+        assertThat(storage.ledgerExists(id2), equalTo(true));
+        assertThat(storage.ledgerExists(id3), equalTo(true));
+        assertThat(storage.entryExists(id3, 0), equalTo(false));
+        assertThat(storage.hasLimboState(id3), equalTo(false));
+
+        // make it possible to recover the ledger by seeding bookie3
+        bookieClient.getMockBookies().seedLedgerForBookie(bookie3, id3, metadata3closed);
+
+        // run again, second time shouldn't error
+        impl.runFullCheck().get();
+
+        assertThat(StorageState.NEEDS_INTEGRITY_CHECK,
+                   not(isIn(storage.getStorageStateFlags())));
+        verify(storage, times(2)).flush();
+
+        assertThat(storage.ledgerExists(id3), equalTo(true));
+        assertThat(storage.entryExists(id3, 0), equalTo(true));
+        assertThat(storage.hasLimboState(id3), equalTo(false));
+    }
+
+
+    @Test
+    public void testFullCheckAllInLimboAndMissing() throws Exception {
+        MockBookieClient bookieClient = spy(new MockBookieClient(executor));
+        MockLedgerManager lm = new MockLedgerManager();
+        ServerConfiguration conf = serverConf();
+        MockLedgerStorage storage = spy(new MockLedgerStorage());
+
+        EntryCopier copier = new EntryCopierImpl(bookie1, bookieClient, storage, new MockTicker());
+        long id1 = 0xdeadL;
+        long id2 = 0xbedeL;
+        long id3 = 0xbebeL;
+        LedgerMetadata metadata1 = newMetadataWithEnsemble(id1, bookie1, bookie2).build();
+        LedgerMetadata metadata2 = newMetadataWithEnsemble(id2, bookie1, bookie3).build();
+        LedgerMetadata metadata3 = newMetadataWithEnsemble(id3, bookie1, bookie3).build();
+
+        DataIntegrityCheckImpl impl = new DataIntegrityCheckImpl(bookie1, lm, storage,
+                                                                 copier,
+                                                                 mock(BookKeeperAdmin.class),
+                                                                 Schedulers.io()) {
+                @Override
+                Single<LedgerMetadata> recoverLedger(long ledgerId, String runId) {
+                    return Single.error(
+                            new BKException.BKNoSuchLedgerExistsOnMetadataServerException());
+                }
+            };
+
+        lm.createLedgerMetadata(id1, metadata1).get();
+        lm.createLedgerMetadata(id2, metadata2).get();
+        lm.createLedgerMetadata(id3, metadata3).get();
+
+        assertThat(storage.ledgerExists(id1), equalTo(false));
+        assertThat(storage.ledgerExists(id2), equalTo(false));
+        assertThat(storage.ledgerExists(id3), equalTo(false));
+        storage.setMasterKey(id1, PASSWD);
+        storage.setLimboState(id1);
+        storage.setMasterKey(id2, PASSWD);
+        storage.setLimboState(id2);
+        storage.setMasterKey(id3, PASSWD);
+        storage.setLimboState(id3);
+        assertThat(storage.hasLimboState(id1), equalTo(true));
+        assertThat(storage.hasLimboState(id2), equalTo(true));
+        assertThat(storage.hasLimboState(id3), equalTo(true));
+
+        storage.setStorageStateFlag(StorageState.NEEDS_INTEGRITY_CHECK);
+        assertThat(StorageState.NEEDS_INTEGRITY_CHECK,
+                   isIn(storage.getStorageStateFlags()));
+
+        impl.runFullCheck().get();
+
+        verify(storage, times(1)).flush();
+
+        assertThat(StorageState.NEEDS_INTEGRITY_CHECK,
+                   not(isIn(storage.getStorageStateFlags())));
+    }
+
+    @Test
+    public void testFullCheckFailFlushRetainsFlag() throws Exception {
+        MockBookieClient bookieClient = spy(new MockBookieClient(executor));
+        MockLedgerManager lm = new MockLedgerManager();
+        ServerConfiguration conf = serverConf();
+        AtomicInteger count = new AtomicInteger(0);
+        MockLedgerStorage storage = spy(new MockLedgerStorage() {
+                @Override
+                public void flush() throws IOException {
+                    if (count.getAndIncrement() == 0) {
+                        throw new IOException("broken flush");
+                    }
+                }
+            });
+
+        EntryCopier copier = new EntryCopierImpl(bookie1, bookieClient, storage, new MockTicker());
+        long id1 = 0xdeadL;
+        long id2 = 0xbedeL;
+        long id3 = 0xbebeL;
+        LedgerMetadata metadata1 = newClosedMetadataWithEnsemble(id1, 100, bookie1, bookie2).build();
+        LedgerMetadata metadata2 = newClosedMetadataWithEnsemble(id2, 100, bookie1, bookie3).build();
+        LedgerMetadata metadata3 = newClosedMetadataWithEnsemble(id3, 100, bookie1, bookie3).build();
+
+        DataIntegrityCheckImpl impl = new DataIntegrityCheckImpl(bookie1, lm, storage,
+                                                                 copier,
+                                                                 mock(BookKeeperAdmin.class),
+                                                                 Schedulers.io());
+        bookieClient.getMockBookies().seedLedgerForBookie(bookie2, id1, metadata1);
+        bookieClient.getMockBookies().seedLedgerForBookie(bookie3, id2, metadata2);
+        bookieClient.getMockBookies().seedLedgerForBookie(bookie3, id3, metadata3);
+
+        lm.createLedgerMetadata(id1, metadata1).get();
+        lm.createLedgerMetadata(id2, metadata2).get();
+        lm.createLedgerMetadata(id3, metadata3).get();
+
+        assertThat(storage.ledgerExists(id1), equalTo(false));
+        assertThat(storage.ledgerExists(id2), equalTo(false));
+        assertThat(storage.ledgerExists(id3), equalTo(false));
+
+        storage.setStorageStateFlag(StorageState.NEEDS_INTEGRITY_CHECK);
+        try {
+            impl.runFullCheck().get();
+            Assert.fail("Should have failed on flush");
+        } catch (ExecutionException ee) {
+            assertThat(ee.getCause(), instanceOf(IOException.class));
+        }
+        assertThat(StorageState.NEEDS_INTEGRITY_CHECK,
+                   isIn(storage.getStorageStateFlags()));
+        verify(storage, times(1)).flush();
+
+        assertThat(storage.ledgerExists(id1), equalTo(true));
+        assertThat(storage.ledgerExists(id2), equalTo(true));
+        assertThat(storage.ledgerExists(id3), equalTo(true));
+
+        // run again, second time shouldn't error
+        impl.runFullCheck().get();
+
+        assertThat(StorageState.NEEDS_INTEGRITY_CHECK,
+                   not(isIn(storage.getStorageStateFlags())));
+        verify(storage, times(2)).flush();
+    }
+}
+
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/datainteg/DataIntegrityServiceTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/datainteg/DataIntegrityServiceTest.java
new file mode 100644
index 0000000..0a92d60
--- /dev/null
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/datainteg/DataIntegrityServiceTest.java
@@ -0,0 +1,169 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.bookkeeper.bookie.datainteg;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.server.conf.BookieConfiguration;
+import org.apache.bookkeeper.stats.NullStatsLogger;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test for DataIntegrityService.
+ */
+public class DataIntegrityServiceTest {
+    private static DataIntegrityService newLowIntervalService(DataIntegrityCheck check) {
+        return new DataIntegrityService(
+                new BookieConfiguration(new ServerConfiguration()),
+                NullStatsLogger.INSTANCE, check) {
+            @Override
+            public int interval() {
+                return 1;
+            }
+            @Override
+            public TimeUnit intervalUnit() {
+                return TimeUnit.MICROSECONDS;
+            }
+        };
+    }
+
+    @Test
+    public void testFullCheckRunsIfRequested() throws Exception {
+        CompletableFuture<Void> promise = new CompletableFuture<>();
+        MockDataIntegrityCheck check = new MockDataIntegrityCheck() {
+                @Override
+                public boolean needsFullCheck() {
+                    return true;
+                }
+                @Override
+                public CompletableFuture<Void> runFullCheck() {
+                    promise.complete(null);
+                    return super.runFullCheck();
+                }
+            };
+        DataIntegrityService service = newLowIntervalService(check);
+        try {
+            service.start();
+
+            promise.get(5, TimeUnit.SECONDS);
+        } finally {
+            service.stop();
+        }
+    }
+
+    @Test
+    public void testFullCheckDoesntRunIfNotRequested() throws Exception {
+        CompletableFuture<Void> promise = new CompletableFuture<>();
+        MockDataIntegrityCheck check = new MockDataIntegrityCheck() {
+                @Override
+                public boolean needsFullCheck() {
+                    return false;
+                }
+                @Override
+                public CompletableFuture<Void> runFullCheck() {
+                    promise.complete(null);
+                    return super.runFullCheck();
+                }
+            };
+        DataIntegrityService service = newLowIntervalService(check);
+        try {
+            service.start();
+
+            try {
+                // timeout set very low, so hard to tell if
+                // it's waiting to run or not running, but it
+                // should be the latter on any modern machine
+                promise.get(100, TimeUnit.MILLISECONDS);
+                Assert.fail("Shouldn't have run");
+            } catch (TimeoutException te) {
+                // expected
+            }
+        } finally {
+            service.stop();
+        }
+    }
+
+    @Test
+    public void testFullCheckRunsMultipleTimes() throws Exception {
+        AtomicInteger count = new AtomicInteger(0);
+        CompletableFuture<Void> promise = new CompletableFuture<>();
+        MockDataIntegrityCheck check = new MockDataIntegrityCheck() {
+                @Override
+                public boolean needsFullCheck() {
+                    return true;
+                }
+                @Override
+                public CompletableFuture<Void> runFullCheck() {
+                    if (count.incrementAndGet() == 10) {
+                        promise.complete(null);
+                    }
+                    return super.runFullCheck();
+                }
+            };
+        DataIntegrityService service = newLowIntervalService(check);
+        try {
+            service.start();
+
+            promise.get(10, TimeUnit.SECONDS);
+        } finally {
+            service.stop();
+        }
+    }
+
+    @Test
+    public void testRunDontRunThenRunAgain() throws Exception {
+        AtomicBoolean needsFullCheck = new AtomicBoolean(true);
+        Semaphore semaphore = new Semaphore(1);
+        semaphore.acquire(); // increment the count, can only be released by a check
+        MockDataIntegrityCheck check = new MockDataIntegrityCheck() {
+                @Override
+                public boolean needsFullCheck() {
+                    return needsFullCheck.getAndSet(false);
+                }
+                @Override
+                public CompletableFuture<Void> runFullCheck() {
+                    semaphore.release();
+                    return super.runFullCheck();
+                }
+            };
+        DataIntegrityService service = newLowIntervalService(check);
+        try {
+            service.start();
+
+            Assert.assertTrue("Check should have run",
+                              semaphore.tryAcquire(10, TimeUnit.SECONDS));
+            Assert.assertFalse("Check shouldn't run again",
+                               semaphore.tryAcquire(100, TimeUnit.MILLISECONDS));
+            needsFullCheck.set(true);
+            Assert.assertTrue("Check should run again",
+                              semaphore.tryAcquire(10, TimeUnit.SECONDS));
+        } finally {
+            service.stop();
+        }
+    }
+}
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/datainteg/EntryCopierTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/datainteg/EntryCopierTest.java
new file mode 100644
index 0000000..0d7d3d8
--- /dev/null
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/datainteg/EntryCopierTest.java
@@ -0,0 +1,635 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.bookkeeper.bookie.datainteg;
+
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.contains;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.instanceOf;
+import static org.mockito.Mockito.anyInt;
+import static org.mockito.Mockito.anyLong;
+import static org.mockito.Mockito.anyObject;
+import static org.mockito.Mockito.eq;
+import static org.mockito.Mockito.inOrder;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+
+import io.netty.buffer.ByteBuf;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.bookkeeper.bookie.BookieException;
+import org.apache.bookkeeper.bookie.MockLedgerStorage;
+import org.apache.bookkeeper.client.BKException;
+import org.apache.bookkeeper.client.LedgerMetadataBuilder;
+import org.apache.bookkeeper.client.api.DigestType;
+import org.apache.bookkeeper.client.api.LedgerMetadata;
+import org.apache.bookkeeper.common.concurrent.FutureUtils;
+import org.apache.bookkeeper.common.util.MockTicker;
+import org.apache.bookkeeper.common.util.OrderedExecutor;
+import org.apache.bookkeeper.net.BookieId;
+import org.apache.bookkeeper.proto.MockBookieClient;
+
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import org.mockito.InOrder;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Tests for EntryCopierImpl.
+ */
+public class EntryCopierTest {
+    private static final Logger log = LoggerFactory.getLogger(EntryCopierTest.class);
+    private static final BookieId bookie1 = BookieId.parse("bookie1:3181");
+    private static final BookieId bookie2 = BookieId.parse("bookie2:3181");
+    private static final BookieId bookie3 = BookieId.parse("bookie3:3181");
+    private static final BookieId bookie4 = BookieId.parse("bookie4:3181");
+    private static final BookieId bookie5 = BookieId.parse("bookie5:3181");
+    private static final BookieId bookie6 = BookieId.parse("bookie6:3181");
+
+    private OrderedExecutor executor = null;
+
+    @Before
+    public void setup() throws Exception {
+        executor = OrderedExecutor.newBuilder().numThreads(1).name("test").build();
+    }
+
+    @After
+    public void teardown() throws Exception {
+        if (executor != null) {
+            executor.shutdownNow();
+        }
+    }
+
+    @Test
+    public void testCopyFromAvailable() throws Exception {
+        MockBookieClient bookieClient = spy(new MockBookieClient(executor));
+        MockLedgerStorage storage = spy(new MockLedgerStorage());
+        long ledgerId = 0xbeeb;
+        LedgerMetadata metadata = LedgerMetadataBuilder.create()
+            .withId(ledgerId)
+            .withPassword(new byte[0])
+            .withDigestType(DigestType.CRC32C)
+            .withEnsembleSize(2)
+            .withWriteQuorumSize(2)
+            .withAckQuorumSize(2)
+            .newEnsembleEntry(0, Lists.newArrayList(bookie1, bookie2))
+            .withLastEntryId(10)
+            .withLength(1000)
+            .withClosedState()
+            .build();
+        bookieClient.getMockBookies().seedLedger(ledgerId, metadata);
+
+        EntryCopier copier = new EntryCopierImpl(bookie1, bookieClient, storage, new MockTicker());
+        EntryCopier.Batch batch = copier.newBatch(
+                ledgerId, metadata);
+
+        CompletableFuture<Long> f1 = batch.copyFromAvailable(0);
+        CompletableFuture<Long> f2 = batch.copyFromAvailable(2);
+        CompletableFuture<Long> f3 = batch.copyFromAvailable(4);
+        CompletableFuture<Long> f4 = batch.copyFromAvailable(10);
+        try {
+            batch.copyFromAvailable(100);
+            Assert.fail("Should have given IllegalArgumentException");
+        } catch (IllegalArgumentException ie) {
+            // correct
+        }
+
+        try {
+            batch.copyFromAvailable(-1);
+            Assert.fail("Should have given IllegalArgumentException");
+        } catch (IllegalArgumentException ie) {
+            // correct
+        }
+        CompletableFuture.allOf(f1, f2, f3, f4).get();
+
+        verify(bookieClient, times(1)).readEntry(eq(bookie2), eq(ledgerId), eq(0L),
+                                                 anyObject(), anyObject(), anyInt(), anyObject());
+        verify(bookieClient, times(1)).readEntry(eq(bookie2), eq(ledgerId), eq(2L),
+                                                 anyObject(), anyObject(), anyInt(), anyObject());
+        verify(bookieClient, times(1)).readEntry(eq(bookie2), eq(ledgerId), eq(4L),
+                                                 anyObject(), anyObject(), anyInt(), anyObject());
+        verify(bookieClient, times(1)).readEntry(eq(bookie2), eq(ledgerId), eq(10L),
+                                                 anyObject(), anyObject(), anyInt(), anyObject());
+        verify(bookieClient, times(4)).readEntry(eq(bookie2), eq(ledgerId), anyLong(),
+                                                 anyObject(), anyObject(), anyInt(), anyObject());
+
+        verify(storage, times(4)).addEntry(anyObject());
+        assertThat(storage.entryExists(ledgerId, 0), equalTo(true));
+        assertThat(storage.entryExists(ledgerId, 2), equalTo(true));
+        assertThat(storage.entryExists(ledgerId, 4), equalTo(true));
+        assertThat(storage.entryExists(ledgerId, 10), equalTo(true));
+    }
+
+    @Test
+    public void testNoCopiesAvailable() throws Exception {
+        MockBookieClient bookieClient = spy(new MockBookieClient(executor));
+        MockLedgerStorage storage = spy(new MockLedgerStorage());
+        long ledgerId = 0xbeeb;
+        LedgerMetadata metadata = LedgerMetadataBuilder.create()
+            .withId(ledgerId)
+            .withPassword(new byte[0])
+            .withDigestType(DigestType.CRC32C)
+            .withEnsembleSize(1)
+            .withWriteQuorumSize(1)
+            .withAckQuorumSize(1)
+            .newEnsembleEntry(0, Lists.newArrayList(bookie1))
+            .withLastEntryId(10)
+            .withLength(1000)
+            .withClosedState()
+            .build();
+        bookieClient.getMockBookies().seedLedger(ledgerId, metadata);
+
+        EntryCopier copier = new EntryCopierImpl(bookie1, bookieClient, storage, new MockTicker());
+        EntryCopier.Batch batch = copier.newBatch(
+                ledgerId, metadata);
+        List<CompletableFuture<Long>> futures = Lists.newArrayList();
+        for (long l = 0; l < 10; l++) {
+            futures.add(batch.copyFromAvailable(l));
+        }
+        try {
+            CompletableFuture.allOf(futures.toArray(new CompletableFuture[0])).get();
+            Assert.fail("Should have failed");
+        } catch (ExecutionException e) {
+            assertThat(e.getCause(), instanceOf(BKException.BKReadException.class));
+        }
+    }
+
+    @Test
+    public void testCopyOneEntryFails() throws Exception {
+        MockBookieClient bookieClient = spy(new MockBookieClient(executor));
+        MockLedgerStorage storage = spy(new MockLedgerStorage());
+        long ledgerId = 0xbeeb;
+        LedgerMetadata metadata = LedgerMetadataBuilder.create()
+            .withId(ledgerId)
+            .withPassword(new byte[0])
+            .withDigestType(DigestType.CRC32C)
+            .withEnsembleSize(2)
+            .withWriteQuorumSize(2)
+            .withAckQuorumSize(2)
+            .newEnsembleEntry(0, Lists.newArrayList(bookie1, bookie2))
+            .withLastEntryId(10)
+            .withLength(1000)
+            .withClosedState()
+            .build();
+        bookieClient.getMockBookies().seedLedger(ledgerId, metadata);
+
+        bookieClient.setPreReadHook((bookie, ledger, entry) -> {
+                if (entry == 2L) {
+                    return FutureUtils.exception(new BKException.BKTimeoutException());
+                } else {
+                    return CompletableFuture.completedFuture(null);
+                }
+            });
+        EntryCopier copier = new EntryCopierImpl(bookie1, bookieClient, storage, new MockTicker());
+        EntryCopier.Batch batch = copier.newBatch(ledgerId, metadata);
+
+        CompletableFuture<Long> f1 = batch.copyFromAvailable(0);
+        CompletableFuture<Long> f2 = batch.copyFromAvailable(2);
+        CompletableFuture<Long> f3 = batch.copyFromAvailable(4);
+        CompletableFuture<Long> f4 = batch.copyFromAvailable(10);
+
+        try {
+            CompletableFuture.allOf(f1, f2, f3, f4).get();
+            Assert.fail("Should have failed");
+        } catch (ExecutionException ee) {
+            assertThat(ee.getCause(), instanceOf(BKException.BKTimeoutException.class));
+        }
+
+        // other entries should still have been added
+        verify(storage, times(3)).addEntry(anyObject());
+        assertThat(storage.entryExists(ledgerId, 0), equalTo(true));
+        assertThat(storage.entryExists(ledgerId, 4), equalTo(true));
+        assertThat(storage.entryExists(ledgerId, 10), equalTo(true));
+    }
+
+    @Test
+    public void testCopyAllEntriesFail() throws Exception {
+        MockBookieClient bookieClient = spy(new MockBookieClient(executor));
+        MockLedgerStorage storage = spy(new MockLedgerStorage());
+        long ledgerId = 0xbeeb;
+        LedgerMetadata metadata = LedgerMetadataBuilder.create()
+            .withId(ledgerId)
+            .withPassword(new byte[0])
+            .withDigestType(DigestType.CRC32C)
+            .withEnsembleSize(2)
+            .withWriteQuorumSize(2)
+            .withAckQuorumSize(2)
+            .newEnsembleEntry(0, Lists.newArrayList(bookie1, bookie2))
+            .withLastEntryId(10)
+            .withLength(1000)
+            .withClosedState()
+            .build();
+        bookieClient.getMockBookies().seedLedger(ledgerId, metadata);
+
+        bookieClient.setPreReadHook((bookie, ledger, entry) ->
+                                    FutureUtils.exception(new BKException.BKTimeoutException()));
+        EntryCopier copier = new EntryCopierImpl(bookie1, bookieClient, storage, new MockTicker());
+        EntryCopier.Batch batch = copier.newBatch(ledgerId, metadata);
+
+        CompletableFuture<Long> f1 = batch.copyFromAvailable(0);
+        CompletableFuture<Long> f2 = batch.copyFromAvailable(2);
+        CompletableFuture<Long> f3 = batch.copyFromAvailable(4);
+        CompletableFuture<Long> f4 = batch.copyFromAvailable(10);
+
+        try {
+            CompletableFuture.allOf(f1, f2, f3, f4).get();
+            Assert.fail("Should have failed");
+        } catch (ExecutionException ee) {
+            assertThat(ee.getCause(), instanceOf(BKException.BKTimeoutException.class));
+        }
+
+        // Nothing should have been added
+        verify(storage, times(0)).addEntry(anyObject());
+    }
+
+    @Test
+    public void testCopyOneEntryFailsOnStorage() throws Exception {
+        MockBookieClient bookieClient = spy(new MockBookieClient(executor));
+        MockLedgerStorage storage = spy(new MockLedgerStorage() {
+                @Override
+                public long addEntry(ByteBuf buffer) throws IOException, BookieException {
+                    long entryId = buffer.getLong(buffer.readerIndex() + 8);
+                    if (entryId == 0L) {
+                        throw new IOException("failing");
+                    }
+                    return super.addEntry(buffer);
+                }
+            });
+        long ledgerId = 0xbeeb;
+        LedgerMetadata metadata = LedgerMetadataBuilder.create()
+            .withId(ledgerId)
+            .withPassword(new byte[0])
+            .withDigestType(DigestType.CRC32C)
+            .withEnsembleSize(2)
+            .withWriteQuorumSize(2)
+            .withAckQuorumSize(2)
+            .newEnsembleEntry(0, Lists.newArrayList(bookie1, bookie2))
+            .withLastEntryId(10)
+            .withLength(1000)
+            .withClosedState()
+            .build();
+        bookieClient.getMockBookies().seedLedger(ledgerId, metadata);
+
+        EntryCopier copier = new EntryCopierImpl(bookie1, bookieClient, storage, new MockTicker());
+        EntryCopier.Batch batch = copier.newBatch(ledgerId, metadata);
+
+        CompletableFuture<Long> f1 = batch.copyFromAvailable(0);
+        CompletableFuture<Long> f2 = batch.copyFromAvailable(2);
+        CompletableFuture<Long> f3 = batch.copyFromAvailable(4);
+        CompletableFuture<Long> f4 = batch.copyFromAvailable(10);
+
+        try {
+            CompletableFuture.allOf(f1, f2, f3, f4).get();
+            Assert.fail("Should have failed");
+        } catch (ExecutionException ee) {
+            assertThat(ee.getCause(), instanceOf(IOException.class));
+        }
+
+        // other entries should still have been added
+        verify(storage, times(4)).addEntry(anyObject());
+        assertThat(storage.entryExists(ledgerId, 0), equalTo(false));
+        assertThat(storage.entryExists(ledgerId, 2), equalTo(true));
+        assertThat(storage.entryExists(ledgerId, 4), equalTo(true));
+        assertThat(storage.entryExists(ledgerId, 10), equalTo(true));
+    }
+
+    @Test
+    public void testCopyAllEntriesFailOnStorage() throws Exception {
+        MockBookieClient bookieClient = spy(new MockBookieClient(executor));
+        MockLedgerStorage storage = spy(new MockLedgerStorage() {
+                @Override
+                public long addEntry(ByteBuf buffer) throws IOException, BookieException {
+                    throw new IOException("failing");
+                }
+            });
+        long ledgerId = 0xbeeb;
+        LedgerMetadata metadata = LedgerMetadataBuilder.create()
+            .withId(ledgerId)
+            .withPassword(new byte[0])
+            .withDigestType(DigestType.CRC32C)
+            .withEnsembleSize(2)
+            .withWriteQuorumSize(2)
+            .withAckQuorumSize(2)
+            .newEnsembleEntry(0, Lists.newArrayList(bookie1, bookie2))
+            .withLastEntryId(10)
+            .withLength(1000)
+            .withClosedState()
+            .build();
+        bookieClient.getMockBookies().seedLedger(ledgerId, metadata);
+
+        EntryCopier copier = new EntryCopierImpl(bookie1, bookieClient, storage, new MockTicker());
+        EntryCopier.Batch batch = copier.newBatch(ledgerId, metadata);
+
+        CompletableFuture<Long> f1 = batch.copyFromAvailable(0);
+        CompletableFuture<Long> f2 = batch.copyFromAvailable(2);
+        CompletableFuture<Long> f3 = batch.copyFromAvailable(4);
+        CompletableFuture<Long> f4 = batch.copyFromAvailable(10);
+
+        try {
+            CompletableFuture.allOf(f1, f2, f3, f4).get();
+            Assert.fail("Should have failed");
+        } catch (ExecutionException ee) {
+            assertThat(ee.getCause(), instanceOf(IOException.class));
+        }
+
+        // other entries should still have been added
+        verify(storage, times(4)).addEntry(anyObject());
+        assertThat(storage.entryExists(ledgerId, 0), equalTo(false));
+        assertThat(storage.entryExists(ledgerId, 2), equalTo(false));
+        assertThat(storage.entryExists(ledgerId, 4), equalTo(false));
+        assertThat(storage.entryExists(ledgerId, 10), equalTo(false));
+    }
+
+    @Test
+    public void testReadOneEntry() throws Exception {
+        long ledgerId = 0xbeeb; // don't change, the shuffle for preferred bookies uses ledger id as seed
+        LedgerMetadata metadata = LedgerMetadataBuilder.create()
+            .withId(ledgerId)
+            .withPassword(new byte[0])
+            .withDigestType(DigestType.CRC32C)
+            .withEnsembleSize(3)
+            .withWriteQuorumSize(3)
+            .withAckQuorumSize(3)
+            .newEnsembleEntry(0, Lists.newArrayList(bookie1, bookie2, bookie3))
+            .withLastEntryId(10)
+            .withLength(1000)
+            .withClosedState()
+            .build();
+
+        MockBookieClient bookieClient = spy(new MockBookieClient(executor));
+        bookieClient.getMockBookies().seedLedger(ledgerId, metadata);
+
+        EntryCopier copier = new EntryCopierImpl(bookie2, bookieClient,
+                                                 new MockLedgerStorage(), new MockTicker());
+        EntryCopierImpl.BatchImpl batch = (EntryCopierImpl.BatchImpl) copier.newBatch(ledgerId, metadata);
+        for (int i = 0; i <= 10; i++) {
+            batch.fetchEntry(i).get();
+            verify(bookieClient, times(i + 1)).readEntry(anyObject(), anyLong(), anyLong(),
+                                                       anyObject(), anyObject(), anyInt());
+            verify(bookieClient, times(i + 1)).readEntry(eq(bookie3), anyLong(), anyLong(),
+                                                       anyObject(), anyObject(), anyInt());
+        }
+    }
+
+    @Test
+    public void testReadOneFirstReplicaFails() throws Exception {
+        long ledgerId = 0xbeeb; // don't change, the shuffle for preferred bookies uses ledger id as seed
+        LedgerMetadata metadata = LedgerMetadataBuilder.create()
+            .withId(ledgerId)
+            .withPassword(new byte[0])
+            .withDigestType(DigestType.CRC32C)
+            .withEnsembleSize(3)
+            .withWriteQuorumSize(3)
+            .withAckQuorumSize(3)
+            .newEnsembleEntry(0, Lists.newArrayList(bookie1, bookie2, bookie3))
+            .withLastEntryId(10)
+            .withLength(1000)
+            .withClosedState()
+            .build();
+
+        MockBookieClient bookieClient = spy(new MockBookieClient(executor));
+        bookieClient.getMockBookies().seedLedger(ledgerId, metadata);
+        bookieClient.errorBookies(bookie3);
+        MockTicker ticker = new MockTicker();
+        EntryCopierImpl copier = new EntryCopierImpl(bookie2, bookieClient,
+                                                     new MockLedgerStorage(), ticker);
+        CompletableFuture<Void> errorProcessedPromise = new CompletableFuture<>();
+        EntryCopierImpl.BatchImpl batch = copier.new BatchImpl(bookie2, ledgerId,
+                                                               metadata,
+                                                               new EntryCopierImpl.SinBin(ticker)) {
+                @Override
+                void notifyBookieError(BookieId bookie) {
+                    super.notifyBookieError(bookie);
+                    errorProcessedPromise.complete(null);
+                }
+            };
+
+        batch.fetchEntry(0).get();
+
+        // will read twice, fail at bookie3, succeed at bookie1
+        verify(bookieClient, times(2)).readEntry(anyObject(), anyLong(), anyLong(),
+                                                 anyObject(), anyObject(), anyInt());
+        verify(bookieClient, times(1)).readEntry(eq(bookie3), anyLong(), anyLong(),
+                                                 anyObject(), anyObject(), anyInt());
+        verify(bookieClient, times(1)).readEntry(eq(bookie1), anyLong(), anyLong(),
+                                                 anyObject(), anyObject(), anyInt());
+        errorProcessedPromise.get(10, TimeUnit.SECONDS);
+        batch.fetchEntry(1).get();
+
+        // subsequent read should go straight for bookie1
+        verify(bookieClient, times(3)).readEntry(anyObject(), anyLong(), anyLong(),
+                                                       anyObject(), anyObject(), anyInt());
+        verify(bookieClient, times(2)).readEntry(eq(bookie1), anyLong(), anyLong(),
+                                                   anyObject(), anyObject(), anyInt());
+    }
+
+    @Test
+    public void testReadOneAllReplicasFail() throws Exception {
+        long ledgerId = 0xbeeb; // don't change, the shuffle for preferred bookies uses ledger id as seed
+        LedgerMetadata metadata = LedgerMetadataBuilder.create()
+            .withId(ledgerId)
+            .withPassword(new byte[0])
+            .withDigestType(DigestType.CRC32C)
+            .withEnsembleSize(3)
+            .withWriteQuorumSize(3)
+            .withAckQuorumSize(3)
+            .newEnsembleEntry(0, Lists.newArrayList(bookie1, bookie2, bookie3))
+            .withLastEntryId(10)
+            .withLength(1000)
+            .withClosedState()
+            .build();
+
+        MockBookieClient bookieClient = spy(new MockBookieClient(executor));
+        bookieClient.getMockBookies().seedLedger(ledgerId, metadata);
+        // we expect to try to read from bookie3 first
+        bookieClient.setPreReadHook((bookie, ledgerId1, entryId) -> {
+                if (bookie.equals(bookie1)) {
+                    return FutureUtils.exception(new BKException.BKReadException());
+                } else if (bookie.equals(bookie3)) {
+                    return FutureUtils.exception(new BKException.BKBookieException());
+                } else {
+                    return CompletableFuture.completedFuture(null);
+                }
+            });
+        EntryCopier copier = new EntryCopierImpl(bookie2, bookieClient,
+                                                 new MockLedgerStorage(), new MockTicker());
+        EntryCopierImpl.BatchImpl batch = (EntryCopierImpl.BatchImpl) copier.newBatch(ledgerId, metadata);
+
+        try {
+            batch.fetchEntry(0).get();
+            Assert.fail("Shouldn't get this far");
+        } catch (ExecutionException ee) {
+            assertThat(ee.getCause(), instanceOf(BKException.BKBookieException.class));
+        }
+
+        InOrder inOrder = inOrder(bookieClient);
+        inOrder.verify(bookieClient, times(1)).readEntry(eq(bookie3), anyLong(), anyLong(),
+                                                         anyObject(), anyObject(), anyInt());
+        inOrder.verify(bookieClient, times(1)).readEntry(eq(bookie1), anyLong(), anyLong(),
+                                                         anyObject(), anyObject(), anyInt());
+    }
+
+    @Test
+    public void testReadOneWithErrorBookieReinstatedAfterSinBin() throws Exception {
+        long ledgerId = 0xbeeb; // don't change, the shuffle for preferred bookies uses ledger id as seed
+        LedgerMetadata metadata = LedgerMetadataBuilder.create()
+            .withId(ledgerId)
+            .withPassword(new byte[0])
+            .withDigestType(DigestType.CRC32C)
+            .withEnsembleSize(3)
+            .withWriteQuorumSize(3)
+            .withAckQuorumSize(3)
+            .newEnsembleEntry(0, Lists.newArrayList(bookie1, bookie2, bookie3))
+            .withLastEntryId(10)
+            .withLength(1000)
+            .withClosedState()
+            .build();
+
+        MockBookieClient bookieClient = spy(new MockBookieClient(executor));
+        bookieClient.getMockBookies().seedLedger(ledgerId, metadata);
+        bookieClient.errorBookies(bookie3);
+
+        CompletableFuture<Void> errorProcessedPromise = new CompletableFuture<>();
+
+        MockTicker ticker = new MockTicker();
+        EntryCopierImpl copier = new EntryCopierImpl(bookie2, bookieClient,
+                                                     new MockLedgerStorage(), ticker);
+        EntryCopierImpl.SinBin sinBin = new EntryCopierImpl.SinBin(ticker);
+        EntryCopierImpl.BatchImpl batch = copier.new BatchImpl(bookie2, ledgerId, metadata, sinBin) {
+                @Override
+                void notifyBookieError(BookieId bookie) {
+                    super.notifyBookieError(bookie);
+                    errorProcessedPromise.complete(null);
+                }
+            };
+        batch.fetchEntry(0).get();
+        verify(bookieClient, times(1)).readEntry(eq(bookie3), anyLong(), anyLong(),
+                                                 anyObject(), anyObject(), anyInt());
+        verify(bookieClient, times(1)).readEntry(eq(bookie1), anyLong(), anyLong(),
+                                                 anyObject(), anyObject(), anyInt());
+        errorProcessedPromise.get(10, TimeUnit.SECONDS);
+
+        // bookie3 should be fine to use again, but we shouldn't use it until if come out
+        // of the sinbin
+        bookieClient.removeErrors(bookie3);
+
+        // read batch again, error should carry over
+        EntryCopierImpl.BatchImpl batch2 = copier.new BatchImpl(bookie2, ledgerId, metadata, sinBin);
+        batch2.fetchEntry(0).get();
+        verify(bookieClient, times(1)).readEntry(eq(bookie3), anyLong(), anyLong(),
+                                                 anyObject(), anyObject(), anyInt());
+        verify(bookieClient, times(2)).readEntry(eq(bookie1), anyLong(), anyLong(),
+                                                 anyObject(), anyObject(), anyInt());
+        // advance time
+        ticker.advance(70, TimeUnit.SECONDS);
+
+        // sinbinned bookie should be restored, read should come from bookie3 again
+        EntryCopierImpl.BatchImpl batch3 = copier.new BatchImpl(bookie2, ledgerId, metadata, sinBin);
+        batch3.fetchEntry(0).get();
+        verify(bookieClient, times(2)).readEntry(eq(bookie3), anyLong(), anyLong(),
+                                                 anyObject(), anyObject(), anyInt());
+        verify(bookieClient, times(2)).readEntry(eq(bookie1), anyLong(), anyLong(),
+                                                 anyObject(), anyObject(), anyInt());
+    }
+
+    @Test
+    public void testReadEntryOnlyOnSelf() throws Exception {
+        long ledgerId = 0xbeeb;
+        LedgerMetadata metadata = LedgerMetadataBuilder.create()
+            .withId(ledgerId)
+            .withPassword(new byte[0])
+            .withDigestType(DigestType.CRC32C)
+            .withEnsembleSize(1)
+            .withWriteQuorumSize(1)
+            .withAckQuorumSize(1)
+            .newEnsembleEntry(0, Lists.newArrayList(bookie2))
+            .withLastEntryId(10)
+            .withLength(1000)
+            .withClosedState()
+            .build();
+
+        MockBookieClient bookieClient = spy(new MockBookieClient(executor));
+        bookieClient.getMockBookies().seedLedger(ledgerId, metadata);
+
+        CompletableFuture<Void> errorProcessedPromise = new CompletableFuture<>();
+
+        MockTicker ticker = new MockTicker();
+        EntryCopierImpl copier = new EntryCopierImpl(bookie2, bookieClient,
+                                                     new MockLedgerStorage(), ticker);
+        EntryCopierImpl.BatchImpl batch = (EntryCopierImpl.BatchImpl) copier.newBatch(ledgerId, metadata);
+        try {
+            batch.fetchEntry(0).get();
+        } catch (ExecutionException ee) {
+            assertThat(ee.getCause(), instanceOf(BKException.BKReadException.class));
+        }
+        verify(bookieClient, times(0)).readEntry(anyObject(), anyLong(), anyLong(),
+                                                 anyObject(), anyObject(), anyInt());
+    }
+
+    @Test
+    public void testPreferredBookieIndices() throws Exception {
+        long ledgerId = 0xbeeb;
+        LedgerMetadata metadata1 = LedgerMetadataBuilder.create()
+            .withId(ledgerId)
+            .withPassword(new byte[0])
+            .withDigestType(DigestType.CRC32C)
+            .withEnsembleSize(5)
+            .withWriteQuorumSize(2)
+            .withAckQuorumSize(2)
+            .newEnsembleEntry(0, Lists.newArrayList(bookie1, bookie2, bookie3, bookie4, bookie5))
+            .newEnsembleEntry(3, Lists.newArrayList(bookie1, bookie6, bookie3, bookie4, bookie5))
+            .newEnsembleEntry(5, Lists.newArrayList(bookie1, bookie2, bookie3, bookie4, bookie5))
+            .withLastEntryId(10)
+            .withLength(1000)
+            .withClosedState()
+            .build();
+
+        Map<Long, ? extends List<Integer>> order =
+            EntryCopierImpl.preferredBookieIndices(bookie2, metadata1,
+                                                     Collections.emptySet(),
+                                                     ledgerId);
+        assertThat(order.get(0L), contains(4, 0, 3, 2));
+        assertThat(order.get(3L), contains(4, 1, 0, 3, 2));
+        assertThat(order.get(5L), contains(4, 0, 3, 2));
+
+        Map<Long, ? extends List<Integer>> orderWithErr =
+            EntryCopierImpl.preferredBookieIndices(bookie2, metadata1,
+                                                     Sets.newHashSet(bookie1, bookie3),
+                                                     ledgerId);
+        assertThat(orderWithErr.get(0L), contains(4, 3, 0, 2));
+        assertThat(orderWithErr.get(3L), contains(4, 1, 3, 0, 2));
+        assertThat(orderWithErr.get(5L), contains(4, 3, 0, 2));
+    }
+}
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/datainteg/MetadataAsyncIteratorTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/datainteg/MetadataAsyncIteratorTest.java
new file mode 100644
index 0000000..fe0759a
--- /dev/null
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/datainteg/MetadataAsyncIteratorTest.java
@@ -0,0 +1,301 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.bookkeeper.bookie.datainteg;
+
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.instanceOf;
+
+import com.google.common.collect.Lists;
+
+import io.reactivex.rxjava3.schedulers.Schedulers;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.bookkeeper.client.BKException;
+import org.apache.bookkeeper.client.LedgerMetadataBuilder;
+import org.apache.bookkeeper.client.api.DigestType;
+import org.apache.bookkeeper.client.api.LedgerMetadata;
+import org.apache.bookkeeper.common.concurrent.FutureUtils;
+import org.apache.bookkeeper.meta.LedgerManager;
+import org.apache.bookkeeper.meta.MockLedgerManager;
+import org.apache.bookkeeper.net.BookieId;
+import org.apache.bookkeeper.versioning.Versioned;
+
+import org.junit.Assert;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Tests for MetadataAsyncIterator.
+ */
+public class MetadataAsyncIteratorTest {
+    private static Logger log = LoggerFactory.getLogger(MetadataAsyncIteratorTest.class);
+
+    private LedgerMetadata newRandomMetadata(long randBit) throws Exception {
+        return LedgerMetadataBuilder.create()
+            .withId(1)
+            .withPassword(new byte[0])
+            .withDigestType(DigestType.CRC32C)
+            .withEnsembleSize(1)
+            .withWriteQuorumSize(1)
+            .withAckQuorumSize(1)
+            .newEnsembleEntry(0, Lists.newArrayList(
+                                      BookieId.parse("foobar-" + randBit + ":3181")))
+            .build();
+    }
+
+    private ConcurrentHashMap<Long, LedgerMetadata> addLedgers(LedgerManager lm, int count)
+            throws Exception {
+        ConcurrentHashMap<Long, LedgerMetadata> added = new ConcurrentHashMap<>();
+        for (long i = 0; i < count; i++) {
+            LedgerMetadata metadata = newRandomMetadata(i);
+            lm.createLedgerMetadata(i, metadata).get();
+            added.put(i, metadata);
+        }
+        return added;
+    }
+
+    private static CompletableFuture<Void> removeFromMap(
+            ConcurrentHashMap<Long, LedgerMetadata> map,
+            long ledgerId, LedgerMetadata metadata) {
+        log.debug("removing ledger {}", ledgerId);
+        if (map.remove(ledgerId, metadata)) {
+            return CompletableFuture.completedFuture(null);
+        } else {
+            log.debug("ledger {} already removed", ledgerId);
+            return FutureUtils.exception(new Exception("ledger already removed"));
+        }
+    }
+
+    @Test
+    public void testIteratorOverAll() throws Exception {
+        MockLedgerManager lm = new MockLedgerManager();
+        ConcurrentHashMap<Long, LedgerMetadata> added = addLedgers(lm, 10000);
+        MetadataAsyncIterator iterator = new MetadataAsyncIterator(Schedulers.io(),
+                                                                   lm, 100 /* inflight */,
+                                                                   3 /* timeout */, TimeUnit.SECONDS);
+        iterator.forEach((ledgerId, metadata) -> removeFromMap(added, ledgerId, metadata))
+            .get(10, TimeUnit.SECONDS);
+        assertThat(added.isEmpty(), equalTo(true));
+    }
+
+    @Test
+    public void testSingleLedger() throws Exception {
+        MockLedgerManager lm = new MockLedgerManager();
+        LedgerMetadata single = newRandomMetadata(0xdeadbeef);
+        MetadataAsyncIterator iterator = new MetadataAsyncIterator(Schedulers.io(),
+                                                                   lm, 100 /* inflight */,
+                                                                   3 /* timeout */, TimeUnit.SECONDS);
+        iterator.forEach((ledgerId, metadata) -> {
+                if (ledgerId == 0xdeadbeef && metadata.equals(single)) {
+                    return CompletableFuture.completedFuture(null);
+                } else {
+                    return FutureUtils.exception(new Exception("Unexpected metadata"));
+                }
+            }).get(10, TimeUnit.SECONDS);
+    }
+
+    @Test
+    public void testEmptyRange() throws Exception {
+        MockLedgerManager lm = new MockLedgerManager();
+        MetadataAsyncIterator iterator = new MetadataAsyncIterator(Schedulers.io(),
+                                                                   lm, 100 /* inflight */,
+                                                                   3 /* timeout */, TimeUnit.SECONDS);
+        iterator.forEach((ledgerId, metadata) -> FutureUtils.exception(new Exception("Should be empty")))
+            .get(10, TimeUnit.SECONDS);
+    }
+
+    @Test
+    public void testOneLedgerErrorsOnRead() throws Exception {
+        MockLedgerManager lm = new MockLedgerManager() {
+                @Override
+                public CompletableFuture<Versioned<LedgerMetadata>> readLedgerMetadata(long ledgerId) {
+                    if (ledgerId == 403) {
+                        return FutureUtils.exception(new BKException.ZKException());
+                    } else {
+                        return super.readLedgerMetadata(ledgerId);
+                    }
+                }
+            };
+        ConcurrentHashMap<Long, LedgerMetadata> added = addLedgers(lm, 10000);
+        MetadataAsyncIterator iterator = new MetadataAsyncIterator(Schedulers.io(),
+                                                                   lm, 100 /* inflight */,
+                                                                   3 /* timeout */, TimeUnit.SECONDS);
+        try {
+            iterator.forEach((ledgerId, metadata) -> removeFromMap(added, ledgerId, metadata))
+                .get(10, TimeUnit.SECONDS);
+        } catch (ExecutionException ee) {
+            assertThat(ee.getCause(), instanceOf(BKException.ZKException.class));
+        }
+    }
+
+    @Test
+    public void testOneLedgerErrorsOnProcessing() throws Exception {
+        MockLedgerManager lm = new MockLedgerManager();
+        ConcurrentHashMap<Long, LedgerMetadata> added = addLedgers(lm, 10000);
+        MetadataAsyncIterator iterator = new MetadataAsyncIterator(Schedulers.io(),
+                                                                   lm, 100 /* inflight */,
+                                                                   3 /* timeout */, TimeUnit.SECONDS);
+        try {
+            iterator.forEach((ledgerId, metadata) -> {
+                    if (ledgerId == 403) {
+                        log.info("IKDEBUG erroring");
+                        return FutureUtils.exception(new Exception("foobar"));
+                    } else {
+                        return CompletableFuture.completedFuture(null);
+                    }
+                }).get(10, TimeUnit.SECONDS);
+            Assert.fail("shouldn't succeed");
+        } catch (ExecutionException ee) {
+            assertThat(ee.getCause().getMessage(), equalTo("foobar"));
+        }
+    }
+
+    @Test
+    public void testAllLedgersErrorOnRead() throws Exception {
+        MockLedgerManager lm = new MockLedgerManager() {
+                @Override
+                public CompletableFuture<Versioned<LedgerMetadata>> readLedgerMetadata(long ledgerId) {
+                    CompletableFuture<Versioned<LedgerMetadata>> promise = new CompletableFuture<>();
+                    promise.completeExceptionally(new BKException.ZKException());
+                    return promise;
+                }
+            };
+        ConcurrentHashMap<Long, LedgerMetadata> added = addLedgers(lm, 10000);
+        MetadataAsyncIterator iterator = new MetadataAsyncIterator(Schedulers.io(),
+                                                                   lm, 100 /* inflight */,
+                                                                   3 /* timeout */, TimeUnit.SECONDS);
+        try {
+            iterator.forEach((ledgerId, metadata) -> CompletableFuture.completedFuture(null))
+                .get(10, TimeUnit.SECONDS);
+        } catch (ExecutionException ee) {
+            assertThat(ee.getCause(), instanceOf(BKException.ZKException.class));
+        }
+    }
+
+    @Test
+    public void testAllLedgersErrorOnProcessing() throws Exception {
+        MockLedgerManager lm = new MockLedgerManager();
+        ConcurrentHashMap<Long, LedgerMetadata> added = addLedgers(lm, 10000);
+        MetadataAsyncIterator iterator = new MetadataAsyncIterator(Schedulers.io(),
+                                                                   lm, 100 /* inflight */,
+                                                                   3 /* timeout */, TimeUnit.SECONDS);
+        try {
+            iterator.forEach((ledgerId, metadata) -> FutureUtils.exception(new Exception("foobar")))
+                .get(10, TimeUnit.SECONDS);
+            Assert.fail("shouldn't succeed");
+        } catch (ExecutionException ee) {
+            assertThat(ee.getCause().getMessage(), equalTo("foobar"));
+        }
+    }
+
+    @Test
+    public void testOneLedgerDisappearsBetweenListAndRead() throws Exception {
+        MockLedgerManager lm = new MockLedgerManager() {
+                @Override
+                public CompletableFuture<Versioned<LedgerMetadata>> readLedgerMetadata(long ledgerId) {
+                    if (ledgerId == 501) {
+                        CompletableFuture<Versioned<LedgerMetadata>> promise = new CompletableFuture<>();
+                        promise.completeExceptionally(new BKException.BKNoSuchLedgerExistsOnMetadataServerException());
+                        return promise;
+                    } else {
+                        return super.readLedgerMetadata(ledgerId);
+                    }
+                }
+            };
+        ConcurrentHashMap<Long, LedgerMetadata> added = addLedgers(lm, 10000);
+        MetadataAsyncIterator iterator = new MetadataAsyncIterator(Schedulers.io(),
+                                                                   lm, 100 /* inflight */,
+                                                                   3 /* timeout */, TimeUnit.SECONDS);
+        iterator.forEach((ledgerId, metadata) -> removeFromMap(added, ledgerId, metadata))
+            .get(10, TimeUnit.SECONDS);
+        assertThat(added.size(), equalTo(1));
+        log.info("IKDEBUG {} {}", added, added.containsKey(5L));
+        assertThat(added.containsKey(501L), equalTo(true));
+    }
+
+    @Test
+    public void testEverySecondLedgerDisappearsBetweenListAndRead() throws Exception {
+        MockLedgerManager lm = new MockLedgerManager() {
+                @Override
+                public CompletableFuture<Versioned<LedgerMetadata>> readLedgerMetadata(long ledgerId) {
+                    if (ledgerId % 2 == 0) {
+                        return FutureUtils.exception(
+                                new BKException.BKNoSuchLedgerExistsOnMetadataServerException());
+                    } else {
+                        return super.readLedgerMetadata(ledgerId);
+                    }
+                }
+            };
+        int numLedgers = 10000;
+        ConcurrentHashMap<Long, LedgerMetadata> added = addLedgers(lm, numLedgers);
+        MetadataAsyncIterator iterator = new MetadataAsyncIterator(Schedulers.io(),
+                                                                   lm, 100,
+                                                                   3, TimeUnit.SECONDS);
+        iterator.forEach((ledgerId, metadata) -> removeFromMap(added, ledgerId, metadata))
+            .get(10, TimeUnit.SECONDS);
+        assertThat(added.size(), equalTo(numLedgers / 2));
+        assertThat(added.keySet().stream().allMatch(k -> k % 2 == 0), equalTo(true));
+        assertThat(added.keySet().stream().noneMatch(k -> k % 2 == 1), equalTo(true));
+    }
+
+    @Test
+    public void testEveryLedgerDisappearsBetweenListAndRead() throws Exception {
+        MockLedgerManager lm = new MockLedgerManager() {
+                @Override
+                public CompletableFuture<Versioned<LedgerMetadata>> readLedgerMetadata(long ledgerId) {
+                    return FutureUtils.exception(
+                            new BKException.BKNoSuchLedgerExistsOnMetadataServerException());
+                }
+            };
+        int numLedgers = 10000;
+        ConcurrentHashMap<Long, LedgerMetadata> added = addLedgers(lm, numLedgers);
+        MetadataAsyncIterator iterator = new MetadataAsyncIterator(Schedulers.io(),
+                                                                   lm, 100,
+                                                                   3, TimeUnit.SECONDS);
+        iterator.forEach((ledgerId, metadata) -> removeFromMap(added, ledgerId, metadata))
+            .get(10, TimeUnit.SECONDS);
+        assertThat(added.size(), equalTo(numLedgers));
+    }
+
+    @Test
+    public void testMaxOutInFlight() throws Exception {
+        MockLedgerManager lm = new MockLedgerManager();
+        int numLedgers = 1000;
+        ConcurrentHashMap<Long, LedgerMetadata> added = addLedgers(lm, numLedgers);
+        MetadataAsyncIterator iterator = new MetadataAsyncIterator(Schedulers.io(),
+                                                                   lm, 10,
+                                                                   3, TimeUnit.SECONDS);
+        CompletableFuture<Void> blocker = new CompletableFuture<>();
+        CompletableFuture<Void> iterFuture = iterator.forEach(
+                (ledgerId, metadata) ->
+                blocker.thenCompose(ignore -> removeFromMap(added, ledgerId, metadata)));
+        assertThat(iterFuture.isDone(), equalTo(false));
+        blocker.complete(null);
+        iterFuture.get(10, TimeUnit.SECONDS);
+        assertThat(added.isEmpty(), equalTo(true));
+    }
+
+}
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/datainteg/MockDataIntegrityCheck.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/datainteg/MockDataIntegrityCheck.java
new file mode 100644
index 0000000..cfbb1b1
--- /dev/null
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/datainteg/MockDataIntegrityCheck.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.bookkeeper.bookie.datainteg;
+
+import java.util.concurrent.CompletableFuture;
+
+/**
+ * Mock implementation of MockDataIntegrity.
+ */
+public class MockDataIntegrityCheck implements DataIntegrityCheck {
+    @Override
+    public CompletableFuture<Void> runPreBootCheck(String reason) {
+        return CompletableFuture.completedFuture(null);
+    }
+    @Override
+    public boolean needsFullCheck() {
+        return false;
+    }
+    @Override
+    public CompletableFuture<Void> runFullCheck() {
+        return CompletableFuture.completedFuture(null);
+    }
+}
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/datainteg/WriteSetsTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/datainteg/WriteSetsTest.java
new file mode 100644
index 0000000..1a82b0b
--- /dev/null
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/datainteg/WriteSetsTest.java
@@ -0,0 +1,174 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.bookkeeper.bookie.datainteg;
+
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.contains;
+import static org.hamcrest.Matchers.empty;
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.isIn;
+
+import com.google.common.collect.ImmutableList;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.bookkeeper.client.DistributionSchedule;
+import org.apache.bookkeeper.client.RoundRobinDistributionSchedule;
+
+import org.junit.Test;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Tests for WriteSets.
+ */
+public class WriteSetsTest {
+    private static final Logger log = LoggerFactory.getLogger(WriteSetsTest.class);
+
+    @Test
+    public void testOrderPreserved() throws Exception {
+        WriteSets writeSets = new WriteSets(ImmutableList.of(0, 3, 2, 4, 1),
+                                            5 /* ensemble */, 2 /* writeQ */);
+        assertThat(writeSets.getForEntry(0), contains(0, 1));
+        assertThat(writeSets.getForEntry(1), contains(2, 1));
+        assertThat(writeSets.getForEntry(2), contains(3, 2));
+        assertThat(writeSets.getForEntry(3), contains(3, 4));
+        assertThat(writeSets.getForEntry(4), contains(0, 4));
+    }
+
+    @Test
+    public void testOrderPreservedWithGapForCurrentBookie() throws Exception {
+        // my bookie id maps to 2, so it is missing from the preferred order
+        WriteSets writeSets = new WriteSets(ImmutableList.of(0, 3, 4, 1),
+                                            5 /* ensemble */, 2 /* writeQ */);
+        assertThat(writeSets.getForEntry(0), contains(0, 1));
+        assertThat(writeSets.getForEntry(1), contains(1));
+        assertThat(writeSets.getForEntry(2), contains(3));
+        assertThat(writeSets.getForEntry(3), contains(3, 4));
+        assertThat(writeSets.getForEntry(4), contains(0, 4));
+    }
+
+    @Test
+    public void testEmptyWriteSet() throws Exception {
+        // As can happen if we are the only bookie for a entry
+        WriteSets writeSets = new WriteSets(ImmutableList.of(0, 3, 4, 1),
+                                            5 /* ensemble */, 1 /* writeQ */);
+        assertThat(writeSets.getForEntry(0), contains(0));
+        assertThat(writeSets.getForEntry(1), contains(1));
+        assertThat(writeSets.getForEntry(2), empty());
+        assertThat(writeSets.getForEntry(3), contains(3));
+        assertThat(writeSets.getForEntry(4), contains(4));
+    }
+
+    @Test
+    public void testE2W2() throws Exception {
+        DistributionSchedule schedule = new RoundRobinDistributionSchedule(
+                2 /* write */, 2 /* ack */, 2 /* ensemble */);
+        WriteSets writeSets = new WriteSets(ImmutableList.of(0, 1),
+                                            2 /* ensemble */, 2 /* writeQ */);
+        for (int i = 0; i < 100; i++) {
+            ImmutableList<Integer> writeSet = writeSets.getForEntry(i);
+            DistributionSchedule.WriteSet distWriteSet = schedule.getWriteSet(i);
+            assertContentsMatch(writeSet, distWriteSet);
+        }
+
+        WriteSets writeSets2 = new WriteSets(ImmutableList.of(1, 0),
+                                             2 /* ensemble */, 2 /* writeQ */);
+        for (int i = 0; i < 100; i++) {
+            ImmutableList<Integer> writeSet = writeSets2.getForEntry(i);
+            DistributionSchedule.WriteSet distWriteSet = schedule.getWriteSet(i);
+            assertContentsMatch(writeSet, distWriteSet);
+        }
+    };
+
+    @Test
+    public void testE10W2() throws Exception {
+        DistributionSchedule schedule = new RoundRobinDistributionSchedule(
+                2 /* write */, 2 /* ack */, 10 /* ensemble */);
+        WriteSets writeSets = new WriteSets(ImmutableList.of(0, 8, 1, 9, 6, 3, 7, 4, 2, 5),
+                                            10 /* ensemble */,
+                                            2 /* writeQ */);
+        for (int i = 0; i < 100; i++) {
+            ImmutableList<Integer> writeSet = writeSets.getForEntry(i);
+            DistributionSchedule.WriteSet distWriteSet = schedule.getWriteSet(i);
+            assertContentsMatch(writeSet, distWriteSet);
+        }
+
+        WriteSets writeSets2 = new WriteSets(ImmutableList.of(7, 5, 1, 6, 3, 0, 8, 9, 4, 2),
+                                             10 /* ensemble */,
+                                             2 /* writeQ */);
+        for (int i = 0; i < 100; i++) {
+            ImmutableList<Integer> writeSet = writeSets2.getForEntry(i);
+            DistributionSchedule.WriteSet distWriteSet = schedule.getWriteSet(i);
+            assertContentsMatch(writeSet, distWriteSet);
+        }
+
+        WriteSets writeSets3 = new WriteSets(ImmutableList.of(0, 1, 2, 3, 4, 5, 6, 7, 8, 9),
+                                             10 /* ensemble */,
+                                             2 /* writeQ */);
+        for (int i = 0; i < 100; i++) {
+            ImmutableList<Integer> writeSet = writeSets3.getForEntry(i);
+            DistributionSchedule.WriteSet distWriteSet = schedule.getWriteSet(i);
+            assertContentsMatch(writeSet, distWriteSet);
+        }
+    };
+
+    @Test
+    public void testManyVariants() throws Exception {
+        for (int w = 1; w <= 12; w++) {
+            for (int e = w; e <= 12; e++) {
+                DistributionSchedule schedule = new RoundRobinDistributionSchedule(
+                        w /* write */, w /* ack */, e /* ensemble */);
+
+                // Create shuffled set of indices
+                List<Integer> indices = new ArrayList<>();
+                for (int i = 0; i < e; i++) {
+                    indices.add(i);
+                }
+                Collections.shuffle(indices);
+
+                WriteSets writeSets = new WriteSets(ImmutableList.copyOf(indices),
+                                                    e, w);
+                for (int i = 0; i < 100; i++) {
+                    ImmutableList<Integer> writeSet = writeSets.getForEntry(i);
+                    DistributionSchedule.WriteSet distWriteSet = schedule.getWriteSet(i);
+                    assertContentsMatch(writeSet, distWriteSet);
+                }
+            }
+        }
+    }
+
+    private static void assertContentsMatch(ImmutableList<Integer> writeSet,
+                                            DistributionSchedule.WriteSet distWriteSet)
+            throws Exception {
+        log.info("writeSet {} distWriteSet {}", writeSet, distWriteSet.size());
+        assertThat(writeSet.size(), equalTo(distWriteSet.size()));
+        for (Integer i : writeSet) {
+            assertThat(distWriteSet.contains(i), equalTo(true));
+        }
+
+        for (int i = 0; i < distWriteSet.size(); i++) {
+            assertThat(distWriteSet.get(i), isIn(writeSet));
+        }
+    }
+}
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorageTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorageTest.java
index 6239e64..00efee9 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorageTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorageTest.java
@@ -21,6 +21,7 @@
 package org.apache.bookkeeper.bookie.storage.ldb;
 
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 import com.google.common.collect.Lists;
@@ -37,6 +38,7 @@ import org.apache.bookkeeper.bookie.BookieImpl;
 import org.apache.bookkeeper.bookie.EntryLocation;
 import org.apache.bookkeeper.bookie.EntryLogger;
 import org.apache.bookkeeper.bookie.LedgerDirsManager;
+import org.apache.bookkeeper.bookie.LedgerStorage;
 import org.apache.bookkeeper.bookie.TestBookieImpl;
 import org.apache.bookkeeper.conf.ServerConfiguration;
 import org.apache.bookkeeper.conf.TestBKConfiguration;
@@ -45,14 +47,18 @@ import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 /**
  * Unit test for {@link DbLedgerStorage}.
  */
 public class DbLedgerStorageTest {
-
+    private static final Logger log = LoggerFactory.getLogger(DbLedgerStorageTest.class);
     private DbLedgerStorage storage;
     private File tmpDir;
     private LedgerDirsManager ledgerDirsManager;
+    private ServerConfiguration conf;
 
     @Before
     public void setup() throws Exception {
@@ -63,7 +69,7 @@ public class DbLedgerStorageTest {
         BookieImpl.checkDirectoryStructure(curDir);
 
         int gcWaitTime = 1000;
-        ServerConfiguration conf = TestBKConfiguration.newServerConfiguration();
+        conf = TestBKConfiguration.newServerConfiguration();
         conf.setGcWaitTime(gcWaitTime);
         conf.setLedgerStorageClass(DbLedgerStorage.class.getName());
         conf.setLedgerDirNames(new String[] { tmpDir.toString() });
@@ -133,6 +139,7 @@ public class DbLedgerStorageTest {
         assertEquals(true, ((DbLedgerStorage) storage).isFlushRequired());
 
         // Read from write cache
+        assertTrue(storage.entryExists(4, 1));
         ByteBuf res = storage.getEntry(4, 1);
         assertEquals(entry, res);
 
@@ -141,6 +148,7 @@ public class DbLedgerStorageTest {
         assertEquals(false, ((DbLedgerStorage) storage).isFlushRequired());
 
         // Read from db
+        assertTrue(storage.entryExists(4, 1));
         res = storage.getEntry(4, 1);
         assertEquals(entry, res);
 
@@ -190,22 +198,13 @@ public class DbLedgerStorageTest {
         storage.deleteLedger(4);
         assertEquals(false, storage.ledgerExists(4));
 
-        // Should not throw exception event if the ledger was deleted
-        storage.getEntry(4, 4);
-        assertEquals(3, storage.getLastAddConfirmed(4));
-
-        storage.addEntry(Unpooled.wrappedBuffer(entry2));
-        res = storage.getEntry(4, BookieProtocol.LAST_ADD_CONFIRMED);
-        assertEquals(entry4, res);
-        assertEquals(3, storage.getLastAddConfirmed(4));
-
-        // Get last entry from storage
+        // remove entries for ledger 4 from cache
         storage.flush();
 
         try {
             storage.getEntry(4, 4);
             fail("Should have thrown exception since the ledger was deleted");
-        } catch (NoEntryException e) {
+        } catch (Bookie.NoLedgerException e) {
             // ok
         }
     }
@@ -436,9 +435,201 @@ public class DbLedgerStorageTest {
     }
 
     @Test
-    public void testGetLedgerDirsListeners() throws IOException {
-        // we should have two listeners, one is the SingleLedgerDirectories listener,
-        // and another is EntryLogManagerForEntryLogPerLedger
-        assertEquals(2, ledgerDirsManager.getListeners().size());
+    public void testLimboStateSucceedsWhenInLimboButHasEntry() throws Exception {
+        storage.setMasterKey(1, "foobar".getBytes());
+
+        ByteBuf entry0 = Unpooled.buffer(1024);
+        entry0.writeLong(1); // ledger id
+        entry0.writeLong(0); // entry id
+        entry0.writeBytes("entry-0".getBytes());
+
+        storage.addEntry(entry0);
+        storage.flush();
+        storage.setLimboState(1);
+
+        try {
+            storage.getEntry(1, 0);
+        } catch (BookieException.DataUnknownException e) {
+            fail("Should have been able to read entry");
+        }
+
+        storage.shutdown();
+    }
+
+    @Test
+    public void testLimboStateThrowsInLimboWhenNoEntry() throws Exception {
+        storage.setMasterKey(1, "foobar".getBytes());
+
+        ByteBuf entry0 = Unpooled.buffer(1024);
+        entry0.writeLong(1); // ledger id
+        entry0.writeLong(0); // entry id
+        entry0.writeBytes("entry-0".getBytes());
+
+        storage.addEntry(entry0);
+        storage.flush();
+        storage.setLimboState(1);
+
+        try {
+            storage.getEntry(1, 1);
+        } catch (NoEntryException nee) {
+            fail("Shouldn't have seen NoEntryException");
+        } catch (BookieException.DataUnknownException e) {
+            // expected
+        }
+
+        storage.shutdown();
+        Bookie restartedBookie = new TestBookieImpl(conf);
+        DbLedgerStorage restartedStorage = (DbLedgerStorage) restartedBookie.getLedgerStorage();
+        try {
+            try {
+                restartedStorage.getEntry(1, 1);
+            } catch (NoEntryException nee) {
+                fail("Shouldn't have seen NoEntryException");
+            } catch (BookieException.DataUnknownException e) {
+                // expected
+            }
+        } finally {
+            restartedStorage.shutdown();
+        }
+    }
+
+    @Test
+    public void testLimboStateThrowsNoEntryExceptionWhenLimboCleared() throws Exception {
+        storage.setMasterKey(1, "foobar".getBytes());
+
+        ByteBuf entry0 = Unpooled.buffer(1024);
+        entry0.writeLong(1); // ledger id
+        entry0.writeLong(0); // entry id
+        entry0.writeBytes("entry-0".getBytes());
+
+        storage.addEntry(entry0);
+        storage.flush();
+        storage.setLimboState(1);
+
+        try {
+            storage.getEntry(1, 1);
+        } catch (NoEntryException nee) {
+            fail("Shouldn't have seen NoEntryException");
+        } catch (BookieException.DataUnknownException e) {
+            // expected
+        }
+
+        storage.clearLimboState(1);
+        try {
+            storage.getEntry(1, 1);
+        } catch (NoEntryException nee) {
+            // expected
+        } catch (BookieException.DataUnknownException e) {
+            fail("Should have seen NoEntryException");
+        }
+    }
+
+    @Test
+    public void testLimboStateSucceedsWhenFenced() throws Exception {
+        storage.setMasterKey(1, "foobar".getBytes());
+
+        ByteBuf entry0 = Unpooled.buffer(1024);
+        entry0.writeLong(1); // ledger id
+        entry0.writeLong(0); // entry id
+        entry0.writeBytes("entry-0".getBytes());
+
+        storage.addEntry(entry0);
+        storage.flush();
+        storage.setFenced(1);
+        storage.setLimboState(1);
+
+        try {
+            storage.isFenced(1);
+        } catch (IOException ioe) {
+            fail("Should have been able to get isFenced response");
+        }
+
+        storage.shutdown();
+    }
+
+    @Test
+    public void testLimboStateThrowsInLimboWhenNotFenced() throws Exception {
+        storage.setMasterKey(1, "foobar".getBytes());
+
+        ByteBuf entry0 = Unpooled.buffer(1024);
+        entry0.writeLong(1); // ledger id
+        entry0.writeLong(0); // entry id
+        entry0.writeBytes("entry-0".getBytes());
+
+        storage.addEntry(entry0);
+        storage.flush();
+        storage.setLimboState(1);
+
+        try {
+            storage.isFenced(1);
+            fail("Shouldn't have been able to get isFenced response");
+        } catch (BookieException.DataUnknownException e) {
+            // expected
+        }
+
+        storage.shutdown();
+    }
+
+    @Test
+    public void testHasEntry() throws Exception {
+        long ledgerId = 0xbeefee;
+        storage.setMasterKey(ledgerId, "foobar".getBytes());
+
+        ByteBuf entry0 = Unpooled.buffer(1024);
+        entry0.writeLong(ledgerId); // ledger id
+        entry0.writeLong(0); // entry id
+        entry0.writeBytes("entry-0".getBytes());
+
+        storage.addEntry(entry0);
+
+        // should come from write cache
+        assertTrue(storage.entryExists(ledgerId, 0));
+        assertFalse(storage.entryExists(ledgerId, 1));
+
+        storage.flush();
+
+        // should come from storage
+        assertTrue(storage.entryExists(ledgerId, 0));
+        assertFalse(storage.entryExists(ledgerId, 1));
+
+        // pull entry into readcache
+        storage.getEntry(ledgerId, 0);
+
+        // should come from read cache
+        assertTrue(storage.entryExists(ledgerId, 0));
+        assertFalse(storage.entryExists(ledgerId, 1));
+    }
+
+    @Test
+    public void testStorageStateFlags() throws Exception {
+        assertTrue(storage.getStorageStateFlags().isEmpty());
+
+        storage.setStorageStateFlag(LedgerStorage.StorageState.NEEDS_INTEGRITY_CHECK);
+        assertTrue(storage.getStorageStateFlags()
+                   .contains(LedgerStorage.StorageState.NEEDS_INTEGRITY_CHECK));
+
+        storage.shutdown();
+        Bookie restartedBookie1 = new TestBookieImpl(conf);
+        DbLedgerStorage restartedStorage1 = (DbLedgerStorage) restartedBookie1.getLedgerStorage();
+        try {
+            assertTrue(restartedStorage1.getStorageStateFlags()
+                   .contains(LedgerStorage.StorageState.NEEDS_INTEGRITY_CHECK));
+            restartedStorage1.clearStorageStateFlag(LedgerStorage.StorageState.NEEDS_INTEGRITY_CHECK);
+
+            assertFalse(restartedStorage1.getStorageStateFlags()
+                   .contains(LedgerStorage.StorageState.NEEDS_INTEGRITY_CHECK));
+
+        } finally {
+            restartedStorage1.shutdown();
+        }
+
+        Bookie restartedBookie2 = new TestBookieImpl(conf);
+        DbLedgerStorage restartedStorage2 = (DbLedgerStorage) restartedBookie2.getLedgerStorage();
+        try {
+            assertFalse(restartedStorage2.getStorageStateFlags()
+                   .contains(LedgerStorage.StorageState.NEEDS_INTEGRITY_CHECK));
+        } finally {
+            restartedStorage2.shutdown();
+        }
     }
 }
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/ReadCacheTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/ReadCacheTest.java
index 337140c..d02f066 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/ReadCacheTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/ReadCacheTest.java
@@ -20,8 +20,11 @@
  */
 package org.apache.bookkeeper.bookie.storage.ldb;
 
+
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
 
 import io.netty.buffer.ByteBuf;
 import io.netty.buffer.Unpooled;
@@ -116,4 +119,24 @@ public class ReadCacheTest {
 
         cache.close();
     }
+
+    @Test
+    public void testHasEntry() {
+        ReadCache cache = new ReadCache(UnpooledByteBufAllocator.DEFAULT, 10 * 1024, 2 * 1024);
+
+        long ledgerId = 0xfefe;
+        for (int i = 0; i < 10; i++) {
+            ByteBuf entry = Unpooled.wrappedBuffer(new byte[1024]);
+            entry.setInt(0, i);
+            cache.put(ledgerId, i, entry);
+        }
+
+        assertFalse(cache.hasEntry(0xdead, 0));
+        assertFalse(cache.hasEntry(ledgerId, -1));
+        for (int i = 0; i < 10; i++) {
+            assertTrue(cache.hasEntry(ledgerId, i));
+        }
+        assertFalse(cache.hasEntry(ledgerId, 10));
+    }
+
 }
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/WriteCacheTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/WriteCacheTest.java
index 5726bbb..e92f0e7 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/WriteCacheTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/WriteCacheTest.java
@@ -286,4 +286,24 @@ public class WriteCacheTest {
         cache.close();
     }
 
+    @Test
+    public void testHasEntry() {
+        // Create cache with max size 4 KB and each segment is 128 bytes
+        WriteCache cache = new WriteCache(allocator, 4 * 1024, 128);
+
+        long ledgerId = 0xdede;
+        for (int i = 0; i < 48; i++) {
+            boolean inserted = cache.put(ledgerId, i, Unpooled.wrappedBuffer(("test-" + i).getBytes()));
+            assertTrue(inserted);
+        }
+
+        assertEquals(48, cache.count());
+
+        assertFalse(cache.hasEntry(0xfede, 1));
+        assertFalse(cache.hasEntry(ledgerId, -1));
+        for (int i = 0; i < 48; i++) {
+            assertTrue(cache.hasEntry(ledgerId, i));
+        }
+        assertFalse(cache.hasEntry(ledgerId, 48));
+    }
 }
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperClientTestsWithBookieErrors.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperClientTestsWithBookieErrors.java
index 8a09c62..8171c7d 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperClientTestsWithBookieErrors.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperClientTestsWithBookieErrors.java
@@ -30,6 +30,7 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.function.Consumer;
 
+import org.apache.bookkeeper.bookie.BookieException;
 import org.apache.bookkeeper.bookie.SortedLedgerStorage;
 import org.apache.bookkeeper.client.BookKeeper.DigestType;
 import org.apache.bookkeeper.conf.ClientConfiguration;
@@ -107,7 +108,7 @@ public class BookKeeperClientTestsWithBookieErrors extends BookKeeperClusterTest
         }
 
         @Override
-        public ByteBuf getEntry(long ledgerId, long entryId) throws IOException {
+        public ByteBuf getEntry(long ledgerId, long entryId) throws IOException, BookieException {
             Consumer<ByteBuf> faultInjection;
             synchronized (lock) {
                 faultInjection = storageFaultInjectionsMap.get(this);
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperCloseTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperCloseTest.java
index 92d105d..f40a09a 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperCloseTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperCloseTest.java
@@ -107,7 +107,7 @@ public class BookKeeperCloseTest extends BookKeeperClusterTestCase {
 
                 @Override
                 public ByteBuf readEntry(long ledgerId, long entryId)
-                        throws IOException, NoLedgerException {
+                        throws IOException, NoLedgerException, BookieException {
                     try {
                         Thread.sleep(5000);
                     } catch (InterruptedException ie) {
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookieWriteLedgerTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookieWriteLedgerTest.java
index e56c19b..fb9f83a 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookieWriteLedgerTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookieWriteLedgerTest.java
@@ -49,6 +49,7 @@ import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
 import java.util.stream.Collectors;
+import org.apache.bookkeeper.bookie.BookieException;
 import org.apache.bookkeeper.bookie.TestBookieImpl;
 import org.apache.bookkeeper.client.AsyncCallback.AddCallback;
 import org.apache.bookkeeper.client.BKException.BKLedgerClosedException;
@@ -1504,7 +1505,7 @@ public class BookieWriteLedgerTest extends
         }
 
         @Override
-        public ByteBuf readEntry(long ledgerId, long entryId) throws IOException, NoLedgerException {
+        public ByteBuf readEntry(long ledgerId, long entryId) throws IOException, NoLedgerException, BookieException {
             localBuf = super.readEntry(ledgerId, entryId);
 
             int capacity = 0;
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ParallelLedgerRecoveryTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ParallelLedgerRecoveryTest.java
index 6aa1ed7..74033ae 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ParallelLedgerRecoveryTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ParallelLedgerRecoveryTest.java
@@ -538,7 +538,7 @@ public class ParallelLedgerRecoveryTest extends BookKeeperClusterTestCase {
         }
 
         @Override
-        public ByteBuf readEntry(long ledgerId, long entryId) throws IOException, NoLedgerException {
+        public ByteBuf readEntry(long ledgerId, long entryId) throws IOException, NoLedgerException, BookieException {
             LOG.info("ReadEntry {} - {}", ledgerId, entryId);
             if (delayReadResponse.get() && delayReadOnEntry.get() == entryId) {
                 CountDownLatch latch = delayReadLatch;
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestReadLastConfirmedAndEntry.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestReadLastConfirmedAndEntry.java
index 662e2ae..551dbd5 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestReadLastConfirmedAndEntry.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestReadLastConfirmedAndEntry.java
@@ -33,6 +33,7 @@ import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.atomic.AtomicReference;
 import org.apache.bookkeeper.bookie.Bookie;
+import org.apache.bookkeeper.bookie.BookieException;
 import org.apache.bookkeeper.bookie.InterleavedLedgerStorage;
 import org.apache.bookkeeper.bookie.LedgerStorage;
 import org.apache.bookkeeper.bookie.SortedLedgerStorage;
@@ -88,7 +89,7 @@ public class TestReadLastConfirmedAndEntry extends BookKeeperClusterTestCase {
 
         @Override
         public ByteBuf readEntry(long ledgerId, long entryId)
-                throws IOException, NoLedgerException {
+                throws IOException, NoLedgerException, BookieException {
             if (entryId == expectedEntryToFail) {
                 if (stallOrRespondNull) {
                     try {
@@ -185,7 +186,7 @@ public class TestReadLastConfirmedAndEntry extends BookKeeperClusterTestCase {
         }
 
         @Override
-        public long readLastAddConfirmed(long ledgerId) throws IOException {
+        public long readLastAddConfirmed(long ledgerId) throws IOException, BookieException {
             long lac = super.readLastAddConfirmed(ledgerId);
             logger.info("Last Add Confirmed for ledger {} is {}", ledgerId, lac);
             if (lacToSlowRead == lac) {
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/discover/MockRegistrationManager.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/discover/MockRegistrationManager.java
new file mode 100644
index 0000000..2928250
--- /dev/null
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/discover/MockRegistrationManager.java
@@ -0,0 +1,141 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.bookkeeper.discover;
+
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.apache.bookkeeper.bookie.BookieException;
+import org.apache.bookkeeper.net.BookieId;
+import org.apache.bookkeeper.versioning.LongVersion;
+import org.apache.bookkeeper.versioning.Version;
+import org.apache.bookkeeper.versioning.Versioned;
+
+/**
+ * Mock implementation of RegistrationManager.
+ */
+public class MockRegistrationManager implements RegistrationManager {
+    private final ConcurrentHashMap<BookieId, Versioned<byte[]>> cookies = new ConcurrentHashMap<>();
+
+    @Override
+    public void close() {}
+
+    @Override
+    public String getClusterInstanceId() throws BookieException {
+        return "mock-cluster";
+    }
+
+    @Override
+    public void registerBookie(BookieId bookieId, boolean readOnly,
+                               BookieServiceInfo serviceInfo) throws BookieException {
+        throw new UnsupportedOperationException("Not implemented in mock. Implement if you need it");
+    }
+
+    @Override
+    public void unregisterBookie(BookieId bookieId, boolean readOnly) throws BookieException {
+        throw new UnsupportedOperationException("Not implemented in mock. Implement if you need it");
+    }
+
+    @Override
+    public boolean isBookieRegistered(BookieId bookieId) throws BookieException {
+        throw new UnsupportedOperationException("Not implemented in mock. Implement if you need it");
+    }
+
+    @Override
+    public void writeCookie(BookieId bookieId, Versioned<byte[]> cookieData) throws BookieException {
+        try {
+            cookies.compute(bookieId, (bookieId1, current) -> {
+                    if (cookieData.getVersion() == Version.NEW) {
+                        if (current == null) {
+                            return new Versioned<byte[]>(cookieData.getValue(), new LongVersion(1));
+                        } else {
+                            throw new RuntimeException(new BookieException.CookieExistException(bookieId.getId()));
+                        }
+                    } else {
+                        if (current != null
+                            && cookieData.getVersion().equals(current.getVersion())) {
+                            LongVersion oldVersion = (LongVersion) current.getVersion();
+                            LongVersion newVersion = new LongVersion(oldVersion.getLongVersion() + 1);
+                            return new Versioned<byte[]>(cookieData.getValue(), newVersion);
+                        } else {
+                            throw new RuntimeException(new BookieException.CookieExistException(bookieId.getId()));
+                        }
+                    }
+                });
+        } catch (RuntimeException e) {
+            if (e.getCause() instanceof BookieException) {
+                throw (BookieException) e.getCause();
+            }
+        }
+    }
+
+    @Override
+    public Versioned<byte[]> readCookie(BookieId bookieId) throws BookieException {
+        Versioned<byte[]> cookie = cookies.get(bookieId);
+        if (cookie == null) {
+            throw new BookieException.CookieNotFoundException(bookieId.toString());
+        }
+        return cookie;
+    }
+
+    @Override
+    public void removeCookie(BookieId bookieId, Version version) throws BookieException {
+        try {
+            cookies.compute(bookieId, (bookieId1, current) -> {
+                    if (current == null) {
+                        throw new RuntimeException(new BookieException.CookieNotFoundException(bookieId.toString()));
+                    } else if (current.getVersion().equals(version)) {
+                        return null;
+                    } else {
+                        throw new RuntimeException(new BookieException.MetadataStoreException("Bad version"));
+                    }
+                });
+        } catch (RuntimeException e) {
+            if (e.getCause() instanceof BookieException) {
+                throw (BookieException) e.getCause();
+            }
+        }
+
+    }
+
+    @Override
+    public boolean prepareFormat() throws Exception {
+        throw new UnsupportedOperationException("Not implemented in mock. Implement if you need it");
+    }
+
+    @Override
+    public boolean initNewCluster() throws Exception {
+        throw new UnsupportedOperationException("Not implemented in mock. Implement if you need it");
+    }
+
+    @Override
+    public boolean format() throws Exception {
+        throw new UnsupportedOperationException("Not implemented in mock. Implement if you need it");
+    }
+
+    @Override
+    public boolean nukeExistingCluster() throws Exception {
+        throw new UnsupportedOperationException("Not implemented in mock. Implement if you need it");
+    }
+
+    @Override
+    public void addRegistrationListener(RegistrationListener listener) {
+        throw new UnsupportedOperationException("Not implemented in mock. Implement if you need it");
+    }
+}
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/GcLedgersTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/GcLedgersTest.java
index ae11f0b..bedb76b 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/GcLedgersTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/GcLedgersTest.java
@@ -33,6 +33,7 @@ import io.netty.buffer.ByteBufAllocator;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
+import java.util.EnumSet;
 import java.util.HashSet;
 import java.util.LinkedList;
 import java.util.List;
@@ -616,6 +617,11 @@ public class GcLedgersTest extends LedgerManagerTestCase {
         }
 
         @Override
+        public boolean entryExists(long ledgerId, long entryId) throws IOException {
+            return false;
+        }
+
+        @Override
         public boolean setFenced(long ledgerId) throws IOException {
             return false;
         }
@@ -701,6 +707,37 @@ public class GcLedgersTest extends LedgerManagerTestCase {
         public OfLong getListOfEntriesOfLedger(long ledgerId) throws IOException {
             return null;
         }
+
+        @Override
+        public void setLimboState(long ledgerId) throws IOException {
+            throw new UnsupportedOperationException(
+                    "Limbo state only supported for DbLedgerStorage");
+        }
+
+        @Override
+        public boolean hasLimboState(long ledgerId) throws IOException {
+            throw new UnsupportedOperationException(
+                    "Limbo state only supported for DbLedgerStorage");
+        }
+
+        @Override
+        public void clearLimboState(long ledgerId) throws IOException {
+            throw new UnsupportedOperationException(
+                    "Limbo state only supported for DbLedgerStorage");
+        }
+
+        @Override
+        public EnumSet<StorageState> getStorageStateFlags() throws IOException {
+            return EnumSet.noneOf(StorageState.class);
+        }
+
+        @Override
+        public void setStorageStateFlag(StorageState flag) throws IOException {
+        }
+
+        @Override
+        public void clearStorageStateFlag(StorageState flag) throws IOException {
+        }
     }
 
     /**
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/LedgerManagerTestCase.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/LedgerManagerTestCase.java
index 3796250..8740577 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/LedgerManagerTestCase.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/LedgerManagerTestCase.java
@@ -28,6 +28,7 @@ import java.io.IOException;
 import java.net.URI;
 import java.util.Arrays;
 import java.util.Collection;
+import java.util.EnumSet;
 import java.util.Map;
 import java.util.NavigableMap;
 import java.util.Optional;
@@ -199,6 +200,11 @@ public abstract class LedgerManagerTestCase extends BookKeeperClusterTestCase {
         }
 
         @Override
+        public boolean entryExists(long ledgerId, long entryId) throws IOException {
+            return false;
+        }
+
+        @Override
         public boolean setFenced(long ledgerId) throws IOException {
             return false;
         }
@@ -298,5 +304,36 @@ public abstract class LedgerManagerTestCase extends BookKeeperClusterTestCase {
         public OfLong getListOfEntriesOfLedger(long ledgerId) {
             return null;
         }
+
+        @Override
+        public void setLimboState(long ledgerId) throws IOException {
+            throw new UnsupportedOperationException(
+                    "Limbo state only supported for DbLedgerStorage");
+        }
+
+        @Override
+        public boolean hasLimboState(long ledgerId) throws IOException {
+            throw new UnsupportedOperationException(
+                    "Limbo state only supported for DbLedgerStorage");
+        }
+
+        @Override
+        public void clearLimboState(long ledgerId) throws IOException {
+            throw new UnsupportedOperationException(
+                    "Limbo state only supported for DbLedgerStorage");
+        }
+
+        @Override
+        public EnumSet<StorageState> getStorageStateFlags() throws IOException {
+            return EnumSet.noneOf(StorageState.class);
+        }
+
+        @Override
+        public void setStorageStateFlag(StorageState flag) throws IOException {
+        }
+
+        @Override
+        public void clearStorageStateFlag(StorageState flag) throws IOException {
+        }
     }
 }
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/MockLedgerManager.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/MockLedgerManager.java
index 279fb71..9b22b60 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/MockLedgerManager.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/meta/MockLedgerManager.java
@@ -19,12 +19,18 @@
  */
 package org.apache.bookkeeper.meta;
 
-import java.util.HashMap;
+import com.google.common.collect.Lists;
+
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.List;
 import java.util.Map;
 import java.util.Optional;
 
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.CompletionException;
+import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 
@@ -63,7 +69,7 @@ public class MockLedgerManager implements LedgerManager {
     private Hook preWriteHook = (ledgerId, metadata) -> FutureUtils.value(null);
 
     public MockLedgerManager() {
-        this(new HashMap<>(),
+        this(new ConcurrentHashMap<>(),
              Executors.newSingleThreadExecutor((r) -> new Thread(r, "MockLedgerManager")), true);
     }
 
@@ -129,7 +135,7 @@ public class MockLedgerManager implements LedgerManager {
                     Versioned<LedgerMetadata> metadata = readMetadata(ledgerId);
                     if (metadata == null) {
                         executeCallback(() -> promise.completeExceptionally(
-                                                new BKException.BKNoSuchLedgerExistsException()));
+                                                new BKException.BKNoSuchLedgerExistsOnMetadataServerException()));
                     } else {
                         executeCallback(() -> promise.complete(metadata));
                     }
@@ -150,7 +156,8 @@ public class MockLedgerManager implements LedgerManager {
                     try {
                         Versioned<LedgerMetadata> oldMetadata = readMetadata(ledgerId);
                         if (oldMetadata == null) {
-                            return FutureUtils.exception(new BKException.BKNoSuchLedgerExistsException());
+                            return FutureUtils.exception(
+                                    new BKException.BKNoSuchLedgerExistsOnMetadataServerException());
                         } else if (!oldMetadata.getVersion().equals(currentVersion)) {
                             return FutureUtils.exception(new BKException.BKMetadataVersionException());
                         } else {
@@ -189,7 +196,25 @@ public class MockLedgerManager implements LedgerManager {
 
     @Override
     public LedgerRangeIterator getLedgerRanges(long zkOpTimeoutMs) {
-        return null;
+        List<Long> ledgerIds = new ArrayList<>(metadataMap.keySet());
+        ledgerIds.sort(Comparator.naturalOrder());
+        List<List<Long>> partitions = Lists.partition(ledgerIds, 100);
+        return new LedgerRangeIterator() {
+            int i = 0;
+            @Override
+            public boolean hasNext() {
+                if (i >= partitions.size()) {
+                    return false;
+                } else {
+                    return true;
+                }
+            }
+
+            @Override
+            public LedgerRange next() {
+                return new LedgerRange(new HashSet<>(partitions.get(i++)));
+            }
+        };
     }
 
     @Override
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/TestPerChannelBookieClient.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/TestPerChannelBookieClient.java
index 668a03e..d7c0165 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/TestPerChannelBookieClient.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/proto/TestPerChannelBookieClient.java
@@ -42,6 +42,7 @@ import java.util.concurrent.atomic.AtomicBoolean;
 import org.apache.bookkeeper.auth.AuthProviderFactoryFactory;
 import org.apache.bookkeeper.auth.ClientAuthProvider;
 import org.apache.bookkeeper.bookie.Bookie;
+import org.apache.bookkeeper.bookie.BookieException;
 import org.apache.bookkeeper.bookie.TestBookieImpl;
 import org.apache.bookkeeper.client.BKException;
 import org.apache.bookkeeper.common.util.OrderedExecutor;
@@ -247,7 +248,7 @@ public class TestPerChannelBookieClient extends BookKeeperClusterTestCase {
         Bookie delayBookie = new TestBookieImpl(conf) {
             @Override
             public ByteBuf readEntry(long ledgerId, long entryId)
-                    throws IOException, NoLedgerException {
+                    throws IOException, NoLedgerException, BookieException {
                 try {
                     Thread.sleep(3000);
                 } catch (InterruptedException ie) {
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookKeeperClusterTestCase.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookKeeperClusterTestCase.java
index bb66b7a..aa4b510 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookKeeperClusterTestCase.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookKeeperClusterTestCase.java
@@ -50,7 +50,10 @@ import org.apache.bookkeeper.bookie.BookieResources;
 import org.apache.bookkeeper.bookie.LedgerDirsManager;
 import org.apache.bookkeeper.bookie.LedgerStorage;
 import org.apache.bookkeeper.bookie.LegacyCookieValidation;
+import org.apache.bookkeeper.bookie.MockUncleanShutdownDetection;
 import org.apache.bookkeeper.bookie.ReadOnlyBookie;
+import org.apache.bookkeeper.bookie.UncleanShutdownDetection;
+import org.apache.bookkeeper.bookie.UncleanShutdownDetectionImpl;
 import org.apache.bookkeeper.client.BookKeeperTestClient;
 import org.apache.bookkeeper.common.allocator.ByteBufAllocatorWithOomHandler;
 import org.apache.bookkeeper.common.allocator.PoolingPolicy;
@@ -834,6 +837,9 @@ public abstract class BookKeeperClusterTestCase {
                     conf, diskChecker, bookieStats.scope(LD_LEDGER_SCOPE));
             LedgerDirsManager indexDirsManager = BookieResources.createIndexDirsManager(
                     conf, diskChecker, bookieStats.scope(LD_INDEX_SCOPE), ledgerDirsManager);
+
+            UncleanShutdownDetection uncleanShutdownDetection = new UncleanShutdownDetectionImpl(ledgerDirsManager);
+
             storage = BookieResources.createLedgerStorage(
                     conf, ledgerManager, ledgerDirsManager, indexDirsManager,
                     bookieStats, allocator);
@@ -847,7 +853,8 @@ public abstract class BookKeeperClusterTestCase {
                                         diskChecker, ledgerDirsManager, indexDirsManager,
                                         bookieStats, allocator, BookieServiceInfo.NO_INFO);
             }
-            server = new BookieServer(conf, bookie, rootStatsLogger, allocator);
+            server = new BookieServer(conf, bookie, rootStatsLogger, allocator,
+                    uncleanShutdownDetection);
             address = BookieImpl.getBookieAddress(conf);
 
             autoRecovery = null;
@@ -864,7 +871,8 @@ public abstract class BookKeeperClusterTestCase {
             storage = null;
 
             bookie = b;
-            server = new BookieServer(conf, b, provider.getStatsLogger(""), allocator);
+            server = new BookieServer(conf, b, provider.getStatsLogger(""),
+                    allocator, new MockUncleanShutdownDetection());
             address = BookieImpl.getBookieAddress(conf);
 
             autoRecovery = null;
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieClientTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieClientTest.java
index eca8856..659e50a 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieClientTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieClientTest.java
@@ -37,7 +37,7 @@ import java.util.Arrays;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledExecutorService;
-
+import org.apache.bookkeeper.bookie.MockUncleanShutdownDetection;
 import org.apache.bookkeeper.bookie.TestBookieImpl;
 import org.apache.bookkeeper.client.BKException;
 import org.apache.bookkeeper.client.BKException.Code;
@@ -93,7 +93,8 @@ public class BookieClientTest {
 
         bs = new BookieServer(
                 conf, new TestBookieImpl(conf),
-                NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
+                NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT,
+                new MockUncleanShutdownDetection());
         bs.start();
         eventLoopGroup = new NioEventLoopGroup();
         executor = OrderedExecutor.newBuilder()
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieZKExpireTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieZKExpireTest.java
index 6b4e1ad..006c485 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieZKExpireTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieZKExpireTest.java
@@ -30,6 +30,7 @@ import io.netty.buffer.UnpooledByteBufAllocator;
 import java.io.File;
 import java.util.HashSet;
 
+import org.apache.bookkeeper.bookie.MockUncleanShutdownDetection;
 import org.apache.bookkeeper.bookie.TestBookieImpl;
 import org.apache.bookkeeper.conf.ServerConfiguration;
 import org.apache.bookkeeper.proto.BookieServer;
@@ -69,7 +70,8 @@ public class BookieZKExpireTest extends BookKeeperClusterTestCase {
             ServerConfiguration conf = newServerConfiguration(PortManager.nextFreePort(), f, new File[] { f });
             server = new BookieServer(
                     conf, new TestBookieImpl(conf),
-                    NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT);
+                    NullStatsLogger.INSTANCE, UnpooledByteBufAllocator.DEFAULT,
+                    new MockUncleanShutdownDetection());
             server.start();
 
             int secondsToWait = 5;
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/ZooKeeperCluster.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/ZooKeeperCluster.java
index 7713503..2b28bfc 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/ZooKeeperCluster.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/ZooKeeperCluster.java
@@ -20,10 +20,13 @@
  */
 package org.apache.bookkeeper.test;
 
+import static java.nio.charset.StandardCharsets.UTF_8;
 import static org.apache.bookkeeper.util.BookKeeperConstants.AVAILABLE_NODE;
+import static org.apache.bookkeeper.util.BookKeeperConstants.INSTANCEID;
 import static org.apache.bookkeeper.util.BookKeeperConstants.READONLY;
 
 import java.io.IOException;
+import java.util.UUID;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
 
@@ -74,6 +77,9 @@ public interface ZooKeeperCluster {
         txn.create(ledgersPath + "/" + AVAILABLE_NODE, new byte[0], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
         txn.create(ledgersPath + "/" + AVAILABLE_NODE + "/" + READONLY, new byte[0], Ids.OPEN_ACL_UNSAFE,
                 CreateMode.PERSISTENT);
+        String instanceId = UUID.randomUUID().toString();
+        txn.create(ledgersPath + "/" + INSTANCEID, instanceId.getBytes(UTF_8),
+                Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
         txn.commit();
     }
 }
diff --git a/build.gradle b/build.gradle
index e6147a3..ce3d3c0 100644
--- a/build.gradle
+++ b/build.gradle
@@ -82,7 +82,7 @@ allprojects {
                 source ='src/test/java'
             }
         }
- 
+
         spotbugs {
             toolVersion = '3.1.8'
             excludeFilter = file("$rootDir/buildtools/src/main/resources/bookkeeper/findbugsExclude.xml")
@@ -239,7 +239,4 @@ allprojects {
     repositories {
         mavenCentral()
     }
-
-}
-
-
+}
\ No newline at end of file
diff --git a/dependencies.gradle b/dependencies.gradle
index bf98f82..f023869 100644
--- a/dependencies.gradle
+++ b/dependencies.gradle
@@ -73,6 +73,7 @@ depVersions = [
     protobuf: "3.14.0",
     reflections: "0.9.11",
     rocksDb: "6.22.1.1",
+    rxjava: "3.0.1",
     slf4j: "1.7.32",
     snakeyaml: "1.19",
     spotbugsAnnotations: "3.1.8",
@@ -196,6 +197,7 @@ depLibs = [
     protoc: "com.google.protobuf:protoc:${depVersions.protobuf}",
     reflections: "org.reflections:reflections:${depVersions.reflections}",
     rocksDb: "org.rocksdb:rocksdbjni:${depVersions.rocksDb}",
+    rxjava: "io.reactivex.rxjava3:rxjava:${depVersions.rxjava}",
     slf4j: "org.slf4j:slf4j-api:${depVersions.slf4j}",
     slf4jSimple: "org.slf4j:slf4j-simple:${depVersions.slf4j}",
     shrinkwrapImpl: "org.jboss.shrinkwrap.resolver:shrinkwrap-resolver-impl-maven:${depVersions.shrinkwrap}",
diff --git a/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/testing/EtcdBKClusterTestBase.java b/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/testing/EtcdBKClusterTestBase.java
index ee4cf65..c7e81f4 100644
--- a/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/testing/EtcdBKClusterTestBase.java
+++ b/metadata-drivers/etcd/src/test/java/org/apache/bookkeeper/metadata/etcd/testing/EtcdBKClusterTestBase.java
@@ -23,6 +23,7 @@ import java.util.ArrayList;
 import java.util.List;
 import lombok.extern.slf4j.Slf4j;
 import org.apache.bookkeeper.bookie.Bookie;
+import org.apache.bookkeeper.bookie.MockUncleanShutdownDetection;
 import org.apache.bookkeeper.bookie.TestBookieImpl;
 import org.apache.bookkeeper.client.BookKeeperAdmin;
 import org.apache.bookkeeper.client.api.BookKeeper;
@@ -125,7 +126,8 @@ public abstract class EtcdBKClusterTestBase extends EtcdTestBase {
         TestStatsProvider provider = new TestStatsProvider();
         Bookie bookie = new TestBookieImpl(conf);
         BookieServer server = new BookieServer(conf, bookie, provider.getStatsLogger(""),
-                                               ByteBufAllocator.DEFAULT);
+                                               ByteBufAllocator.DEFAULT,
+                                               new MockUncleanShutdownDetection());
         server.start();
         return server;
     }
diff --git a/stream/server/src/main/java/org/apache/bookkeeper/stream/server/service/BookieService.java b/stream/server/src/main/java/org/apache/bookkeeper/stream/server/service/BookieService.java
index 88e5392..7d34b4d 100644
--- a/stream/server/src/main/java/org/apache/bookkeeper/stream/server/service/BookieService.java
+++ b/stream/server/src/main/java/org/apache/bookkeeper/stream/server/service/BookieService.java
@@ -35,6 +35,8 @@ import org.apache.bookkeeper.bookie.LedgerDirsManager;
 import org.apache.bookkeeper.bookie.LedgerStorage;
 import org.apache.bookkeeper.bookie.LegacyCookieValidation;
 import org.apache.bookkeeper.bookie.ReadOnlyBookie;
+import org.apache.bookkeeper.bookie.UncleanShutdownDetection;
+import org.apache.bookkeeper.bookie.UncleanShutdownDetectionImpl;
 import org.apache.bookkeeper.common.component.AbstractLifecycleComponent;
 import org.apache.bookkeeper.common.component.ComponentInfoPublisher;
 import org.apache.bookkeeper.conf.ServerConfiguration;
@@ -98,6 +100,7 @@ public class BookieService extends AbstractLifecycleComponent<BookieConfiguratio
                 serverConf, diskChecker, bookieStats.scope(LD_INDEX_SCOPE), ledgerDirsManager);
         LedgerStorage storage = BookieResources.createLedgerStorage(
                 serverConf, ledgerManager, ledgerDirsManager, indexDirsManager, bookieStats, allocator);
+        UncleanShutdownDetection uncleanShutdownDetection = new UncleanShutdownDetectionImpl(ledgerDirsManager);
 
         LegacyCookieValidation cookieValidation = new LegacyCookieValidation(serverConf, rm);
         cookieValidation.checkCookies(Main.storageDirectoriesFromConf(serverConf));
@@ -116,8 +119,7 @@ public class BookieService extends AbstractLifecycleComponent<BookieConfiguratio
         }
 
         this.bs = new BookieServer(serverConf, bookie,
-                statsLogger, allocator);
-
+                statsLogger, allocator, uncleanShutdownDetection);
         log.info(hello);
     }