|
21 | 21 | package org.apache.bookkeeper.client; |
22 | 22 |
|
23 | 23 | import static org.apache.bookkeeper.common.concurrent.FutureUtils.result; |
| 24 | +import static org.junit.Assert.assertArrayEquals; |
24 | 25 | import static org.junit.Assert.assertEquals; |
25 | 26 | import static org.junit.Assert.assertFalse; |
26 | 27 | import static org.junit.Assert.assertNotNull; |
27 | 28 | import static org.junit.Assert.assertTrue; |
28 | 29 | import static org.junit.Assert.fail; |
29 | 30 |
|
| 31 | +import io.netty.buffer.ByteBuf; |
| 32 | +import java.io.IOException; |
| 33 | +import java.nio.charset.StandardCharsets; |
30 | 34 | import java.util.Iterator; |
31 | 35 | import java.util.List; |
32 | 36 | import java.util.concurrent.CompletableFuture; |
33 | 37 | import java.util.concurrent.CountDownLatch; |
| 38 | +import java.util.concurrent.TimeUnit; |
| 39 | +import org.apache.bookkeeper.bookie.Bookie; |
| 40 | +import org.apache.bookkeeper.bookie.BookieException; |
| 41 | +import org.apache.bookkeeper.bookie.TestBookieImpl; |
34 | 42 | import org.apache.bookkeeper.client.BKException.Code; |
35 | 43 | import org.apache.bookkeeper.client.BookKeeper.DigestType; |
36 | 44 | import org.apache.bookkeeper.client.api.LedgerEntry; |
37 | 45 | import org.apache.bookkeeper.conf.ClientConfiguration; |
| 46 | +import org.apache.bookkeeper.conf.ServerConfiguration; |
38 | 47 | import org.apache.bookkeeper.net.BookieId; |
39 | 48 | import org.apache.bookkeeper.test.BookKeeperClusterTestCase; |
| 49 | +import org.awaitility.Awaitility; |
40 | 50 | import org.junit.Test; |
41 | 51 | import org.slf4j.Logger; |
42 | 52 | import org.slf4j.LoggerFactory; |
@@ -289,4 +299,130 @@ public void testReadFailureWithFailedBookies() throws Exception { |
289 | 299 | lh.close(); |
290 | 300 | newBk.close(); |
291 | 301 | } |
| 302 | + |
| 303 | + @Test |
| 304 | + public void testDigestMismatchRetriesNextReplicaAndCompletes() throws Exception { |
| 305 | + ClientConfiguration conf = new ClientConfiguration(baseClientConf) |
| 306 | + .setUseV2WireProtocol(true) |
| 307 | + .setReorderReadSequenceEnabled(false) |
| 308 | + .setMetadataServiceUri(zkUtil.getMetadataServiceUri()); |
| 309 | + |
| 310 | + try (BookKeeperTestClient bk = new BookKeeperTestClient(conf)) { |
| 311 | + byte[] data = "batch-digest-data".getBytes(StandardCharsets.UTF_8); |
| 312 | + LedgerHandle writer = bk.createLedger(3, 3, 2, digestType, passwd); |
| 313 | + writer.addEntry(data); |
| 314 | + long ledgerId = writer.getId(); |
| 315 | + BookieId corruptReplica = writer.getLedgerMetadata().getAllEnsembles().get(0L).get(0); |
| 316 | + writer.close(); |
| 317 | + |
| 318 | + ServerConfiguration corruptConf = killBookie(corruptReplica); |
| 319 | + startAndAddBookie(corruptConf, new CorruptReadBookie(corruptConf)); |
| 320 | + |
| 321 | + LedgerHandle reader = bk.openLedger(ledgerId, digestType, passwd); |
| 322 | + BatchedReadOp readOp = new BatchedReadOp(reader, bk.getClientCtx(), 0, 1, 1024, false); |
| 323 | + readOp.submit(); |
| 324 | + |
| 325 | + Iterator<LedgerEntry> entries = readOp.future().get().iterator(); |
| 326 | + assertTrue(entries.hasNext()); |
| 327 | + LedgerEntry entry = entries.next(); |
| 328 | + assertArrayEquals(data, entry.getEntryBytes()); |
| 329 | + entry.close(); |
| 330 | + assertFalse(entries.hasNext()); |
| 331 | + reader.close(); |
| 332 | + } |
| 333 | + } |
| 334 | + |
| 335 | + @Test |
| 336 | + public void testDigestMismatchAfterPartialVerificationDoesNotRetainEntries() throws Exception { |
| 337 | + ClientConfiguration conf = new ClientConfiguration(baseClientConf) |
| 338 | + .setUseV2WireProtocol(true) |
| 339 | + .setReorderReadSequenceEnabled(false) |
| 340 | + .setMetadataServiceUri(zkUtil.getMetadataServiceUri()); |
| 341 | + |
| 342 | + try (BookKeeperTestClient bk = new BookKeeperTestClient(conf)) { |
| 343 | + byte[] entry0 = "batch-digest-entry-0".getBytes(StandardCharsets.UTF_8); |
| 344 | + byte[] entry1 = "batch-digest-entry-1".getBytes(StandardCharsets.UTF_8); |
| 345 | + LedgerHandle writer = bk.createLedger(3, 3, 2, digestType, passwd); |
| 346 | + writer.addEntry(entry0); |
| 347 | + writer.addEntry(entry1); |
| 348 | + long ledgerId = writer.getId(); |
| 349 | + List<BookieId> ensemble = writer.getLedgerMetadata().getAllEnsembles().get(0L); |
| 350 | + BookieId corruptReplica = ensemble.get(0); |
| 351 | + BookieId retryReplica = ensemble.get(1); |
| 352 | + writer.close(); |
| 353 | + |
| 354 | + CountDownLatch corruptReadLatch = new CountDownLatch(1); |
| 355 | + ServerConfiguration corruptConf = killBookie(corruptReplica); |
| 356 | + startAndAddBookie(corruptConf, new CorruptReadBookie(corruptConf, 1L, corruptReadLatch)); |
| 357 | + |
| 358 | + CountDownLatch retryLatch = new CountDownLatch(1); |
| 359 | + sleepBookie(retryReplica, retryLatch); |
| 360 | + |
| 361 | + LedgerHandle reader = null; |
| 362 | + try { |
| 363 | + reader = bk.openLedger(ledgerId, digestType, passwd); |
| 364 | + BatchedReadOp readOp = new BatchedReadOp(reader, bk.getClientCtx(), 0, 2, 2048, false); |
| 365 | + readOp.submit(); |
| 366 | + |
| 367 | + assertTrue("corrupt replica did not read the corrupted entry", |
| 368 | + corruptReadLatch.await(10, TimeUnit.SECONDS)); |
| 369 | + Awaitility.await().atMost(10, TimeUnit.SECONDS).untilAsserted(() -> { |
| 370 | + assertNotNull(readOp.request); |
| 371 | + BatchedReadOp.SequenceReadRequest request = |
| 372 | + (BatchedReadOp.SequenceReadRequest) readOp.request; |
| 373 | + assertTrue(request.nextReplicaIndexToReadFrom >= 2); |
| 374 | + }); |
| 375 | + assertTrue("digest mismatch must not retain partially verified entries", |
| 376 | + readOp.request.entries.isEmpty()); |
| 377 | + |
| 378 | + retryLatch.countDown(); |
| 379 | + Iterator<LedgerEntry> entries = readOp.future().get(10, TimeUnit.SECONDS).iterator(); |
| 380 | + assertTrue(entries.hasNext()); |
| 381 | + LedgerEntry first = entries.next(); |
| 382 | + assertArrayEquals(entry0, first.getEntryBytes()); |
| 383 | + first.close(); |
| 384 | + assertTrue(entries.hasNext()); |
| 385 | + LedgerEntry second = entries.next(); |
| 386 | + assertArrayEquals(entry1, second.getEntryBytes()); |
| 387 | + second.close(); |
| 388 | + assertFalse(entries.hasNext()); |
| 389 | + } finally { |
| 390 | + retryLatch.countDown(); |
| 391 | + if (reader != null) { |
| 392 | + reader.close(); |
| 393 | + } |
| 394 | + } |
| 395 | + } |
| 396 | + } |
| 397 | + |
| 398 | + static class CorruptReadBookie extends TestBookieImpl { |
| 399 | + private final long corruptEntryId; |
| 400 | + private final CountDownLatch corruptReadLatch; |
| 401 | + |
| 402 | + CorruptReadBookie(ServerConfiguration conf) throws Exception { |
| 403 | + this(conf, -1L, null); |
| 404 | + } |
| 405 | + |
| 406 | + CorruptReadBookie(ServerConfiguration conf, long corruptEntryId, CountDownLatch corruptReadLatch) |
| 407 | + throws Exception { |
| 408 | + super(conf); |
| 409 | + this.corruptEntryId = corruptEntryId; |
| 410 | + this.corruptReadLatch = corruptReadLatch; |
| 411 | + } |
| 412 | + |
| 413 | + @Override |
| 414 | + public ByteBuf readEntry(long ledgerId, long entryId) |
| 415 | + throws IOException, Bookie.NoLedgerException, BookieException { |
| 416 | + ByteBuf localBuf = super.readEntry(ledgerId, entryId); |
| 417 | + if (corruptEntryId < 0 || corruptEntryId == entryId) { |
| 418 | + for (int i = 0; i < localBuf.capacity(); i++) { |
| 419 | + localBuf.setByte(i, 0); |
| 420 | + } |
| 421 | + if (corruptReadLatch != null) { |
| 422 | + corruptReadLatch.countDown(); |
| 423 | + } |
| 424 | + } |
| 425 | + return localBuf; |
| 426 | + } |
| 427 | + } |
292 | 428 | } |
0 commit comments