bookkeeper-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From si...@apache.org
Subject [bookkeeper] branch master updated: ISSUE #343: TestFencing.testManyOpenParallel
Date Mon, 28 Aug 2017 08:54:34 GMT
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 5c83b6e  ISSUE #343: TestFencing.testManyOpenParallel
5c83b6e is described below

commit 5c83b6e2ab7329c428b2c63415f3c695f4edaf12
Author: Sijie Guo <sijie@apache.org>
AuthorDate: Mon Aug 28 01:54:27 2017 -0700

    ISSUE #343: TestFencing.testManyOpenParallel
    
    Descriptions of the changes in this PR:
    
    Problem:
    
    The `LastAddConfirmed` was advanced due to piggyback on recovery adds. It will cause recovery
adds become stall because
    of the entry id check.
    
    This problem only occurs when there are multiple clients concurrently recover same ledger.
    
    Solution:
    
    Disable lac piggyback on recovery adds.
    
    Author: Sijie Guo <sijie@apache.org>
    
    Reviewers: Jia Zhai <None>, Enrico Olivelli <eolivelli@gmail.com>
    
    This closes #477 from sijie/tests, closes #343
---
 .../apache/bookkeeper/client/LedgerRecoveryOp.java |  3 +-
 .../client/ListenerBasedPendingReadOp.java         | 28 ++++++++++++++++---
 .../apache/bookkeeper/client/PendingReadOp.java    | 32 ++++++++++++++++++++--
 3 files changed, 55 insertions(+), 8 deletions(-)

diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerRecoveryOp.java
b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerRecoveryOp.java
index 5ea8935..1276fb0 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerRecoveryOp.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerRecoveryOp.java
@@ -24,6 +24,7 @@ import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.bookkeeper.client.AsyncCallback.AddCallback;
 import org.apache.bookkeeper.client.AsyncCallback.CloseCallback;
+import org.apache.bookkeeper.client.BKException.Code;
 import org.apache.bookkeeper.client.DigestManager.RecoveryData;
 import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.GenericCallback;
 import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ReadEntryListener;
@@ -63,7 +64,7 @@ class LedgerRecoveryOp implements ReadEntryListener, AddCallback {
         RecoveryReadOp(LedgerHandle lh, ScheduledExecutorService scheduler,
                        long startEntryId, long endEntryId,
                        ReadEntryListener cb, Object ctx) {
-            super(lh, scheduler, startEntryId, endEntryId, cb, ctx);
+            super(lh, scheduler, startEntryId, endEntryId, cb, ctx, true);
         }
 
         @Override
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ListenerBasedPendingReadOp.java
b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ListenerBasedPendingReadOp.java
index 69221d0..afb21bf 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ListenerBasedPendingReadOp.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ListenerBasedPendingReadOp.java
@@ -30,10 +30,30 @@ class ListenerBasedPendingReadOp extends PendingReadOp {
 
     final ReadEntryListener listener;
 
-    ListenerBasedPendingReadOp(LedgerHandle lh, ScheduledExecutorService scheduler,
-                               long startEntryId, long endEntryId,
-                               ReadEntryListener listener, Object ctx) {
-        super(lh, scheduler, startEntryId, endEntryId, null, ctx);
+    ListenerBasedPendingReadOp(LedgerHandle lh,
+                               ScheduledExecutorService scheduler,
+                               long startEntryId,
+                               long endEntryId,
+                               ReadEntryListener listener,
+                               Object ctx) {
+        this(
+            lh,
+            scheduler,
+            startEntryId,
+            endEntryId,
+            listener,
+            ctx,
+            false);
+    }
+
+    ListenerBasedPendingReadOp(LedgerHandle lh,
+                               ScheduledExecutorService scheduler,
+                               long startEntryId,
+                               long endEntryId,
+                               ReadEntryListener listener,
+                               Object ctx,
+                               boolean isRecoveryRead) {
+        super(lh, scheduler, startEntryId, endEntryId, null, ctx, isRecoveryRead);
         this.listener = listener;
     }
 
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingReadOp.java
b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingReadOp.java
index 3ca70ed..06b17ec 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingReadOp.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingReadOp.java
@@ -74,6 +74,7 @@ class PendingReadOp implements Enumeration<LedgerEntry>, ReadEntryCallback
{
     OpStatsLogger readOpLogger;
 
     final int maxMissedReadsAllowed;
+    final boolean isRecoveryRead;
     boolean parallelRead = false;
     final AtomicBoolean complete = new AtomicBoolean(false);
 
@@ -419,8 +420,29 @@ class PendingReadOp implements Enumeration<LedgerEntry>, ReadEntryCallback
{
 
     }
 
-    PendingReadOp(LedgerHandle lh, ScheduledExecutorService scheduler,
-                  long startEntryId, long endEntryId, ReadCallback cb, Object ctx) {
+    PendingReadOp(LedgerHandle lh,
+                  ScheduledExecutorService scheduler,
+                  long startEntryId,
+                  long endEntryId,
+                  ReadCallback cb,
+                  Object ctx) {
+        this(
+            lh,
+            scheduler,
+            startEntryId,
+            endEntryId,
+            cb,
+            ctx,
+            false);
+    }
+
+    PendingReadOp(LedgerHandle lh,
+                  ScheduledExecutorService scheduler,
+                  long startEntryId,
+                  long endEntryId,
+                  ReadCallback cb,
+                  Object ctx,
+                  boolean isRecoveryRead) {
         seq = new ArrayBlockingQueue<LedgerEntryRequest>((int) ((endEntryId + 1) -
startEntryId));
         this.cb = cb;
         this.ctx = ctx;
@@ -428,6 +450,7 @@ class PendingReadOp implements Enumeration<LedgerEntry>, ReadEntryCallback
{
         this.startEntryId = startEntryId;
         this.endEntryId = endEntryId;
         this.scheduler = scheduler;
+        this.isRecoveryRead = isRecoveryRead;
         numPendingEntries = endEntryId - startEntryId + 1;
         maxMissedReadsAllowed = getLedgerMetadata().getWriteQuorumSize()
                 - getLedgerMetadata().getAckQuorumSize();
@@ -527,7 +550,10 @@ class PendingReadOp implements Enumeration<LedgerEntry>, ReadEntryCallback
{
         heardFromHostsBitSet.set(rctx.bookieIndex, true);
 
         if (entry.complete(rctx.bookieIndex, rctx.to, buffer)) {
-            lh.updateLastConfirmed(rctx.getLastAddConfirmed(), 0L);
+            if (!isRecoveryRead) {
+                // do not advance LastAddConfirmed for recovery reads
+                lh.updateLastConfirmed(rctx.getLastAddConfirmed(), 0L);
+            }
             submitCallback(BKException.Code.OK);
         }
 

-- 
To stop receiving notification emails like this one, please contact
['"commits@bookkeeper.apache.org" <commits@bookkeeper.apache.org>'].

Mime
View raw message