This is an automated email from the ASF dual-hosted git repository.

sijie pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/bookkeeper.git


The following commit(s) were added to refs/heads/master by this push:
     new 9f6c7da  ISSUE #1162: Test for unsetSuccessAndSendWriteRequest with 
LedgerHandleAdv
9f6c7da is described below

commit 9f6c7da19eef89e248a4b2ffa82d9c2a08789270
Author: JV Jujjuri <[email protected]>
AuthorDate: Sun Feb 18 23:17:20 2018 -0800

    ISSUE #1162: Test for unsetSuccessAndSendWriteRequest with LedgerHandleAdv
    
    Descriptions of the changes in this PR:
    
    Add a test case to verify unsetSuccessAndSendWriteRequest with
    LedgerHandleAdv out of order writers which forces ensemble change.
    
    Signed-off-by: Venkateswararao Jujjuri (JV) <vjujjurisalesforce.com>
    
    Master Issue: #1162
    
    Author: JV Jujjuri <[email protected]>
    
    Reviewers: Enrico Olivelli <[email protected]>, Sijie Guo 
<[email protected]>
    
    This closes #1163 from jvrao/bk-issue-1162-test1, closes #1162
---
 .../bookkeeper/client/BookieWriteLedgerTest.java   | 91 ++++++++++++++++++++++
 1 file changed, 91 insertions(+)

diff --git 
a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookieWriteLedgerTest.java
 
b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookieWriteLedgerTest.java
index 262f0c8..cab3f8a 100644
--- 
a/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookieWriteLedgerTest.java
+++ 
b/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookieWriteLedgerTest.java
@@ -56,6 +56,7 @@ import org.apache.zookeeper.KeeperException;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
+import org.powermock.reflect.Whitebox;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -569,6 +570,7 @@ public class BookieWriteLedgerTest extends
         bkc.deleteLedger(ledgerId);
     }
 
+
     /**
      * In a loop create/write/delete the ledger with same ledgerId through
      * the functionality of Advanced Ledger which accepts ledgerId as input.
@@ -819,6 +821,95 @@ public class BookieWriteLedgerTest extends
     }
 
     /**
+     * LedgerHandleAdv out of order writers with ensemble changes.
+     * Verify that entry that was written to old ensemble will be
+     * written to new enseble too after ensemble change.
+     *
+     * @throws Exception
+     */
+    @Test
+    public void testLedgerHandleAdvOutOfOrderWriteAndFrocedEnsembleChange() 
throws Exception {
+        // Create a ledger
+        long ledgerId = 0xABCDEF;
+        SyncObj syncObj1 = new SyncObj();
+        ByteBuffer entry;
+        lh = bkc.createLedgerAdv(ledgerId, 3, 3, 3, digestType, 
ledgerPassword, null);
+        entry = ByteBuffer.allocate(4);
+        // Add entries 0-4
+        for (int i = 0; i < 5; i++) {
+            entry.rewind();
+            entry.putInt(rng.nextInt(maxInt));
+            lh.addEntry(i, entry.array());
+        }
+
+        // Add 10 as Async Entry, which goes to first ensemble
+        ByteBuffer entry1 = ByteBuffer.allocate(4);
+        entry1.putInt(rng.nextInt(maxInt));
+        lh.asyncAddEntry(10, entry1.array(), 0, entry1.capacity(), this, 
syncObj1);
+
+        // Make sure entry-10 goes to the bookies and gets response.
+        java.util.Queue<PendingAddOp> myPendingAddOps = 
Whitebox.getInternalState(lh, "pendingAddOps");
+        PendingAddOp addOp = null;
+        boolean pendingAddOpReceived = false;
+
+        while (!pendingAddOpReceived) {
+            addOp = myPendingAddOps.peek();
+            if (addOp.entryId == 10 && addOp.completed) {
+                pendingAddOpReceived = true;
+            } else {
+                Thread.sleep(1000);
+            }
+        }
+
+        CountDownLatch sleepLatch1 = new CountDownLatch(1);
+        ArrayList<BookieSocketAddress> ensemble;
+
+        ensemble = 
lh.getLedgerMetadata().getEnsembles().entrySet().iterator().next().getValue();
+
+        // Put all 3 bookies to sleep and start 3 new ones
+        sleepBookie(ensemble.get(0), sleepLatch1);
+        sleepBookie(ensemble.get(1), sleepLatch1);
+        sleepBookie(ensemble.get(2), sleepLatch1);
+        startNewBookie();
+        startNewBookie();
+        startNewBookie();
+
+        // Original bookies are in sleep, new bookies added.
+        // Now add entries 5-9 which forces ensemble changes
+        // So at this point entries 0-4, 10 went to first
+        // ensemble, 5-9 will go to new ensemble.
+        for (int i = 5; i < 10; i++) {
+            entry.rewind();
+            entry.putInt(rng.nextInt(maxInt));
+            lh.addEntry(i, entry.array());
+        }
+
+        // Wakeup all 3 bookies that went to sleep
+        sleepLatch1.countDown();
+
+        // Wait for all entries to be acknowledged for the first ledger
+        synchronized (syncObj1) {
+            while (syncObj1.counter < 1) {
+                syncObj1.wait();
+            }
+            assertEquals(BKException.Code.OK, syncObj1.rc);
+        }
+
+        // Close write handle
+        lh.close();
+
+        // Open read handle
+        lh = bkc.openLedger(ledgerId, digestType, ledgerPassword);
+
+        // Make sure to read all 10 entries.
+        for (int i = 0; i < 11; i++) {
+            lh.readEntries(i, i);
+        }
+        lh.close();
+        bkc.deleteLedger(ledgerId);
+    }
+
+    /**
      * Verify Advanced asynchronous writing with entryIds in pseudo random 
order with bookie failures between writes.
      */
     @Test

-- 
To stop receiving notification emails like this one, please contact
[email protected].

Reply via email to