Author: sijie
Date: Mon Dec  3 05:23:40 2012
New Revision: 1416352

URL: http://svn.apache.org/viewvc?rev=1416352&view=rev
Log:
BOOKKEEPER-351: asyncAddEntry should not throw an exception (Matteo Merli via 
sijie)

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/test/java/org/apache/bookkeeper/client/BookKeeperTest.java

Modified: zookeeper/bookkeeper/trunk/CHANGES.txt
URL: 
http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/CHANGES.txt?rev=1416352&r1=1416351&r2=1416352&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/CHANGES.txt (original)
+++ zookeeper/bookkeeper/trunk/CHANGES.txt Mon Dec  3 05:23:40 2012
@@ -112,6 +112,8 @@ Trunk (unreleased changes)
 
        BOOKKEEPER-485: TestFencing hung (ivank via fpj)
 
+        BOOKKEEPER-351: asyncAddEntry should not throw an exception (Matteo 
Merli via sijie)
+
       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=1416352&r1=1416351&r2=1416352&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 Dec  3 05:23:40 2012
@@ -526,7 +526,8 @@ public class LedgerHandle {
             });
         } catch (RuntimeException e) {
             opCounterSem.release();
-            throw e;
+            cb.addComplete(BKException.Code.InterruptedException,
+                    LedgerHandle.this, INVALID_ENTRY_ID, ctx);
         }
     }
 

Modified: 
zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperTest.java
URL: 
http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperTest.java?rev=1416352&r1=1416351&r2=1416352&view=diff
==============================================================================
--- 
zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperTest.java
 (original)
+++ 
zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookKeeperTest.java
 Mon Dec  3 05:23:40 2012
@@ -22,8 +22,10 @@ package org.apache.bookkeeper.client;
 */
 
 import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.bookkeeper.conf.ClientConfiguration;
+import org.apache.bookkeeper.client.AsyncCallback.AddCallback;
 import org.apache.bookkeeper.client.BookKeeper.DigestType;
 import org.apache.bookkeeper.test.BaseTestCase;
 import org.apache.zookeeper.ZooKeeper;
@@ -31,6 +33,7 @@ import org.apache.zookeeper.Watcher;
 import org.apache.zookeeper.WatchedEvent;
 import org.apache.zookeeper.KeeperException;
 
+import org.junit.Assert;
 import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -147,4 +150,31 @@ public class BookKeeperTest extends Base
             bkc.close();
         }
     }
+
+    /**
+     * Tests that when trying to use a closed BK client object we get
+     * a callback error and not an InterruptedException.
+     * @throws Exception
+     */
+    @Test
+    public void testAsyncReadWithError() throws Exception {
+        LedgerHandle lh = bkc.createLedger(3, 3, DigestType.CRC32, 
"testPasswd".getBytes());
+        bkc.close();
+
+        final AtomicInteger result = new AtomicInteger(0);
+        final CountDownLatch counter = new CountDownLatch(1);
+
+        // Try to write, we shoud get and error callback but not an exception
+        lh.asyncAddEntry("test".getBytes(), new AddCallback() {
+            public void addComplete(int rc, LedgerHandle lh, long entryId, 
Object ctx) {
+                result.set(rc);
+                counter.countDown();
+            }
+        }, null);
+
+        counter.await();
+
+        Assert.assertTrue(result.get() != 0);
+    }
+
 }


Reply via email to