You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@zookeeper.apache.org by fp...@apache.org on 2012/12/27 23:16:41 UTC

svn commit: r1426349 - in /zookeeper/bookkeeper/trunk: ./ bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/

Author: fpj
Date: Thu Dec 27 22:16:40 2012
New Revision: 1426349

URL: http://svn.apache.org/viewvc?rev=1426349&view=rev
Log:
BOOKKEEPER-520: BookieFailureTest hangs on precommit build (sijie via fpj, jira reopened)


Modified:
    zookeeper/bookkeeper/trunk/CHANGES.txt
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingAddOp.java
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingReadOp.java
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieClient.java

Modified: zookeeper/bookkeeper/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/CHANGES.txt?rev=1426349&r1=1426348&r2=1426349&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/CHANGES.txt (original)
+++ zookeeper/bookkeeper/trunk/CHANGES.txt Thu Dec 27 22:16:40 2012
@@ -153,6 +153,7 @@ Trunk (unreleased changes)
         BOOKKEEPER-520: BookieFailureTest hangs on precommit build (ivank via sijie)
 
         BOOKKEEPER-447: Bookie can fail to recover if index pages flushed before ledger flush acknowledged (ivank via sijie)
+	BOOKKEEPER-520: BookieFailureTest hangs on precommit build (sijie via fpj, jira reopened)
 
       hedwig-protocol:
 

Modified: zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingAddOp.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingAddOp.java?rev=1426349&r1=1426348&r2=1426349&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingAddOp.java (original)
+++ zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingAddOp.java Thu Dec 27 22:16:40 2012
@@ -1,5 +1,3 @@
-package org.apache.bookkeeper.client;
-
 /**
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
@@ -17,7 +15,10 @@ package org.apache.bookkeeper.client;
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+package org.apache.bookkeeper.client;
 
+import java.util.HashSet;
+import java.util.Set;
 import java.net.InetSocketAddress;
 import org.apache.bookkeeper.client.AsyncCallback.AddCallback;
 import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteCallback;
@@ -42,6 +43,7 @@ class PendingAddOp implements WriteCallb
     AddCallback cb;
     Object ctx;
     long entryId;
+    Set<Integer> writeSet;
 
     DistributionSchedule.AckSet ackSet;
     boolean completed = false;
@@ -69,6 +71,7 @@ class PendingAddOp implements WriteCallb
 
     void setEntryId(long entryId) {
         this.entryId = entryId;
+        writeSet = new HashSet<Integer>(lh.distributionSchedule.getWriteSet(entryId));
     }
 
     void sendWriteRequest(int bookieIndex) {
@@ -85,6 +88,24 @@ class PendingAddOp implements WriteCallb
             // now
             return;
         }
+        // Suppose that unset doesn't happen on the write set of an entry. In this
+        // case we don't need to resend the write request upon an ensemble change.
+        // We do need to invoke #sendAddSuccessCallbacks() for such entries because
+        // they may have already completed, but they are just waiting for the ensemble
+        // to change.
+        // E.g.
+        // ensemble (A, B, C, D), entry k is written to (A, B, D). An ensemble change
+        // happens to replace C with E. Entry k does not complete until C is
+        // replaced with E successfully. When the ensemble change completes, it tries
+        // to unset entry k. C however is not in k's write set, so no entry is written
+        // again, and no one triggers #sendAddSuccessCallbacks. Consequently, k never
+        // completes.
+        //
+        // We call sendAddSuccessCallback when unsetting t cover this case.
+        if (!writeSet.contains(bookieIndex)) {
+            lh.sendAddSuccessCallbacks();
+            return;
+        }
 
         if (LOG.isDebugEnabled()) {
             LOG.debug("Unsetting success for ledger: " + lh.ledgerId + " entry: " + entryId + " bookie index: "
@@ -102,7 +123,7 @@ class PendingAddOp implements WriteCallb
 
     void initiate(ChannelBuffer toSend) {
         this.toSend = toSend;
-        for (int bookieIndex : lh.distributionSchedule.getWriteSet(entryId)) {
+        for (int bookieIndex : writeSet) {
             sendWriteRequest(bookieIndex);
         }
     }
@@ -130,14 +151,18 @@ class PendingAddOp implements WriteCallb
             return;
         }
 
+        if (!writeSet.contains(bookieIndex)) {
+            LOG.warn("Received a response for (lid:{}, eid:{}) from {}@{}, but it doesn't belong to {}.",
+                     new Object[] { ledgerId, entryId, addr, bookieIndex, writeSet });
+            return;
+        }
+
         if (ackSet.addBookieAndCheck(bookieIndex) && !completed) {
             completed = true;
 
-            // do some quick checks to see if some adds may have finished. All
-            // this will be checked under locks again
-            if (lh.pendingAddOps.peek() == this) {
-                lh.sendAddSuccessCallbacks();
-            }
+            LOG.debug("Complete (lid:{}, eid:{}).", ledgerId, entryId);
+            // when completed an entry, try to send success add callbacks in order
+            lh.sendAddSuccessCallbacks();
         }
     }
 
@@ -146,4 +171,13 @@ class PendingAddOp implements WriteCallb
         lh.opCounterSem.release();
     }
 
+    @Override
+    public String toString() {
+        StringBuilder sb = new StringBuilder();
+        sb.append("PendingAddOp(lid:").append(lh.ledgerId)
+          .append(", eid:").append(entryId).append(", completed:")
+          .append(completed).append(")");
+        return sb.toString();
+    }
+
 }

Modified: zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingReadOp.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingReadOp.java?rev=1426349&r1=1426348&r2=1426349&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingReadOp.java (original)
+++ zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingReadOp.java Thu Dec 27 22:16:40 2012
@@ -150,7 +150,7 @@ class PendingReadOp implements Enumerati
                 // we are done, the read has failed from all replicas, just fail the
                 // read
 
-                // Do it a bit perssimistically, only when finished trying all replicas
+                // Do it a bit pessimistically, only when finished trying all replicas
                 // to check whether we received more missed reads than maxMissedReadsAllowed
                 if (BKException.Code.BookieHandleNotAvailableException == firstError &&
                     numMissedEntryReads > maxMissedReadsAllowed) {

Modified: zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieClient.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieClient.java?rev=1426349&r1=1426348&r2=1426349&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieClient.java (original)
+++ zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/proto/BookieClient.java Thu Dec 27 22:16:40 2012
@@ -107,9 +107,14 @@ public class BookieClient {
 
         client.connectIfNeededAndDoOp(new GenericCallback<Void>() {
             @Override
-            public void operationComplete(int rc, Void result) {
+            public void operationComplete(final int rc, Void result) {
                 if (rc != BKException.Code.OK) {
-                    cb.writeComplete(rc, ledgerId, entryId, addr, ctx);
+                    executor.submitOrdered(ledgerId, new SafeRunnable() {
+                        @Override
+                        public void safeRun() {
+                            cb.writeComplete(rc, ledgerId, entryId, addr, ctx);
+                        }
+                    });
                     return;
                 }
                 client.addEntry(ledgerId, masterKey, entryId, toSend, cb, ctx, options);
@@ -127,9 +132,14 @@ public class BookieClient {
 
         client.connectIfNeededAndDoOp(new GenericCallback<Void>() {
             @Override
-            public void operationComplete(int rc, Void result) {
+            public void operationComplete(final int rc, Void result) {
                 if (rc != BKException.Code.OK) {
-                    cb.readEntryComplete(rc, ledgerId, entryId, null, ctx);
+                    executor.submitOrdered(ledgerId, new SafeRunnable() {
+                        @Override
+                        public void safeRun() {
+                            cb.readEntryComplete(rc, ledgerId, entryId, null, ctx);
+                        }
+                    });
                     return;
                 }
                 client.readEntryAndFenceLedger(ledgerId, masterKey, entryId, cb, ctx);
@@ -143,10 +153,14 @@ public class BookieClient {
 
         client.connectIfNeededAndDoOp(new GenericCallback<Void>() {
             @Override
-            public void operationComplete(int rc, Void result) {
-
+            public void operationComplete(final int rc, Void result) {
                 if (rc != BKException.Code.OK) {
-                    cb.readEntryComplete(rc, ledgerId, entryId, null, ctx);
+                    executor.submitOrdered(ledgerId, new SafeRunnable() {
+                        @Override
+                        public void safeRun() {
+                            cb.readEntryComplete(rc, ledgerId, entryId, null, ctx);
+                        }
+                    });
                     return;
                 }
                 client.readEntry(ledgerId, entryId, cb, ctx);