zookeeper-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From iv...@apache.org
Subject svn commit: r1421235 - in /zookeeper/bookkeeper/trunk: ./ bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ bookkeeper-server/src/test/java/org/apache/bookkeeper/client/ bookkeeper-server/src/test/java/org/apache/bookkeeper/test/
Date Thu, 13 Dec 2012 11:33:13 GMT
Author: ivank
Date: Thu Dec 13 11:33:11 2012
New Revision: 1421235

URL: http://svn.apache.org/viewvc?rev=1421235&view=rev
Log:
BOOKKEEPER-365: Ledger will never recover if one of the quorum bookie is down forever and
others dont have entry (sijie via ivank)

Added:
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerRecoveryTest.java
      - copied, changed from r1421083, zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/LedgerRecoveryTest.java
Removed:
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/LedgerRecoveryTest.java
Modified:
    zookeeper/bookkeeper/trunk/CHANGES.txt
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingReadOp.java
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookKeeperClusterTestCase.java

Modified: zookeeper/bookkeeper/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/CHANGES.txt?rev=1421235&r1=1421234&r2=1421235&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/CHANGES.txt (original)
+++ zookeeper/bookkeeper/trunk/CHANGES.txt Thu Dec 13 11:33:11 2012
@@ -136,6 +136,8 @@ Trunk (unreleased changes)
 
         BOOKKEEPER-493: moveLedgerIndexFile might have chance pickup same directory (sijie
via ivank)
 
+        BOOKKEEPER-365: Ledger will never recover if one of the quorum bookie is down forever
and others dont have entry (sijie via ivank)
+
       hedwig-protocol:
 
         BOOKKEEPER-394: CompositeException message is not useful (Stu Hood via sijie)

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=1421235&r1=1421234&r2=1421235&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 13 11:33:11 2012
@@ -52,12 +52,14 @@ class PendingReadOp implements Enumerati
     long numPendingEntries;
     long startEntryId;
     long endEntryId;
+    final int maxMissedReadsAllowed;
 
     private class LedgerEntryRequest extends LedgerEntry {
         int nextReplicaIndexToReadFrom = 0;
         AtomicBoolean complete = new AtomicBoolean(false);
 
         int firstError = BKException.Code.OK;
+        int numMissedEntryReads = 0;
 
         final ArrayList<InetSocketAddress> ensemble;
 
@@ -71,6 +73,14 @@ class PendingReadOp implements Enumerati
             if (nextReplicaIndexToReadFrom >= lh.metadata.getWriteQuorumSize()) {
                 // 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
+                // to check whether we received more missed reads than maxMissedReadsAllowed
+                if (BKException.Code.BookieHandleNotAvailableException == firstError &&
+                    numMissedEntryReads > maxMissedReadsAllowed) {
+                    firstError = BKException.Code.NoSuchEntryException;
+                }
+
                 submitCallback(firstError);
                 return;
             }
@@ -88,8 +98,17 @@ class PendingReadOp implements Enumerati
         }
 
         void logErrorAndReattemptRead(String errMsg, int rc) {
-            if (firstError == BKException.Code.OK) {
+            if (BKException.Code.OK == firstError ||
+                BKException.Code.NoSuchEntryException == firstError) {
                 firstError = rc;
+            } else if (BKException.Code.BookieHandleNotAvailableException == firstError &&
+                       BKException.Code.NoSuchEntryException != rc) {
+                // if other exception rather than NoSuchEntryException is returned
+                // we need to update firstError to indicate that it might be a valid read
but just failed.
+                firstError = rc;
+            }
+            if (BKException.Code.NoSuchEntryException == rc) {
+                ++numMissedEntryReads;
             }
 
             int bookieIndex = lh.distributionSchedule.getWriteSet(entryId).get(nextReplicaIndexToReadFrom
- 1);
@@ -141,6 +160,7 @@ class PendingReadOp implements Enumerati
         this.startEntryId = startEntryId;
         this.endEntryId = endEntryId;
         numPendingEntries = endEntryId - startEntryId + 1;
+        maxMissedReadsAllowed = lh.metadata.getWriteQuorumSize() - lh.metadata.getAckQuorumSize();
     }
 
     public void initiate() throws InterruptedException {
@@ -175,20 +195,6 @@ class PendingReadOp implements Enumerati
 
         lh.opCounterSem.release();
 
-        // if we just read only one entry, and this entry is not existed (in recoveryRead
case)
-        // we don't need to do ReattemptRead, otherwise we could not handle following case:
-        //
-        // an empty ledger with quorum (bk1, bk2), bk2 is failed forever.
-        // bk1 return NoLedgerException, client do ReattemptRead to bk2 but bk2 isn't connected
-        // so the read 0 entry would failed. this ledger could never be closed.
-        if (startEntryId == endEntryId) {
-            if (BKException.Code.NoSuchLedgerExistsException == rc ||
-                BKException.Code.NoSuchEntryException == rc) {
-                submitCallback(rc);
-                return;
-            }
-        }
-
         if (rc != BKException.Code.OK) {
             entry.logErrorAndReattemptRead("Error: " + BKException.getMessage(rc), rc);
             return;

Copied: zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerRecoveryTest.java
(from r1421083, zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/LedgerRecoveryTest.java)
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerRecoveryTest.java?p2=zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerRecoveryTest.java&p1=zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/LedgerRecoveryTest.java&r1=1421083&r2=1421235&rev=1421235&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/LedgerRecoveryTest.java
(original)
+++ zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerRecoveryTest.java
Thu Dec 13 11:33:11 2012
@@ -1,4 +1,4 @@
-package org.apache.bookkeeper.test;
+package org.apache.bookkeeper.client;
 
 /*
  *
@@ -21,10 +21,21 @@ package org.apache.bookkeeper.test;
  *
  */
 
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+
 import org.junit.*;
+import org.apache.bookkeeper.bookie.Bookie;
+import org.apache.bookkeeper.bookie.BookieException;
 import org.apache.bookkeeper.client.LedgerHandle;
 import org.apache.bookkeeper.client.BKException;
 import org.apache.bookkeeper.client.BookKeeper.DigestType;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.WriteCallback;
+import org.apache.bookkeeper.test.BaseTestCase;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -149,4 +160,61 @@ public class LedgerRecoveryTest extends 
         assertEquals(numEntries - 1, afterlh.getLastAddConfirmed());
     }
 
+    @Test
+    public void testLedgerRecoveryWithSlowBookie() throws Exception {
+        for (int i = 0; i < 3; i++) {
+            LOG.info("TestLedgerRecoveryWithAckQuorum @ slow bookie {}", i);
+            ledgerRecoveryWithSlowBookie(3, 3, 2, 1, i);
+        }
+    }
+
+    private void ledgerRecoveryWithSlowBookie(int ensembleSize, int writeQuorumSize,
+        int ackQuorumSize, int numEntries, int slowBookieIdx) throws Exception {
+
+        // Create a ledger
+        LedgerHandle beforelh = null;
+        beforelh = bkc.createLedger(ensembleSize, writeQuorumSize, ackQuorumSize,
+                                    digestType, "".getBytes());
+
+        // kill first bookie server to start a fake one to simulate a slow bookie
+        // and failed to add entry on crash
+        // until write succeed
+        InetSocketAddress host = beforelh.getLedgerMetadata().currentEnsemble.get(slowBookieIdx);
+        ServerConfiguration conf = killBookie(host);
+
+        Bookie fakeBookie = new Bookie(conf) {
+            @Override
+            public void addEntry(ByteBuffer entry, WriteCallback cb, Object ctx, byte[] masterKey)
+                    throws IOException, BookieException {
+                // drop request to simulate a slow and failed bookie
+            }
+        };
+        bsConfs.add(conf);
+        bs.add(startBookie(conf, fakeBookie));
+
+        // avoid not-enough-bookies case
+        startNewBookie();
+
+        // write would still succeed with 2 bookies ack
+        String tmp = "BookKeeper is cool!";
+        for (int i = 0; i < numEntries; i++) {
+            beforelh.addEntry(tmp.getBytes());
+        }
+
+        conf = killBookie(host);
+        bsConfs.add(conf);
+        // the bookie goes normally
+        bs.add(startBookie(conf));
+
+        /*
+         * Try to open ledger.
+         */
+        LedgerHandle afterlh = bkc.openLedger(beforelh.getId(), digestType, "".getBytes());
+
+        /*
+         * Check if has recovered properly.
+         */
+        assertEquals(numEntries - 1, afterlh.getLastAddConfirmed());
+    }
+
 }

Modified: zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookKeeperClusterTestCase.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookKeeperClusterTestCase.java?rev=1421235&r1=1421234&r2=1421235&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookKeeperClusterTestCase.java
(original)
+++ zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookKeeperClusterTestCase.java
Thu Dec 13 11:33:11 2012
@@ -29,13 +29,14 @@ import java.util.LinkedList;
 import java.util.List;
 import java.util.concurrent.CountDownLatch;
 
+import org.apache.bookkeeper.bookie.BookieException;
+import org.apache.bookkeeper.bookie.Bookie;
 import org.apache.bookkeeper.client.BookKeeperTestClient;
 import org.apache.bookkeeper.conf.AbstractConfiguration;
 import org.apache.bookkeeper.conf.ClientConfiguration;
 import org.apache.bookkeeper.conf.ServerConfiguration;
 import org.apache.bookkeeper.metastore.InMemoryMetaStore;
 import org.apache.bookkeeper.proto.BookieServer;
-import org.apache.bookkeeper.bookie.BookieException;
 import org.apache.commons.io.FileUtils;
 import org.apache.zookeeper.KeeperException;
 import org.apache.zookeeper.ZooKeeper;
@@ -389,8 +390,31 @@ public abstract class BookKeeperClusterT
         return server;
     }
 
+    /**
+     * Start a bookie with the given bookie instance.
+     */
+    protected BookieServer startBookie(ServerConfiguration conf, final Bookie b)
+            throws IOException, InterruptedException, KeeperException, BookieException {
+        BookieServer server = new BookieServer(conf) {
+            @Override
+            protected Bookie newBookie(ServerConfiguration conf) {
+                return b;
+            }
+        };
+        server.start();
+
+        int port = conf.getBookiePort();
+        while(bkc.getZkHandle().exists("/ledgers/available/" + InetAddress.getLocalHost().getHostAddress()
+ ":" + port, false) == null) {
+            Thread.sleep(500);
+        }
+
+        bkc.readBookiesBlocking();
+        LOG.info("New bookie on port " + port + " has been created.");
+
+        return server;
+    }
+
     public void setMetastoreImplClass(AbstractConfiguration conf) {
         conf.setMetastoreImplClass(InMemoryMetaStore.class.getName());
     }
-
 }



Mime
View raw message