You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@zookeeper.apache.org by iv...@apache.org on 2014/05/30 14:01:34 UTC

svn commit: r1598565 - in /zookeeper/bookkeeper/trunk: ./ bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ bookkeeper-server/src/test/java/org/apache/bookkeeper/client/

Author: ivank
Date: Fri May 30 12:01:34 2014
New Revision: 1598565

URL: http://svn.apache.org/r1598565
Log:
BOOKKEEPER-758: Add TryReadLastAddConfirmed API (sijie via ivank)

Added:
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/TryReadLastConfirmedOp.java
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestTryReadLastConfirmed.java
Modified:
    zookeeper/bookkeeper/trunk/CHANGES.txt
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeperClientStats.java
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandle.java

Modified: zookeeper/bookkeeper/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/CHANGES.txt?rev=1598565&r1=1598564&r2=1598565&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/CHANGES.txt (original)
+++ zookeeper/bookkeeper/trunk/CHANGES.txt Fri May 30 12:01:34 2014
@@ -196,6 +196,8 @@ Trunk (unreleased changes)
 
         BOOKKEEPER-756: Use HashedwheelTimer for request timeouts for PCBC (sijie via ivank)
 
+        BOOKKEEPER-758: Add TryReadLastAddConfirmed API (sijie via ivank)
+
       hedwig-server:
 
         BOOKKEEPER-601: readahead cache size isn't updated correctly (sijie via fpj)

Modified: zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeperClientStats.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeperClientStats.java?rev=1598565&r1=1598564&r2=1598565&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeperClientStats.java (original)
+++ zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeperClientStats.java Fri May 30 12:01:34 2014
@@ -30,6 +30,8 @@ public interface BookKeeperClientStats {
     public final static String READ_OP = "READ_ENTRY";
     public final static String PENDING_ADDS = "NUM_PENDING_ADD";
     public final static String ENSEMBLE_CHANGES = "NUM_ENSEMBLE_CHANGE";
+    public final static String LAC_UPDATE_HITS = "LAC_UPDATE_HITS";
+    public final static String LAC_UPDATE_MISSES = "LAC_UPDATE_MISSES";
 
     // per channel stats
     public final static String CHANNEL_SCOPE = "per_channel_bookie_client";

Modified: zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandle.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandle.java?rev=1598565&r1=1598564&r2=1598565&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandle.java (original)
+++ zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandle.java Fri May 30 12:01:34 2014
@@ -29,6 +29,7 @@ import java.util.Enumeration;
 import java.util.Queue;
 import java.util.concurrent.ConcurrentLinkedQueue;
 import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.bookkeeper.client.AsyncCallback.AddCallback;
@@ -79,6 +80,8 @@ public class LedgerHandle {
     final Queue<PendingAddOp> pendingAddOps = new ConcurrentLinkedQueue<PendingAddOp>();
 
     final Counter ensembleChangeCounter;
+    final Counter lacUpdateHitsCounter;
+    final Counter lacUpdateMissesCounter;
 
     LedgerHandle(BookKeeper bk, long ledgerId, LedgerMetadata metadata,
                  DigestType digestType, byte[] password)
@@ -104,6 +107,8 @@ public class LedgerHandle {
                 metadata.getWriteQuorumSize(), metadata.getAckQuorumSize(), metadata.getEnsembleSize());
 
         ensembleChangeCounter = bk.getStatsLogger().getCounter(BookKeeperClientStats.ENSEMBLE_CHANGES);
+        lacUpdateHitsCounter = bk.getStatsLogger().getCounter(BookKeeperClientStats.LAC_UPDATE_HITS);
+        lacUpdateMissesCounter = bk.getStatsLogger().getCounter(BookKeeperClientStats.LAC_UPDATE_MISSES);
         bk.getStatsLogger().registerGauge(BookKeeperClientStats.PENDING_ADDS,
                                           new Gauge<Integer>() {
                                               public Integer getDefaultValue() {
@@ -547,6 +552,17 @@ public class LedgerHandle {
         }
     }
 
+    synchronized void updateLastConfirmed(long lac, long len) {
+        if (lac > lastAddConfirmed) {
+            lastAddConfirmed = lac;
+            lacUpdateHitsCounter.inc();
+        } else {
+            lacUpdateMissesCounter.inc();
+        }
+        lastAddPushed = Math.max(lastAddPushed, lac);
+        length = Math.max(length, len);
+    }
+
     /**
      * Obtains asynchronously the last confirmed write from a quorum of bookies. This
      * call obtains the the last add confirmed each bookie has received for this ledger
@@ -577,9 +593,7 @@ public class LedgerHandle {
                 @Override
                 public void readLastConfirmedDataComplete(int rc, DigestManager.RecoveryData data) {
                     if (rc == BKException.Code.OK) {
-                        lastAddConfirmed = Math.max(lastAddConfirmed, data.lastAddConfirmed);
-                        lastAddPushed = Math.max(lastAddPushed, data.lastAddConfirmed);
-                        length = Math.max(length, data.length);
+                        updateLastConfirmed(data.lastAddConfirmed, data.length);
                         cb.readLastConfirmedComplete(rc, data.lastAddConfirmed, ctx);
                     } else {
                         cb.readLastConfirmedComplete(rc, INVALID_ENTRY_ID, ctx);
@@ -589,6 +603,49 @@ public class LedgerHandle {
         new ReadLastConfirmedOp(this, innercb).initiate();
     }
 
+    /**
+     * Obtains asynchronously the last confirmed write from a quorum of bookies.
+     * It is similar as
+     * {@link #asyncTryReadLastConfirmed(org.apache.bookkeeper.client.AsyncCallback.ReadLastConfirmedCallback, Object)},
+     * but it doesn't wait all the responses from the quorum. It would callback
+     * immediately if it received a LAC which is larger than current LAC.
+     *
+     * @see #asyncTryReadLastConfirmed(org.apache.bookkeeper.client.AsyncCallback.ReadLastConfirmedCallback, Object)
+     *
+     * @param cb
+     *          callback to return read last confirmed
+     * @param ctx
+     *          callback context
+     */
+    public void asyncTryReadLastConfirmed(final ReadLastConfirmedCallback cb, final Object ctx) {
+        boolean isClosed;
+        long lastEntryId;
+        synchronized (this) {
+            isClosed = metadata.isClosed();
+            lastEntryId = metadata.getLastEntryId();
+        }
+        if (isClosed) {
+            cb.readLastConfirmedComplete(BKException.Code.OK, lastEntryId, ctx);
+            return;
+        }
+        ReadLastConfirmedOp.LastConfirmedDataCallback innercb = new ReadLastConfirmedOp.LastConfirmedDataCallback() {
+            AtomicBoolean completed = new AtomicBoolean(false);
+            @Override
+            public void readLastConfirmedDataComplete(int rc, DigestManager.RecoveryData data) {
+                if (rc == BKException.Code.OK) {
+                    updateLastConfirmed(data.lastAddConfirmed, data.length);
+                    if (completed.compareAndSet(false, true)) {
+                        cb.readLastConfirmedComplete(rc, data.lastAddConfirmed, ctx);
+                    }
+                } else {
+                    if (completed.compareAndSet(false, true)) {
+                        cb.readLastConfirmedComplete(rc, INVALID_ENTRY_ID, ctx);
+                    }
+                }
+            }
+        };
+        new TryReadLastConfirmedOp(this, innercb, getLastAddConfirmed()).initiate();
+    }
 
     /**
      * Context objects for synchronous call to read last confirmed.
@@ -653,6 +710,31 @@ public class LedgerHandle {
         return ctx.getlastConfirmed();
     }
 
+    /**
+     * Obtains synchronously the last confirmed write from a quorum of bookies.
+     * It is similar as {@link #readLastConfirmed()}, but it doesn't wait all the responses
+     * from the quorum. It would callback immediately if it received a LAC which is larger
+     * than current LAC.
+     *
+     * @see #readLastConfirmed()
+     *
+     * @return The entry id of the last confirmed write or {@link #INVALID_ENTRY_ID INVALID_ENTRY_ID}
+     *         if no entry has been confirmed
+     * @throws InterruptedException
+     * @throws BKException
+     */
+    public long tryReadLastConfirmed() throws InterruptedException, BKException {
+        LastConfirmedCtx ctx = new LastConfirmedCtx();
+        asyncTryReadLastConfirmed(new SyncReadLastConfirmedCallback(), ctx);
+        synchronized (ctx) {
+            while (!ctx.ready()) {
+                ctx.wait();
+            }
+        }
+        if (ctx.getRC() != BKException.Code.OK) throw BKException.create(ctx.getRC());
+        return ctx.getlastConfirmed();
+    }
+
     // close the ledger and send fails to all the adds in the pipeline
     void handleUnrecoverableErrorDuringAdd(int rc) {
         if (metadata.isInRecovery()) {

Added: zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/TryReadLastConfirmedOp.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/TryReadLastConfirmedOp.java?rev=1598565&view=auto
==============================================================================
--- zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/TryReadLastConfirmedOp.java (added)
+++ zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/TryReadLastConfirmedOp.java Fri May 30 12:01:34 2014
@@ -0,0 +1,107 @@
+/**
+ * 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.client;
+
+import org.apache.bookkeeper.client.DigestManager.RecoveryData;
+import org.apache.bookkeeper.client.ReadLastConfirmedOp.LastConfirmedDataCallback;
+import org.apache.bookkeeper.proto.BookieProtocol;
+import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ReadEntryCallback;
+import org.jboss.netty.buffer.ChannelBuffer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+
+/**
+ * This op is try to read last confirmed without involving quorum coverage checking.
+ * Use {@link ReadLastConfirmedOp} if you need quorum coverage checking.
+ */
+class TryReadLastConfirmedOp implements ReadEntryCallback {
+
+    static final Logger LOG = LoggerFactory.getLogger(TryReadLastConfirmedOp.class);
+
+    final LedgerHandle lh;
+    final LastConfirmedDataCallback cb;
+
+    int numResponsesPending;
+    volatile boolean hasValidResponse = false;
+    volatile boolean completed = false;
+    RecoveryData maxRecoveredData;
+
+    TryReadLastConfirmedOp(LedgerHandle lh, LastConfirmedDataCallback cb, long lac) {
+        this.lh = lh;
+        this.cb = cb;
+        this.maxRecoveredData = new RecoveryData(lac, 0);
+        this.numResponsesPending = lh.metadata.getEnsembleSize();
+    }
+
+    public void initiate() {
+        for (int i = 0; i < lh.metadata.currentEnsemble.size(); i++) {
+            lh.bk.bookieClient.readEntry(lh.metadata.currentEnsemble.get(i),
+                                         lh.ledgerId,
+                                         BookieProtocol.LAST_ADD_CONFIRMED,
+                                         this, i);
+        }
+    }
+
+    @Override
+    public void readEntryComplete(int rc, long ledgerId, long entryId, ChannelBuffer buffer, Object ctx) {
+        if (LOG.isTraceEnabled()) {
+            LOG.trace("TryReadLastConfirmed received response for (lid={}, eid={}) : {}",
+                    new Object[] { ledgerId, entryId, rc });
+        }
+
+        int bookieIndex = (Integer) ctx;
+        numResponsesPending--;
+        if (BKException.Code.OK == rc) {
+            try {
+                RecoveryData recoveryData = lh.macManager.verifyDigestAndReturnLastConfirmed(buffer);
+                if (LOG.isTraceEnabled()) {
+                    LOG.trace("Received lastAddConfirmed (lac={}, length={}) from bookie({}) for (lid={}).",
+                            new Object[] { recoveryData.lastAddConfirmed, recoveryData.length, bookieIndex, ledgerId });
+                }
+                if (recoveryData.lastAddConfirmed > maxRecoveredData.lastAddConfirmed) {
+                    maxRecoveredData = recoveryData;
+                    // callback immediately
+                    cb.readLastConfirmedDataComplete(BKException.Code.OK, maxRecoveredData);
+                }
+                hasValidResponse = true;
+            } catch (BKException.BKDigestMatchException e) {
+                LOG.error("Mac mismatch for ledger: " + ledgerId + ", entry: " + entryId
+                          + " while reading last entry from bookie: "
+                          + lh.metadata.currentEnsemble.get(bookieIndex));
+            }
+        } else if (BKException.Code.UnauthorizedAccessException == rc && !completed) {
+            cb.readLastConfirmedDataComplete(rc, maxRecoveredData);
+            completed = true;
+        } else if (BKException.Code.NoSuchLedgerExistsException == rc ||
+                   BKException.Code.NoSuchEntryException == rc) {
+            hasValidResponse = true;
+        }
+        if (numResponsesPending == 0 && !completed) {
+            if (!hasValidResponse) {
+                // no success called
+                cb.readLastConfirmedDataComplete(BKException.Code.LedgerRecoveryException, maxRecoveredData);
+            } else {
+                // callback
+                cb.readLastConfirmedDataComplete(BKException.Code.OK, maxRecoveredData);
+            }
+            completed = true;
+        }
+    }
+}

Added: zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestTryReadLastConfirmed.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestTryReadLastConfirmed.java?rev=1598565&view=auto
==============================================================================
--- zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestTryReadLastConfirmed.java (added)
+++ zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestTryReadLastConfirmed.java Fri May 30 12:01:34 2014
@@ -0,0 +1,187 @@
+/*
+ * 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.client;
+
+import org.apache.bookkeeper.client.BookKeeper.DigestType;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.test.BookKeeperClusterTestCase;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+
+public class TestTryReadLastConfirmed extends BookKeeperClusterTestCase {
+
+    static final Logger logger = LoggerFactory.getLogger(TestTryReadLastConfirmed.class);
+
+    final DigestType digestType;
+
+    public TestTryReadLastConfirmed() {
+        super(6);
+        this.digestType = DigestType.CRC32;
+    }
+
+    @Test(timeout = 60000)
+    public void testTryReadLACWhenAllBookiesUp() throws Exception {
+        final int numEntries = 3;
+
+        final LedgerHandle lh = bkc.createLedger(3, 3, 1, digestType, "".getBytes());
+        LedgerHandle readLh = bkc.openLedgerNoRecovery(lh.getId(), digestType, "".getBytes());
+        assertEquals(LedgerHandle.INVALID_ENTRY_ID, readLh.getLastAddConfirmed());
+        // add entries
+        for (int i = 0; i < numEntries; i++) {
+            lh.addEntry(("data" + i).getBytes());
+        }
+        final AtomicBoolean success = new AtomicBoolean(false);
+        final AtomicInteger numCallbacks = new AtomicInteger(0);
+        final CountDownLatch latch1 = new CountDownLatch(1);
+        readLh.asyncTryReadLastConfirmed(new AsyncCallback.ReadLastConfirmedCallback() {
+            @Override
+            public void readLastConfirmedComplete(int rc, long lastConfirmed, Object ctx) {
+                numCallbacks.incrementAndGet();
+                if (BKException.Code.OK == rc) {
+                    success.set(true);
+                } else {
+                    success.set(false);
+                }
+                latch1.countDown();
+            }
+        }, null);
+        latch1.await();
+        TimeUnit.SECONDS.sleep(2);
+        assertTrue(success.get());
+        assertTrue(numCallbacks.get() == 1);
+        assertEquals(numEntries - 2, readLh.getLastAddConfirmed());
+        // try read last confirmed again
+        success.set(false);
+        numCallbacks.set(0);
+        final CountDownLatch latch2 = new CountDownLatch(1);
+        readLh.asyncTryReadLastConfirmed(new AsyncCallback.ReadLastConfirmedCallback() {
+            @Override
+            public void readLastConfirmedComplete(int rc, long lastConfirmed, Object ctx) {
+                numCallbacks.incrementAndGet();
+                if (BKException.Code.OK == rc && lastConfirmed == (numEntries - 2)) {
+                    success.set(true);
+                } else {
+                    success.set(false);
+                }
+                latch2.countDown();
+            }
+        }, null);
+        latch2.await();
+        TimeUnit.SECONDS.sleep(2);
+        assertTrue(success.get());
+        assertTrue(numCallbacks.get() == 1);
+        assertEquals(numEntries - 2, readLh.getLastAddConfirmed());
+
+        lh.close();
+        readLh.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testTryReadLaCWhenSomeBookiesDown() throws Exception {
+        final int numEntries = 3;
+        final int ensembleSize = 3;
+        final LedgerHandle lh = bkc.createLedger(ensembleSize, 1, 1, digestType, "".getBytes());
+        LedgerHandle readLh = bkc.openLedgerNoRecovery(lh.getId(), digestType, "".getBytes());
+        assertEquals(LedgerHandle.INVALID_ENTRY_ID, readLh.getLastAddConfirmed());
+        // add entries
+        for (int i = 0; i < numEntries; i++) {
+            lh.addEntry(("data" + i).getBytes());
+        }
+        for (int i = 0; i < numEntries; i++) {
+            ServerConfiguration[] confs = new ServerConfiguration[ensembleSize - 1];
+            for (int j = 0; j < ensembleSize - 1; j++) {
+                int idx = (i + 1 + j) % ensembleSize;
+                confs[j] = killBookie(lh.getLedgerMetadata().currentEnsemble.get(idx));
+            }
+
+            final AtomicBoolean success = new AtomicBoolean(false);
+            final AtomicInteger numCallbacks = new AtomicInteger(0);
+            final CountDownLatch latch = new CountDownLatch(1);
+            final int entryId = i;
+            readLh.asyncTryReadLastConfirmed(new AsyncCallback.ReadLastConfirmedCallback() {
+                @Override
+                public void readLastConfirmedComplete(int rc, long lastConfirmed, Object ctx) {
+                    numCallbacks.incrementAndGet();
+                    if (BKException.Code.OK == rc) {
+                        success.set(lastConfirmed == (entryId - 1));
+                    } else {
+                        success.set(false);
+                    }
+                    latch.countDown();
+                }
+            }, null);
+            latch.await();
+            assertTrue(success.get());
+            assertTrue(numCallbacks.get() == 1);
+
+            lh.close();
+            readLh.close();
+
+            // start the bookies
+            for (ServerConfiguration conf : confs) {
+                bs.add(startBookie(conf));
+                bsConfs.add(conf);
+            }
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testTryReadLACWhenAllBookiesDown() throws Exception {
+        final int numEntries = 2;
+        final int ensembleSize = 3;
+        final LedgerHandle lh = bkc.createLedger(ensembleSize, 1, 1, digestType, "".getBytes());
+        LedgerHandle readLh = bkc.openLedgerNoRecovery(lh.getId(), digestType, "".getBytes());
+        assertEquals(LedgerHandle.INVALID_ENTRY_ID, readLh.getLastAddConfirmed());
+        // add entries
+        for (int i = 0; i < numEntries; i++) {
+            lh.addEntry(("data" + i).getBytes());
+        }
+        for (int i = 0; i < ensembleSize; i++) {
+            killBookie(lh.getLedgerMetadata().currentEnsemble.get(i));
+        }
+        final AtomicBoolean success = new AtomicBoolean(false);
+        final AtomicInteger numCallbacks = new AtomicInteger(0);
+        final CountDownLatch latch = new CountDownLatch(1);
+        readLh.asyncTryReadLastConfirmed(new AsyncCallback.ReadLastConfirmedCallback() {
+            @Override
+            public void readLastConfirmedComplete(int rc, long lastConfirmed, Object ctx) {
+                logger.info("ReadLastConfirmedComplete : rc = {}, lac = {}.", rc, lastConfirmed);
+                numCallbacks.incrementAndGet();
+                if (BKException.Code.OK == rc) {
+                    success.set(lastConfirmed == LedgerHandle.INVALID_ENTRY_ID);
+                } else {
+                    success.set(false);
+                }
+                latch.countDown();
+            }
+        }, null);
+        latch.await();
+        TimeUnit.SECONDS.sleep(2);
+        assertFalse(success.get());
+        assertTrue(numCallbacks.get() == 1);
+
+        lh.close();
+        readLh.close();
+    }
+}