public void testSpeculativeReadMultipleReplicasDown() throws Exception {
long id = getLedgerToRead(5,5);
int timeout = 5000;
BookKeeper bkspec = createClient(timeout);
LedgerHandle l = bkspec.openLedger(id, digestType, passwd);
// sleep bookie 1, 2 & 4
CountDownLatch sleepLatch = new CountDownLatch(1);
sleepBookie(l.getLedgerMetadata().getEnsembles().get(0L).get(1), sleepLatch);
sleepBookie(l.getLedgerMetadata().getEnsembles().get(0L).get(2), sleepLatch);
sleepBookie(l.getLedgerMetadata().getEnsembles().get(0L).get(4), sleepLatch);
try {
// read first entry, should complete faster than timeout
// as bookie 0 has the entry
LatchCallback latch0 = new LatchCallback();
l.asyncReadEntries(0, 0, latch0, null);
latch0.expectSuccess(timeout/2);
// second should have to hit two timeouts (bookie 1 & 2)
// bookie 3 has the entry
LatchCallback latch1 = new LatchCallback();
l.asyncReadEntries(1, 1, latch1, null);
latch1.expectTimeout(timeout);
latch1.expectSuccess(timeout*2);
LOG.info("Timeout {} latch1 duration {}", timeout, latch1.getDuration());
assertTrue("should have taken longer than two timeouts, but less than 3",
latch1.getDuration() >= timeout*2
&& latch1.getDuration() < timeout*3);
// third should have to hit one timeouts (bookie 2)
// bookie 3 has the entry
LatchCallback latch2 = new LatchCallback();
l.asyncReadEntries(2, 2, latch2, null);
latch2.expectTimeout(timeout/2);
latch2.expectSuccess(timeout);
LOG.info("Timeout {} latch2 duration {}", timeout, latch2.getDuration());
assertTrue("should have taken longer than one timeout, but less than 2",
latch2.getDuration() >= timeout
&& latch2.getDuration() < timeout*2);
// fourth should have no timeout
// bookie 3 has the entry
LatchCallback latch3 = new LatchCallback();
l.asyncReadEntries(3, 3, latch3, null);
latch3.expectSuccess(timeout/2);
// fifth should hit one timeout, (bookie 4)
// bookie 0 has the entry
LatchCallback latch4 = new LatchCallback();
l.asyncReadEntries(4, 4, latch4, null);
latch4.expectTimeout(timeout/2);
latch4.expectSuccess(timeout);
LOG.info("Timeout {} latch4 duration {}", timeout, latch4.getDuration());
assertTrue("should have taken longer than one timeout, but less than 2",
latch4.getDuration() >= timeout
&& latch4.getDuration() < timeout*2);
} finally {
sleepLatch.countDown();
l.close();
bkspec.close();
}
}