You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@bookkeeper.apache.org by GitBox <gi...@apache.org> on 2017/11/08 15:13:14 UTC

[GitHub] ivankelly commented on a change in pull request #706: Bookies should not queue read request indefinitely

ivankelly commented on a change in pull request #706: Bookies should not queue read request indefinitely
URL: https://github.com/apache/bookkeeper/pull/706#discussion_r149693424
 
 

 ##########
 File path: bookkeeper-server/src/test/java/org/apache/bookkeeper/client/TestMaxSizeWorkersQueue.java
 ##########
 @@ -0,0 +1,106 @@
+package org.apache.bookkeeper.client;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.util.Enumeration;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.bookkeeper.client.AsyncCallback.AddCallback;
+import org.apache.bookkeeper.client.AsyncCallback.ReadCallback;
+import org.apache.bookkeeper.client.BookKeeper.DigestType;
+import org.apache.bookkeeper.test.BookKeeperClusterTestCase;
+import org.junit.Test;
+
+
+public class TestMaxSizeWorkersQueue extends BookKeeperClusterTestCase {
+    DigestType digestType = DigestType.CRC32;
+
+    public TestMaxSizeWorkersQueue() {
+        super(1);
+
+        baseConf.setNumReadWorkerThreads(1);
+        baseConf.setNumAddWorkerThreads(1);
+
+        // Configure very small queue sizes
+        baseConf.setMaxPendingReadRequestPerThread(1);
+        baseConf.setMaxPendingAddRequestPerThread(1);
+    }
+
+    @Test(timeout = 60000)
+    public void testReadRejected() throws Exception {
+        LedgerHandle lh = bkc.createLedger(1, 1, digestType, new byte[0]);
+        byte[] content = new byte[100];
+
+        final int N = 1000;
+        // Write few entries
+        for (int i = 0; i < N; i++) {
+            lh.addEntry(content);
+        }
+
+        // Read asynchronously:
+        // - 1st read must always succeed
+        // - Subsequent reads may fail, depending on timing
+        // - At least few, we expect to fail with TooManyRequestException
+        final CountDownLatch counter = new CountDownLatch(2);
+
+        final AtomicInteger rcFirstReadOperation = new AtomicInteger();
+
+        lh.asyncReadEntries(0, 0, new ReadCallback() {
+            @Override
+            public void readComplete(int rc, LedgerHandle lh, Enumeration<LedgerEntry> seq, Object ctx) {
+                rcFirstReadOperation.set(rc);
+                counter.countDown();
+            }
+        }, lh);
+
+        final AtomicInteger rcSecondReadOperation = new AtomicInteger();
+
+        lh.asyncReadEntries(0, N - 1, new ReadCallback() {
+            @Override
+            public void readComplete(int rc, LedgerHandle lh, Enumeration<LedgerEntry> seq, Object ctx) {
+                rcSecondReadOperation.set(rc);
+                counter.countDown();
+            }
+        }, lh);
+
+        counter.await();
+
+        // assertEquals(BKException.Code.OK, rcFirstReadOperation.get());
+        assertEquals(BKException.Code.TooManyRequestsException, rcSecondReadOperation.get());
+    }
+
+    @Test(timeout = 60000)
+    public void testAddRejected() throws Exception {
+        LedgerHandle lh = bkc.createLedger(1, 1, digestType, new byte[0]);
+        byte[] content = new byte[100];
+
+        final int N = 1000;
+
+        // Write asynchronously, and expect at least few writes to have failed with NotEnoughBookies,
+        // because when we get the TooManyRequestException, the client will try to form a new ensemble and that
+        // operation will fail since we only have 1 bookie available
+        final CountDownLatch counter = new CountDownLatch(N);
+        final AtomicBoolean receivedTooManyRequestsException = new AtomicBoolean();
+
+        // Write few entries
+        for (int i = 0; i < N; i++) {
+            lh.asyncAddEntry(content, new AddCallback() {
+                @Override
+                public void addComplete(int rc, LedgerHandle lh, long entryId, Object ctx) {
+                    if (rc == BKException.Code.NotEnoughBookiesException) {
+                        receivedTooManyRequestsException.set(true);
 
 Review comment:
   Why does it get this error? Shouldn't it be TooManyRequestException?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services