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/12/20 06:48:57 UTC
svn commit: r1424335 - in /zookeeper/bookkeeper/trunk: ./
bookkeeper-server/src/main/java/org/apache/bookkeeper/client/
bookkeeper-server/src/test/java/org/apache/bookkeeper/test/
Author: sijie
Date: Thu Dec 20 05:48:56 2012
New Revision: 1424335
URL: http://svn.apache.org/viewvc?rev=1424335&view=rev
Log:
BOOKKEEPER-520: BookieFailureTest hangs on precommit build (ivank 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/main/java/org/apache/bookkeeper/client/PendingAddOp.java
zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieFailureTest.java
Modified: zookeeper/bookkeeper/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/CHANGES.txt?rev=1424335&r1=1424334&r2=1424335&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/CHANGES.txt (original)
+++ zookeeper/bookkeeper/trunk/CHANGES.txt Thu Dec 20 05:48:56 2012
@@ -150,6 +150,8 @@ Trunk (unreleased changes)
BOOKKEEPER-500: Fencing doesn't work when restarting bookies. (sijie via ivank)
+ BOOKKEEPER-520: BookieFailureTest hangs on precommit build (ivank 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=1424335&r1=1424334&r2=1424335&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 Thu Dec 20 05:48:56 2012
@@ -672,6 +672,14 @@ public class LedgerHandle {
// avoid parallel ensemble changes to same ensemble.
synchronized (metadata) {
+ if (!metadata.currentEnsemble.get(bookieIndex).equals(addr)) {
+ // ensemble has already changed, failure of this addr is immaterial
+ LOG.warn("Write did not succeed to {}, bookieIndex {}, but we have already fixed it.",
+ addr, bookieIndex);
+ blockAddCompletions.decrementAndGet();
+ return;
+ }
+
try {
newBookie = bk.bookieWatcher
.getAdditionalBookie(metadata.currentEnsemble);
Modified: zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingAddOp.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingAddOp.java?rev=1424335&r1=1424334&r2=1424335&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingAddOp.java (original)
+++ zookeeper/bookkeeper/trunk/bookkeeper-server/src/main/java/org/apache/bookkeeper/client/PendingAddOp.java Thu Dec 20 05:48:56 2012
@@ -111,11 +111,6 @@ class PendingAddOp implements WriteCallb
public void writeComplete(int rc, long ledgerId, long entryId, InetSocketAddress addr, Object ctx) {
int bookieIndex = (Integer) ctx;
- if (!lh.metadata.currentEnsemble.get(bookieIndex).equals(addr)) {
- // ensemble has already changed, failure of this addr is immaterial
- LOG.warn("Write did not succeed: " + ledgerId + ", " + entryId + ". But we have already fixed it.");
- return;
- }
switch (rc) {
case BKException.Code.OK:
Modified: zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieFailureTest.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieFailureTest.java?rev=1424335&r1=1424334&r2=1424335&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieFailureTest.java (original)
+++ zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/test/BookieFailureTest.java Thu Dec 20 05:48:56 2012
@@ -60,7 +60,6 @@ public class BookieFailureTest extends M
byte[] ledgerPassword = "aaa".getBytes();
LedgerHandle lh, lh2;
long ledgerId;
- Enumeration<LedgerEntry> ls;
// test related variables
int numEntriesToWrite = 200;
@@ -70,17 +69,17 @@ public class BookieFailureTest extends M
ArrayList<Integer> entriesSize;
DigestType digestType;
- // Synchronization
- SyncObj sync;
- Set<Object> syncObjs;
-
class SyncObj {
int counter;
boolean value;
+ boolean failureOccurred;
+ Enumeration<LedgerEntry> ls;
public SyncObj() {
counter = 0;
value = false;
+ failureOccurred = false;
+ ls = null;
}
}
@@ -151,11 +150,10 @@ public class BookieFailureTest extends M
numScanned++;
}
assertEquals(numEntries, numScanned);
-
-
}
void auxTestReadWriteAsyncSingleClient(BookieServer bs) throws IOException {
+ SyncObj sync = new SyncObj();
try {
// Create a ledger
lh = bkc.createLedger(3, 2, digestType, ledgerPassword);
@@ -181,7 +179,8 @@ public class BookieFailureTest extends M
synchronized (sync) {
while (sync.counter < numEntriesToWrite) {
LOG.debug("Entries counter = " + sync.counter);
- sync.wait();
+ sync.wait(10000);
+ assertFalse("Failure occurred during write", sync.failureOccurred);
}
}
@@ -206,6 +205,7 @@ public class BookieFailureTest extends M
while (sync.value == false) {
sync.wait(10000);
assertTrue("Haven't received entries", sync.value);
+ assertFalse("Failure occurred during read", sync.failureOccurred);
}
}
@@ -214,10 +214,10 @@ public class BookieFailureTest extends M
// at this point, Enumeration<LedgerEntry> ls is filled with the returned
// values
int i = 0;
- while (ls.hasMoreElements()) {
+ while (sync.ls.hasMoreElements()) {
ByteBuffer origbb = ByteBuffer.wrap(entries.get(i));
Integer origEntry = origbb.getInt();
- byte[] entry = ls.nextElement().getEntry();
+ byte[] entry = sync.ls.nextElement().getEntry();
ByteBuffer result = ByteBuffer.wrap(entry);
Integer retrEntry = result.getInt();
@@ -246,9 +246,11 @@ public class BookieFailureTest extends M
@Override
public void addComplete(int rc, LedgerHandle lh, long entryId, Object ctx) {
- if (rc != 0)
- fail("Failed to write entry: " + entryId);
SyncObj x = (SyncObj) ctx;
+ if (rc != 0) {
+ LOG.error("Failure during add {} {}", entryId, rc);
+ x.failureOccurred = true;
+ }
synchronized (x) {
x.counter++;
x.notify();
@@ -257,14 +259,16 @@ public class BookieFailureTest extends M
@Override
public void readComplete(int rc, LedgerHandle lh, Enumeration<LedgerEntry> seq, Object ctx) {
- if (rc != 0)
- fail("Failed to write entry");
- ls = seq;
- synchronized (sync) {
- sync.value = true;
- sync.notify();
+ SyncObj x = (SyncObj) ctx;
+ if (rc != 0) {
+ LOG.error("Failure during add {}", rc);
+ x.failureOccurred = true;
+ }
+ synchronized (x) {
+ x.value = true;
+ x.ls = seq;
+ x.notify();
}
-
}
@Before
@@ -276,7 +280,6 @@ public class BookieFailureTest extends M
// Number Generator
entries = new ArrayList<byte[]>(); // initialize the entries list
entriesSize = new ArrayList<Integer>();
- sync = new SyncObj(); // initialize the synchronization data structure
zkc.close();
}