zookeeper-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From iv...@apache.org
Subject svn commit: r1429756 - 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 Mon, 07 Jan 2013 11:25:07 GMT
Author: ivank
Date: Mon Jan  7 11:25:06 2013
New Revision: 1429756

URL: http://svn.apache.org/viewvc?rev=1429756&view=rev
Log:
BOOKKEEPER-355: Ledger recovery will mark ledger as closed with -1, in case of slow bookie
is added to ensemble during recovery add (ivank)

Modified:
    zookeeper/bookkeeper/trunk/CHANGES.txt
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandle.java
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadOnlyLedgerHandle.java
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerRecoveryTest.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=1429756&r1=1429755&r2=1429756&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/CHANGES.txt (original)
+++ zookeeper/bookkeeper/trunk/CHANGES.txt Mon Jan  7 11:25:06 2013
@@ -153,12 +153,15 @@ 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)
 
 	BOOKKEEPER-514: TestDeadLock hanging sometimes (ivank, sijie via fpj)
 
         BOOKKEEPER-524: Bookie journal filesystem gets full after SyncThread is terminated
with exception (Matteo, fpj via sijie)
 
+        BOOKKEEPER-355: Ledger recovery will mark ledger as closed with -1, in case of slow
bookie is added to ensemble during recovery add (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/LedgerHandle.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandle.java?rev=1429756&r1=1429755&r2=1429756&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandle.java
(original)
+++ zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerHandle.java
Mon Jan  7 11:25:06 2013
@@ -645,16 +645,35 @@ public class LedgerHandle {
 
     }
 
-    void handleBookieFailure(final InetSocketAddress addr, final int bookieIndex) {
+    ArrayList<InetSocketAddress> replaceBookieInMetadata(final InetSocketAddress addr,
final int bookieIndex)
+            throws BKException.BKNotEnoughBookiesException {
         InetSocketAddress newBookie;
-
-        LOG.debug("Handling failure of bookie: {} index: {}", addr, bookieIndex);
+        LOG.info("Handling failure of bookie: {} index: {}", addr, bookieIndex);
         final ArrayList<InetSocketAddress> newEnsemble = new ArrayList<InetSocketAddress>();
-        blockAddCompletions.incrementAndGet();
         final long newEnsembleStartEntry = lastAddConfirmed + 1;
 
         // avoid parallel ensemble changes to same ensemble.
         synchronized (metadata) {
+            newBookie = bk.bookieWatcher.getAdditionalBookie(metadata.currentEnsemble);
+
+            newEnsemble.addAll(metadata.currentEnsemble);
+            newEnsemble.set(bookieIndex, newBookie);
+
+            if (LOG.isDebugEnabled()) {
+                LOG.debug("Changing ensemble from: " + metadata.currentEnsemble
+                        + " to: " + newEnsemble + " for ledger: " + ledgerId
+                        + " starting at entry: " + (lastAddConfirmed + 1));
+            }
+
+            metadata.addEnsemble(newEnsembleStartEntry, newEnsemble);
+        }
+        return newEnsemble;
+    }
+
+    void handleBookieFailure(final InetSocketAddress addr, final int bookieIndex) {
+        blockAddCompletions.incrementAndGet();
+
+        synchronized (metadata) {
             if (!metadata.currentEnsemble.get(bookieIndex).equals(addr)) {
                 // ensemble has already changed, failure of this addr is immaterial
                 LOG.warn("Write did not succeed to {}, bookieIndex {}, but we have already
fixed it.",
@@ -664,30 +683,18 @@ public class LedgerHandle {
             }
 
             try {
-                newBookie = bk.bookieWatcher
-                        .getAdditionalBookie(metadata.currentEnsemble);
-            } catch (BKNotEnoughBookiesException e) {
+                ArrayList<InetSocketAddress> newEnsemble = replaceBookieInMetadata(addr,
bookieIndex);
+
+                EnsembleInfo ensembleInfo = new EnsembleInfo(newEnsemble, bookieIndex,
+                                                             addr);
+                writeLedgerConfig(new ChangeEnsembleCb(ensembleInfo));
+            } catch (BKException.BKNotEnoughBookiesException e) {
                 LOG.error("Could not get additional bookie to "
-                        + "remake ensemble, closing ledger: " + ledgerId);
+                          + "remake ensemble, closing ledger: " + ledgerId);
                 handleUnrecoverableErrorDuringAdd(e.getCode());
                 return;
             }
-
-            newEnsemble.addAll(metadata.currentEnsemble);
-            newEnsemble.set(bookieIndex, newBookie);
-
-            if (LOG.isDebugEnabled()) {
-                LOG.debug("Changing ensemble from: " + metadata.currentEnsemble
-                        + " to: " + newEnsemble + " for ledger: " + ledgerId
-                        + " starting at entry: " + (lastAddConfirmed + 1));
-            }
-
-            metadata.addEnsemble(newEnsembleStartEntry, newEnsemble);
         }
-
-        EnsembleInfo ensembleInfo = new EnsembleInfo(newEnsemble, bookieIndex,
-                addr);
-        writeLedgerConfig(new ChangeEnsembleCb(ensembleInfo));
     }
 
     // Contains newly reformed ensemble, bookieIndex, failedBookieAddress
@@ -819,7 +826,7 @@ public class LedgerHandle {
 
     };
 
-    private void unsetSuccessAndSendWriteRequest(final int bookieIndex) {
+    void unsetSuccessAndSendWriteRequest(final int bookieIndex) {
         for (PendingAddOp pendingAddOp : pendingAddOps) {
             pendingAddOp.unsetSuccessAndSendWriteRequest(bookieIndex);
         }

Modified: zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java?rev=1429756&r1=1429755&r2=1429756&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java
(original)
+++ zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/LedgerMetadata.java
Mon Jan  7 11:25:06 2013
@@ -437,21 +437,24 @@ public class LedgerMetadata {
             Version.Occurred.AFTER == version.compare(newMeta.version)) {
             return false;
         }
-        // ensemble size should be same
-        if (ensembles.size() != newMeta.ensembles.size()) {
-            return false;
-        }
-        // ensemble distribution should be same
-        // we don't check the detail ensemble, since new bookie will be set
-        // using recovery tool.
-        Iterator<Long> keyIter = ensembles.keySet().iterator();
-        Iterator<Long> newMetaKeyIter = newMeta.ensembles.keySet().iterator();
-        for (int i=0; i<ensembles.size(); i++) {
-            Long curKey = keyIter.next();
-            Long newMetaKey = newMetaKeyIter.next();
-            if (!curKey.equals(newMetaKey)) {
+        // if ledger is closed, we can just take the new ensembles
+        if (newMeta.state != LedgerMetadataFormat.State.CLOSED) {
+            // ensemble size should be same
+            if (ensembles.size() != newMeta.ensembles.size()) {
                 return false;
             }
+            // ensemble distribution should be same
+            // we don't check the detail ensemble, since new bookie will be set
+            // using recovery tool.
+            Iterator<Long> keyIter = ensembles.keySet().iterator();
+            Iterator<Long> newMetaKeyIter = newMeta.ensembles.keySet().iterator();
+            for (int i=0; i<ensembles.size(); i++) {
+                Long curKey = keyIter.next();
+                Long newMetaKey = newMetaKeyIter.next();
+                if (!curKey.equals(newMetaKey)) {
+                    return false;
+                }
+            }
         }
         /*
          *  if the conflict has been resolved, then update

Modified: zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadOnlyLedgerHandle.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadOnlyLedgerHandle.java?rev=1429756&r1=1429755&r2=1429756&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadOnlyLedgerHandle.java
(original)
+++ zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ReadOnlyLedgerHandle.java
Mon Jan  7 11:25:06 2013
@@ -25,7 +25,7 @@ import org.apache.bookkeeper.client.Asyn
 import org.apache.bookkeeper.client.AsyncCallback.CloseCallback;
 import org.apache.bookkeeper.client.BookKeeper.DigestType;
 import java.security.GeneralSecurityException;
-
+import java.net.InetSocketAddress;
 
 /**
  * Read only ledger handle. This ledger handle allows you to 
@@ -76,4 +76,31 @@ class ReadOnlyLedgerHandle extends Ledge
         cb.addComplete(BKException.Code.IllegalOpException, this,
                        LedgerHandle.INVALID_ENTRY_ID, ctx);
     }
+
+    @Override
+    void handleBookieFailure(final InetSocketAddress addr, final int bookieIndex) {
+        blockAddCompletions.incrementAndGet();
+        synchronized (metadata) {
+            try {
+                if (!metadata.currentEnsemble.get(bookieIndex).equals(addr)) {
+                    // ensemble has already changed, failure of this addr is immaterial
+                    LOG.warn("Write did not succeed to {}, bookieIndex {}, but we have already
fixed it.",
+                             addr, bookieIndex);
+                    blockAddCompletions.decrementAndGet();
+                    return;
+                }
+
+                replaceBookieInMetadata(addr, bookieIndex);
+
+                blockAddCompletions.decrementAndGet();
+                // the failed bookie has been replaced
+                unsetSuccessAndSendWriteRequest(bookieIndex);
+            } catch (BKException.BKNotEnoughBookiesException e) {
+                LOG.error("Could not get additional bookie to "
+                          + "remake ensemble, closing ledger: " + ledgerId);
+                handleUnrecoverableErrorDuringAdd(e.getCode());
+                return;
+            }
+        }
+    }
 }

Modified: zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerRecoveryTest.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerRecoveryTest.java?rev=1429756&r1=1429755&r2=1429756&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerRecoveryTest.java
(original)
+++ zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerRecoveryTest.java
Mon Jan  7 11:25:06 2013
@@ -26,6 +26,8 @@ import java.net.InetSocketAddress;
 import java.nio.ByteBuffer;
 import java.util.List;
 import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
 
 import org.junit.*;
 import org.apache.bookkeeper.bookie.Bookie;
@@ -217,4 +219,141 @@ public class LedgerRecoveryTest extends 
         assertEquals(numEntries - 1, afterlh.getLastAddConfirmed());
     }
 
+    /**
+     * {@link https://issues.apache.org/jira/browse/BOOKKEEPER-355}
+     * A recovery during a rolling restart shouldn't affect the ability
+     * to recovery the ledger later.
+     * We have a ledger on ensemble B1,B2,B3.
+     * The sequence of events is
+     * 1. B1 brought down for maintenance
+     * 2. Ledger recovery started
+     * 3. B2 answers read last confirmed.
+     * 4. B1 replaced in ensemble by B4
+     * 5. Write to B4 fails for some reason
+     * 6. B1 comes back up.
+     * 7. B2 goes down for maintenance.
+     * 8. Ledger recovery starts (ledger is now unavailable)
+     */
+    @Test(timeout=60000)
+    public void testLedgerRecoveryWithRollingRestart() throws Exception {
+        LedgerHandle lhbefore = bkc.createLedger(numBookies, 2, digestType, "".getBytes());
+        for (int i = 0; i < (numBookies*3)+1; i++) {
+            lhbefore.addEntry("data".getBytes());
+        }
+
+        // Add a dead bookie to the cluster
+        ServerConfiguration conf = newServerConfiguration();
+        Bookie deadBookie1 = new Bookie(conf) {
+            @Override
+            public void recoveryAddEntry(ByteBuffer entry, WriteCallback cb, Object ctx,
byte[] masterKey)
+                    throws IOException, BookieException {
+                // drop request to simulate a slow and failed bookie
+                throw new IOException("Couldn't write for some reason");
+            }
+        };
+        bsConfs.add(conf);
+        bs.add(startBookie(conf, deadBookie1));
+
+        // kill first bookie server
+        InetSocketAddress bookie1 = lhbefore.getLedgerMetadata().currentEnsemble.get(0);
+        ServerConfiguration conf1 = killBookie(bookie1);
+
+        // Try to recover and fence the ledger after killing one bookie in the
+        // ensemble in the ensemble, and another bookie is available in zk, but not writtable
+        try {
+            bkc.openLedger(lhbefore.getId(), digestType, "".getBytes());
+            fail("Shouldn't be able to open ledger, there should be entries missing");
+        } catch (BKException.BKLedgerRecoveryException e) {
+            // expected
+        }
+
+        // restart the first server, kill the second
+        bsConfs.add(conf1);
+        bs.add(startBookie(conf1));
+        InetSocketAddress bookie2 = lhbefore.getLedgerMetadata().currentEnsemble.get(1);
+        ServerConfiguration conf2 = killBookie(bookie2);
+
+        // using async, because this could trigger an assertion
+        final AtomicInteger returnCode = new AtomicInteger(0);
+        final CountDownLatch openLatch = new CountDownLatch(1);
+        bkc.asyncOpenLedger(lhbefore.getId(), digestType, "".getBytes(),
+                            new AsyncCallback.OpenCallback() {
+                                public void openComplete(int rc, LedgerHandle lh, Object
ctx) {
+                                    returnCode.set(rc);
+                                    openLatch.countDown();
+                                    if (rc != BKException.Code.OK) {
+                                        try {
+                                            lh.close();
+                                        } catch (Exception e) {
+                                            LOG.error("Exception closing ledger handle",
e);
+                                        }
+                                    }
+                                }
+                            }, null);
+        assertTrue("Open call should have completed", openLatch.await(5, TimeUnit.SECONDS));
+        assertFalse("Open should not have succeeded", returnCode.get() == BKException.Code.OK);
+
+        bsConfs.add(conf2);
+        bs.add(startBookie(conf2));
+
+        LedgerHandle lhafter = bkc.openLedger(lhbefore.getId(), digestType,
+                "".getBytes());
+        assertEquals("Fenced ledger should have correct lastAddConfirmed",
+                     lhbefore.getLastAddConfirmed(), lhafter.getLastAddConfirmed());
+    }
+
+    /**
+     * {@link https://issues.apache.org/jira/browse/BOOKKEEPER-355}
+     * Verify that if a recovery happens with 1 replica missing, and it's replaced
+     * with a faulty bookie, it doesn't break future recovery from happening.
+     * 1. Ledger is created with quorum size as 2, and entries are written
+     * 2. Now first bookie is in the ensemble is brought down.
+     * 3. Another client fence and trying to recover the same ledger
+     * 4. During this time ensemble change will happen
+     *    and new bookie will be added. But this bookie is not able to write.
+     * 5. This recovery will fail.
+     * 7. A new non-faulty bookie comes up
+     * 8. Another client trying to recover the same ledger.
+     */
+    @Test(timeout=60000)
+    public void testBookieFailureDuringRecovery() throws Exception {
+        LedgerHandle lhbefore = bkc.createLedger(numBookies, 2, digestType, "".getBytes());
+        for (int i = 0; i < (numBookies*3)+1; i++) {
+            lhbefore.addEntry("data".getBytes());
+        }
+
+        // Add a dead bookie to the cluster
+        ServerConfiguration conf = newServerConfiguration();
+        Bookie deadBookie1 = new Bookie(conf) {
+            @Override
+            public void recoveryAddEntry(ByteBuffer entry, WriteCallback cb, Object ctx,
byte[] masterKey)
+                    throws IOException, BookieException {
+                // drop request to simulate a slow and failed bookie
+                throw new IOException("Couldn't write for some reason");
+            }
+        };
+        bsConfs.add(conf);
+        bs.add(startBookie(conf, deadBookie1));
+
+        // kill first bookie server
+        InetSocketAddress bookie1 = lhbefore.getLedgerMetadata().currentEnsemble.get(0);
+        ServerConfiguration conf1 = killBookie(bookie1);
+
+        // Try to recover and fence the ledger after killing one bookie in the
+        // ensemble in the ensemble, and another bookie is available in zk but not writtable
+        try {
+            bkc.openLedger(lhbefore.getId(), digestType, "".getBytes());
+            fail("Shouldn't be able to open ledger, there should be entries missing");
+        } catch (BKException.BKLedgerRecoveryException e) {
+            // expected
+        }
+
+        // start a new good server
+        startNewBookie();
+
+        LedgerHandle lhafter = bkc.openLedger(lhbefore.getId(), digestType,
+                "".getBytes());
+        assertEquals("Fenced ledger should have correct lastAddConfirmed",
+                     lhbefore.getLastAddConfirmed(), lhafter.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=1429756&r1=1429755&r2=1429756&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
Mon Jan  7 11:25:06 2013
@@ -175,6 +175,17 @@ public abstract class BookKeeperClusterT
         }
     }
 
+    protected ServerConfiguration newServerConfiguration() throws IOException {
+        File f = File.createTempFile("bookie", "test");
+        tmpDirs.add(f);
+        f.delete();
+        f.mkdir();
+
+        int port = PortManager.nextFreePort();
+        return newServerConfiguration(port, zkUtil.getZooKeeperConnectString(),
+                                      f, new File[] { f });
+    }
+
     protected ServerConfiguration newServerConfiguration(int port, String zkServers, File
journalDir, File[] ledgerDirs) {
         ServerConfiguration conf = new ServerConfiguration(baseConf);
         conf.setBookiePort(port);
@@ -378,18 +389,11 @@ public abstract class BookKeeperClusterT
      */
     public int startNewBookie()
             throws IOException, InterruptedException, KeeperException, BookieException {
-        File f = File.createTempFile("bookie", "test");
-        tmpDirs.add(f);
-        f.delete();
-        f.mkdir();
-
-        int port = PortManager.nextFreePort();
-        ServerConfiguration conf = newServerConfiguration(port, zkUtil.getZooKeeperConnectString(),
-                                                          f, new File[] { f });
+        ServerConfiguration conf = newServerConfiguration();
         bsConfs.add(conf);
         bs.add(startBookie(conf));
 
-        return port;
+        return conf.getBookiePort();
     }
 
     /**



Mime
View raw message