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/29 17:35:25 UTC

[bookkeeper] branch branch-4.7 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 branch-4.7
in repository https://gitbox.apache.org/repos/asf/bookkeeper.git


The following commit(s) were added to refs/heads/branch-4.7 by this push:
     new dfe1232  ISSUE #1623: ReadOnlyLedgerHandle: don't schedule monitorPendingAddOps()
dfe1232 is described below

commit dfe1232781680828d57b093b4077663477a79d76
Author: Samuel Just <sj...@salesforce.com>
AuthorDate: Wed Aug 29 10:35:17 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 <sjustsalesforce.com>
    
    Reviewers: Ivan Kelly <ivankapache.org>, Enrico Olivelli <eolivelligmail.com>, Sijie Guo <sijieapache.org>
    
    This closes #1624 from athanatos/forupstream/wip-1623, closes #1623
    
    (cherry picked from commit 9279dcc13b95406dbcb7a333fcb8a8961f52bf16)
    
    Conflicts:
    	bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandle.java
    
    Conflicts mainly revolved around refactors in LedgerHandle to remove the
    explicit bkclient.
    
    (cherry picked from commit e58e0747e93bed3c4bff5a2b866986fdba621c4e)
    
    Conflicts:
    	bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandle.java
    
    Author: Samuel Just <sj...@salesforce.com>
    
    Reviewers: Enrico Olivelli <eo...@gmail.com>, Sijie Guo <si...@apache.org>
    
    This closes #1633 from athanatos/forupstream/wip-1623-4.7, closes #1623
---
 .../org/apache/bookkeeper/client/LedgerHandle.java | 45 ++++++++++++----------
 .../bookkeeper/client/ReadOnlyLedgerHandle.java    |  3 +-
 2 files changed, 26 insertions(+), 22 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 3c53a03..58c3cf5 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
@@ -217,32 +217,39 @@ public class LedgerHandle implements WriteHandle {
                                                   return pendingAddOps.size();
                                               }
                                           });
-        initializeExplicitLacFlushPolicy();
-
-        if (bk.getConf().getAddEntryQuorumTimeout() > 0) {
-            SafeRunnable monitor = new SafeRunnable() {
-                    @Override
-                    public void safeRun() {
-                        monitorPendingAddOps();
-                    }
-                };
-            this.timeoutFuture = bk.scheduler.scheduleAtFixedRate(monitor,
-                                                                  bk.getConf().getTimeoutMonitorIntervalSec(),
-                                                                  bk.getConf().getTimeoutMonitorIntervalSec(),
-                                                                  TimeUnit.SECONDS);
-        }
+        initializeWriteHandleState();
     }
 
     BookKeeper getBk() {
         return bk;
     }
 
-    protected void initializeExplicitLacFlushPolicy() {
-        if (!metadata.isClosed() && !(this instanceof ReadOnlyLedgerHandle) && bk.getExplicitLacInterval() > 0) {
+    protected void initializeWriteHandleState() {
+        if (bk.getExplicitLacInterval() > 0) {
             explicitLacFlushPolicy = new ExplicitLacFlushPolicy.ExplicitLacFlushPolicyImpl(this);
         } else {
             explicitLacFlushPolicy = ExplicitLacFlushPolicy.VOID_EXPLICITLAC_FLUSH_POLICY;
         }
+
+        if (bk.getConf().getAddEntryQuorumTimeout() > 0) {
+            SafeRunnable monitor = new SafeRunnable() {
+                @Override
+                public void safeRun() {
+                    monitorPendingAddOps();
+                }
+            };
+            this.timeoutFuture = bk.scheduler.scheduleAtFixedRate(monitor,
+                    bk.getConf().getTimeoutMonitorIntervalSec(),
+                    bk.getConf().getTimeoutMonitorIntervalSec(),
+                    TimeUnit.SECONDS);
+        }
+    }
+
+    private void tearDownWriteHandleState() {
+        explicitLacFlushPolicy.stopExplicitLacFlush();
+        if (timeoutFuture != null) {
+            timeoutFuture.cancel(false);
+        }
     }
 
     /**
@@ -412,10 +419,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;
     }
 
@@ -588,7 +591,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 eef0e70..9aa1489 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
@@ -177,7 +177,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;
     }