You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@zookeeper.apache.org by si...@apache.org on 2012/03/19 15:02:15 UTC

svn commit: r1302458 - in /zookeeper/bookkeeper/trunk: CHANGES.txt bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingReadOp.java bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookieThrottleTest.java

Author: sijie
Date: Mon Mar 19 14:02:15 2012
New Revision: 1302458

URL: http://svn.apache.org/viewvc?rev=1302458&view=rev
Log:
BOOKKEEPER-186: Bookkeeper throttling - permits is not released when read has failed from all replicas (Rakesh R via sijie)

Added:
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookieThrottleTest.java   (with props)
Modified:
    zookeeper/bookkeeper/trunk/CHANGES.txt
    zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingReadOp.java

Modified: zookeeper/bookkeeper/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/CHANGES.txt?rev=1302458&r1=1302457&r2=1302458&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/CHANGES.txt (original)
+++ zookeeper/bookkeeper/trunk/CHANGES.txt Mon Mar 19 14:02:15 2012
@@ -58,6 +58,8 @@ Trunk (unreleased changes)
 
         BOOKKEEPER-182: Entry log file is overwritten when fail to read lastLogId. (sijie via ivank)
 
+        BOOKKEEPER-186: Bookkeeper throttling - permits is not released when read has failed from all replicas (Rakesh R via sijie)
+
       hedwig-server/
       
         BOOKKEEPER-140: Hub server doesn't subscribe remote region correctly when a region is down. (Sijie Gou via ivank)

Modified: zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingReadOp.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingReadOp.java?rev=1302458&r1=1302457&r2=1302458&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingReadOp.java (original)
+++ zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingReadOp.java Mon Mar 19 14:02:15 2012
@@ -97,6 +97,7 @@ class PendingReadOp implements Enumerati
         if (entry.nextReplicaIndexToReadFrom >= lh.metadata.quorumSize) {
             // we are done, the read has failed from all replicas, just fail the
             // read
+            lh.opCounterSem.release();
             submitCallback(lastErrorCode);
             return;
         }

Added: zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookieThrottleTest.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookieThrottleTest.java?rev=1302458&view=auto
==============================================================================
--- zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookieThrottleTest.java (added)
+++ zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookieThrottleTest.java Mon Mar 19 14:02:15 2012
@@ -0,0 +1,127 @@
+package org.apache.bookkeeper.client;
+
+/*
+*
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you under the Apache License, Version 2.0 (the
+* "License"); you may not use this file except in compliance
+* with the License.  You may obtain a copy of the License at
+*
+*   http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing,
+* software distributed under the License is distributed on an
+* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+* KIND, either express or implied.  See the License for the
+* specific language governing permissions and limitations
+* under the License.
+*
+*/
+
+import java.util.Enumeration;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.bookkeeper.client.AsyncCallback.ReadCallback;
+import org.apache.bookkeeper.client.BookKeeper.DigestType;
+import org.apache.bookkeeper.test.BookKeeperClusterTestCase;
+import org.apache.bookkeeper.test.BookieFailureTest;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class BookieThrottleTest extends BookKeeperClusterTestCase{
+
+    private static Logger LOG = LoggerFactory.getLogger(BookieFailureTest.class);
+    private DigestType digestType;
+    private LedgerHandle lh;
+    private static CountDownLatch countDownLatch;
+    private int throttle = 5;
+    private int TIME_OUT = 30;
+
+    // Constructor
+    public BookieThrottleTest() {
+        super(4);
+        this.digestType = DigestType.CRC32;
+    }
+
+    @Test
+    public void testVerifyPermitRelaseInReadFailure() throws Exception {
+        baseClientConf.setThrottleValue(numBookies);
+        int numEntries = throttle * 2;
+        System.setProperty("throttle", String.valueOf(throttle));
+
+        lh = bkc.createLedger(numBookies, 1, digestType, "".getBytes());
+        // Add ledger entries.
+        for (int i = 0; i < numEntries; i++) {
+            lh.addEntry(("LedgerId: " + lh.getId() + ", EntryId: " + (i))
+                    .getBytes());
+        }
+        LOG.info("Finished writing all ledger entries so shutdown all the bookies " +
+                "to verify the read permits.");
+
+        for (int i = 0; i < numBookies; i++) {
+            bs.get(i).shutdown();
+        }
+
+        try {
+            lh.readEntries(0, throttle-1);
+        } catch (BKException e) {
+            LOG.info( "Exception when reading the entries, since all bookies are stopped", e);
+        }
+        LOG.debug("*** READ COMPLETE ***");
+        // grace period, just to avoid randomness
+        Thread.sleep(2000);
+        assertEquals("Permits is not released when read has failed from all replicas",
+                throttle, lh.getAvailablePermits().availablePermits());
+        lh.close();
+    }
+
+    @Test
+    public void testVerifyPermitRelaseInAsyncReadFailure() throws Exception {
+        baseClientConf.setThrottleValue(numBookies);
+        System.setProperty("throttle", String.valueOf(throttle));
+
+        lh = bkc.createLedger(numBookies, 1, digestType, ""
+                .getBytes());
+        // Add ledger entries.
+        int numEntries = throttle * 2;
+        for (int i = 0; i < numEntries; i++) {
+            lh.addEntry(("LedgerId: " + lh.getId() + ", EntryId: " + (i))
+                    .getBytes());
+        }
+        LOG.info("Finished writing all ledger entries so shutdown all the bookies "+
+                "to verify the read permits.");
+
+        for (int i = 0; i < numBookies; i++) {
+            bs.get(i).shutdown();
+        }
+        BookieReadCallback bookieReadCallback = new BookieReadCallback();
+        countDownLatch = new CountDownLatch(throttle);
+        try {
+            lh.asyncReadEntries(0, throttle-1, bookieReadCallback, null);
+        } catch (Exception e) {
+            LOG.info( "Exception when reading the entries, since all bookies are stopped", e);
+        }
+        countDownLatch.await(TIME_OUT, TimeUnit.SECONDS);
+        LOG.debug("*** READ COMPLETE ***");
+        // grace period, just to avoid randomness
+        Thread.sleep(2000);
+        assertEquals("Permits is not released when read has failed from all replicas",
+                throttle, lh.getAvailablePermits().availablePermits());
+        lh.close();
+    }
+
+    static class BookieReadCallback implements ReadCallback
+    {
+        @Override
+        public void readComplete(int rc, LedgerHandle lh,
+                Enumeration<LedgerEntry> seq, Object ctx) {
+            assertTrue("Expected Not OK, since all bookies are stopped", rc != BKException.Code.OK);
+            countDownLatch.countDown();
+        }        
+    }
+}

Propchange: zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/BookieThrottleTest.java
------------------------------------------------------------------------------
    svn:eol-style = native