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

[bookkeeper] branch master updated: Multiple calls to LedgerHandle#close should wait on actual close

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

sijie pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/bookkeeper.git


The following commit(s) were added to refs/heads/master by this push:
     new 52d1957  Multiple calls to LedgerHandle#close should wait on actual close
52d1957 is described below

commit 52d195700381bb3782a86e983f1a8c62d12bad2a
Author: Ivan Kelly <iv...@apache.org>
AuthorDate: Sat Dec 22 00:41:22 2018 +0100

    Multiple calls to LedgerHandle#close should wait on actual close
    
    The previous behaviour was to complete successfully immediately if
    close had already been called on the handle. This allowed for
    potential consistency violations, as the caller could believe that the
    ledger was closed, when in fact the close operation was still in
    progress and could still potentially fail.
    
    Issue: #1712
    
    
    Reviewers: Sijie Guo <si...@apache.org>
    
    This closes #1889 from ivankelly/double-close-anomoly
---
 .../org/apache/bookkeeper/client/LedgerHandle.java | 22 ++++++++-----
 .../apache/bookkeeper/client/LedgerClose2Test.java | 36 ++++++++++++++++++++++
 2 files changed, 50 insertions(+), 8 deletions(-)

diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandle.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandle.java
index 6f0af27..157a1b8 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandle.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandle.java
@@ -106,6 +106,7 @@ public class LedgerHandle implements WriteHandle {
     };
 
     private HandleState handleState = HandleState.OPEN;
+    private final CompletableFuture<Void> closePromise = new CompletableFuture<>();
 
     /**
       * Last entryId which has been confirmed to be written durably to the bookies.
@@ -508,6 +509,16 @@ public class LedgerHandle implements WriteHandle {
                 final long lastEntry;
                 final long finalLength;
 
+                closePromise.whenComplete((ignore, ex) -> {
+                        if (ex != null) {
+                            cb.closeComplete(
+                                    BKException.getExceptionCode(ex, BKException.Code.UnexpectedConditionException),
+                                    LedgerHandle.this, ctx);
+                        } else {
+                            cb.closeComplete(BKException.Code.OK, LedgerHandle.this, ctx);
+                        }
+                    });
+
                 synchronized (LedgerHandle.this) {
                     prevHandleState = handleState;
 
@@ -524,9 +535,7 @@ public class LedgerHandle implements WriteHandle {
                 // running under any bk locks.
                 errorOutPendingAdds(rc, pendingAdds);
 
-                if (prevHandleState == HandleState.CLOSED) {
-                    cb.closeComplete(BKException.Code.OK, LedgerHandle.this, ctx);
-                } else {
+                if (prevHandleState != HandleState.CLOSED) {
                     if (LOG.isDebugEnabled()) {
                         LOG.debug("Closing ledger: {} at entryId {} with {} bytes", getId(), lastEntry, finalLength);
                     }
@@ -564,12 +573,9 @@ public class LedgerHandle implements WriteHandle {
                             LedgerHandle.this::setLedgerMetadata)
                         .run().whenComplete((metadata, ex) -> {
                                 if (ex != null) {
-                                    cb.closeComplete(
-                                            BKException.getExceptionCode(
-                                                    ex, BKException.Code.UnexpectedConditionException),
-                                            LedgerHandle.this, ctx);
+                                    closePromise.completeExceptionally(ex);
                                 } else {
-                                    cb.closeComplete(BKException.Code.OK, LedgerHandle.this, ctx);
+                                    FutureUtils.complete(closePromise, null);
                                 }
                         });
                 }
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerClose2Test.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerClose2Test.java
index 45b2713..612b36e 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerClose2Test.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerClose2Test.java
@@ -268,5 +268,41 @@ public class LedgerClose2Test {
         Assert.assertEquals(lh.getLedgerMetadata().getLastEntryId(), LedgerHandle.INVALID_ENTRY_ID);
         Assert.assertEquals(lh.getLedgerMetadata().getLength(), 0);
     }
+
+    @Test
+    public void testDoubleCloseOnHandle() throws Exception {
+        long ledgerId = 123L;
+        MockClientContext clientCtx = MockClientContext.create();
+
+        Versioned<LedgerMetadata> md = ClientUtil.setupLedger(clientCtx, ledgerId,
+                LedgerMetadataBuilder.create()
+                .withEnsembleSize(3).withWriteQuorumSize(3).withAckQuorumSize(3)
+                .newEnsembleEntry(0L, Lists.newArrayList(b1, b2, b3)));
+
+        CompletableFuture<Void> metadataPromise = new CompletableFuture<>();
+        CompletableFuture<Void> clientPromise = new CompletableFuture<>();
+
+        LedgerHandle writer = new LedgerHandle(clientCtx, ledgerId, md,
+                                               BookKeeper.DigestType.CRC32C,
+                                               ClientUtil.PASSWD, WriteFlag.NONE);
+        long eid1 = writer.append("entry1".getBytes());
+
+        log.info("block writes from completing on bookies and metadata");
+        clientCtx.getMockBookieClient().setPostWriteHook((bookie, lid, eid) -> clientPromise);
+        clientCtx.getMockLedgerManager().setPreWriteHook((lid, metadata) -> metadataPromise);
+
+        log.info("try to add another entry, it will block");
+        writer.appendAsync("entry2".getBytes());
+
+        log.info("attempt one close, should block forever");
+        CompletableFuture<Void> firstClose = writer.closeAsync();
+
+        log.info("attempt second close, should not finish before first one");
+        CompletableFuture<Void> secondClose = writer.closeAsync();
+
+        Thread.sleep(500); // give it a chance to complete, the request jumps around threads
+        Assert.assertFalse(firstClose.isDone());
+        Assert.assertFalse(secondClose.isDone());
+    }
 }