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/08/28 05:40:38 UTC

[bookkeeper] branch master updated: ISSUE #1623: ReadOnlyLedgerHandle: don't schedule monitorPendingAddOps()

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 9279dcc  ISSUE #1623: ReadOnlyLedgerHandle: don't schedule monitorPendingAddOps()
9279dcc is described below

commit 9279dcc13b95406dbcb7a333fcb8a8961f52bf16
Author: Samuel Just <sj...@salesforce.com>
AuthorDate: Mon Aug 27 22:40:27 2018 -0700

    ISSUE #1623: ReadOnlyLedgerHandle: don't schedule monitorPendingAddOps()
    
    The LedgerHandle constructor schedules an addEntryQuorumTimeout check
    with the bk client scheduler. However, the only place this callback is
    canceled is in the closeAsync (the one which returns a future, not to be
    confused with asyncClose) method. asyncClose and close() both leak this
    callback. Moreover, ReadOnlyLedgerHandle invokes the LedgerHandle
    constructor and so also creates this callback, but it overrides close()
    and asyncClose() without passing them through.
    
    ReadOnlyLedgerHandle already overrides
    initializeExplicitLacFlushPolicy() to avoid write specific state.  This
    patch generalizes that hack to initializeWriteHandleState() and the
    cleanup to tearDownWriteHandleState().  tearDownWriteHandleState() is
    moved into doAsyncClose(), which appears to be called for closes in
    general.
    
    (rev cguttapalem)
    (bug W-5362724)
    Signed-off-by: Samuel Just <sjustsalesforce.com>
    
    Author: Samuel Just <sj...@salesforce.com>
    
    Reviewers: Ivan Kelly <iv...@apache.org>, Enrico Olivelli <eo...@gmail.com>, Sijie Guo <si...@apache.org>
    
    This closes #1624 from athanatos/forupstream/wip-1623, closes #1623
---
 .../org/apache/bookkeeper/client/LedgerHandle.java | 40 ++++++++++++----------
 .../bookkeeper/client/ReadOnlyLedgerHandle.java    |  3 +-
 2 files changed, 23 insertions(+), 20 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 3e9f1c2..741610d 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
@@ -238,15 +238,25 @@ public class LedgerHandle implements WriteHandle {
                     return pendingAddOps.size();
                 }
             });
-        initializeExplicitLacFlushPolicy();
+
+        initializeWriteHandleState();
+    }
+
+    protected void initializeWriteHandleState() {
+        if (clientCtx.getConf().explicitLacInterval > 0) {
+            explicitLacFlushPolicy = new ExplicitLacFlushPolicy.ExplicitLacFlushPolicyImpl(
+                    this, clientCtx);
+        } else {
+            explicitLacFlushPolicy = ExplicitLacFlushPolicy.VOID_EXPLICITLAC_FLUSH_POLICY;
+        }
 
         if (clientCtx.getConf().addEntryQuorumTimeoutNanos > 0) {
             SafeRunnable monitor = new SafeRunnable() {
-                    @Override
-                    public void safeRun() {
-                        monitorPendingAddOps();
-                    }
-                };
+                @Override
+                public void safeRun() {
+                    monitorPendingAddOps();
+                }
+            };
             this.timeoutFuture = clientCtx.getScheduler().scheduleAtFixedRate(
                     monitor,
                     clientCtx.getConf().timeoutMonitorIntervalSec,
@@ -255,14 +265,10 @@ public class LedgerHandle implements WriteHandle {
         }
     }
 
-    protected void initializeExplicitLacFlushPolicy() {
-        if (!getLedgerMetadata().isClosed()
-            && !(this instanceof ReadOnlyLedgerHandle)
-            && clientCtx.getConf().explicitLacInterval > 0) {
-            explicitLacFlushPolicy = new ExplicitLacFlushPolicy.ExplicitLacFlushPolicyImpl(
-                    this, clientCtx);
-        } else {
-            explicitLacFlushPolicy = ExplicitLacFlushPolicy.VOID_EXPLICITLAC_FLUSH_POLICY;
+    private void tearDownWriteHandleState() {
+        explicitLacFlushPolicy.stopExplicitLacFlush();
+        if (timeoutFuture != null) {
+            timeoutFuture.cancel(false);
         }
     }
 
@@ -445,10 +451,6 @@ public class LedgerHandle implements WriteHandle {
         CompletableFuture<Void> result = new CompletableFuture<>();
         SyncCloseCallback callback = new SyncCloseCallback(result);
         asyncClose(callback, null);
-        explicitLacFlushPolicy.stopExplicitLacFlush();
-        if (timeoutFuture != null) {
-            timeoutFuture.cancel(false);
-        }
         return result;
     }
 
@@ -625,7 +627,7 @@ public class LedgerHandle implements WriteHandle {
                 }
 
                 writeLedgerConfig(new CloseCb());
-
+                tearDownWriteHandleState();
             }
 
             @Override
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadOnlyLedgerHandle.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadOnlyLedgerHandle.java
index e9f7900..aa0290b 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadOnlyLedgerHandle.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadOnlyLedgerHandle.java
@@ -183,7 +183,8 @@ class ReadOnlyLedgerHandle extends LedgerHandle implements LedgerMetadataListene
     }
 
     @Override
-    protected void initializeExplicitLacFlushPolicy() {
+    protected void initializeWriteHandleState() {
+        // Essentially a noop, we don't want to set up write handle state here for a ReadOnlyLedgerHandle
         explicitLacFlushPolicy = ExplicitLacFlushPolicy.VOID_EXPLICITLAC_FLUSH_POLICY;
     }