You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@zookeeper.apache.org by iv...@apache.org on 2012/09/03 18:54:10 UTC

svn commit: r1380293 - in /zookeeper/bookkeeper/trunk: ./ bookkeeper-server/src/main/java/org/apache/bookkeeper/client/ bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/ bookkeeper-server/src/test/java/org/apache/bookkeeper/client/

Author: ivank
Date: Mon Sep  3 16:54:10 2012
New Revision: 1380293

URL: http://svn.apache.org/viewvc?rev=1380293&view=rev
Log:
BOOKKEEPER-385: replicateLedgerFragment should throw Exceptions in error conditions (umamahesh via ivank)

Modified:
    zookeeper/bookkeeper/trunk/CHANGES.txt
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeperAdmin.java
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/ReplicationWorker.java
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestLedgerFragmentReplication.java

Modified: zookeeper/bookkeeper/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/CHANGES.txt?rev=1380293&r1=1380292&r2=1380293&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/CHANGES.txt (original)
+++ zookeeper/bookkeeper/trunk/CHANGES.txt Mon Sep  3 16:54:10 2012
@@ -72,6 +72,8 @@ Trunk (unreleased changes)
 
         BOOKKEEPER-384: Clean up LedgerManagerFactory and LedgerManager usage in tests (rakeshr via ivank)
 
+        BOOKKEEPER-385: replicateLedgerFragment should throw Exceptions in error conditions (umamahesh via ivank)
+
       hedwig-client:
 
         BOOKKEEPER-274: Hedwig cpp client library should not link to cppunit which is just used for test. (sijie via ivank)

Modified: zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeperAdmin.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeperAdmin.java?rev=1380293&r1=1380292&r2=1380293&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeperAdmin.java (original)
+++ zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/BookKeeperAdmin.java Mon Sep  3 16:54:10 2012
@@ -659,11 +659,11 @@ public class BookKeeperAdmin {
      *            - LedgerFragment to replicate
      * @param targetBookieAddress
      *            - target Bookie, to where entries should be replicated.
-     * @return true - if replication success, false if fails to replicate.
      */
-    public boolean replicateLedgerFragment(LedgerHandle lh,
+    public void replicateLedgerFragment(LedgerHandle lh,
             final LedgerFragment ledgerFragment,
-            InetSocketAddress targetBookieAddress) throws InterruptedException {
+            InetSocketAddress targetBookieAddress) throws InterruptedException,
+            BKException {
         final SyncCounter syncCounter = new SyncCounter();
         ResultCallBack resultCallBack = new ResultCallBack(syncCounter);
         SingleFragmentCallback sfcb = new SingleFragmentCallback(
@@ -672,31 +672,22 @@ public class BookKeeperAdmin {
         syncCounter.inc();
         lfr.replicate(lh, ledgerFragment, sfcb, targetBookieAddress);
         syncCounter.block(0);
-        return resultCallBack.getResult();
+        if (syncCounter.getrc() != BKException.Code.OK) {
+            throw BKException.create(syncCounter.getrc());
+        }
     }
 
     /** This is the class for getting the replication result */
     static class ResultCallBack implements AsyncCallback.VoidCallback {
-
-        private boolean result;
         private SyncCounter sync;
 
-        public boolean getResult() {
-            return this.result;
-        }
-
         public ResultCallBack(SyncCounter sync) {
             this.sync = sync;
         }
 
         @Override
         public void processResult(int rc, String s, Object obj) {
-
-            if (rc != BKException.Code.OK) {
-                this.result = false;
-            } else {
-                this.result = true;
-            }
+            sync.setrc(rc);
             sync.dec();
         }
     }

Modified: zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/ReplicationWorker.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/ReplicationWorker.java?rev=1380293&r1=1380292&r2=1380293&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/ReplicationWorker.java (original)
+++ zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/replication/ReplicationWorker.java Mon Sep  3 16:54:10 2012
@@ -180,9 +180,10 @@ public class ReplicationWorker implement
      *         ensemble, it will skip replication for that particular
      *         fragments). Returns true if all fragments replicated
      *         successfully.
+     * @throws BKException 
      */
     private boolean doReplicateFragments(long ledgerIdToReplicate,
-            LedgerHandle lh) throws InterruptedException {
+            LedgerHandle lh) throws InterruptedException, BKException {
         CheckerCallback checkerCb = new CheckerCallback();
         ledgerChecker.checkLedger(lh, checkerCb);
         Set<LedgerFragment> fragments = checkerCb.waitAndGetResult();
@@ -198,11 +199,18 @@ public class ReplicationWorker implement
                 isTargetBookieExistsInFragmentEnsemble = true;
                 continue;
             }
-            boolean isFragmentReplicated = admin.replicateLedgerFragment(lh,
-                    ledgerFragment, targetBookie);
-            if (!isFragmentReplicated) {
+            try {
+                admin.replicateLedgerFragment(lh, ledgerFragment, targetBookie);
+            } catch (BKException.BKBookieHandleNotAvailableException e) {
+                LOG.warn("BKBookieHandleNotAvailableException "
+                        + "while replicating the fragment", e);
+                isAllFragmentsReplicated = false;
+            } catch (BKException.BKLedgerRecoveryException e) {
+                LOG.warn("BKLedgerRecoveryException "
+                        + "while replicating the fragment", e);
                 isAllFragmentsReplicated = false;
             }
+
         }
         if (isTargetBookieExistsInFragmentEnsemble) {
             LOG.info("Releasing the lock, as target Bookie found"

Modified: zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestLedgerFragmentReplication.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestLedgerFragmentReplication.java?rev=1380293&r1=1380292&r2=1380293&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestLedgerFragmentReplication.java (original)
+++ zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestLedgerFragmentReplication.java Mon Sep  3 16:54:10 2012
@@ -99,7 +99,7 @@ public class TestLedgerFragmentReplicati
         // 0-9 entries should be copy to new bookie
         
         for (LedgerFragment lf : result) {
-            assertTrue(admin.replicateLedgerFragment(lh, lf, newBkAddr));
+            admin.replicateLedgerFragment(lh, lf, newBkAddr);
         }
 
         // Killing all bookies except newly replicated bookie
@@ -157,8 +157,11 @@ public class TestLedgerFragmentReplicati
         InetSocketAddress additionalBK = new InetSocketAddress(InetAddress
                 .getLocalHost().getHostAddress(), startNewBookie);
         for (LedgerFragment lf : fragments) {
-            assertFalse("Replication should fail", admin
-                    .replicateLedgerFragment(lh, lf, additionalBK));
+            try {
+                admin.replicateLedgerFragment(lh, lf, additionalBK);
+            } catch (BKException.BKLedgerRecoveryException e) {
+                // expected
+            }
         }
     }