You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@zookeeper.apache.org by iv...@apache.org on 2013/10/14 18:32:27 UTC
svn commit: r1531970 - in /zookeeper/bookkeeper/trunk: ./
bookkeeper-server/src/main/java/org/apache/bookkeeper/client/
bookkeeper-server/src/test/java/org/apache/bookkeeper/client/
Author: ivank
Date: Mon Oct 14 16:32:27 2013
New Revision: 1531970
URL: http://svn.apache.org/r1531970
Log:
BOOKKEEPER-673: Ledger length can be inaccurate in failure case (sijie via ivank)
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/client/LedgerCloseTest.java
Modified: zookeeper/bookkeeper/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/CHANGES.txt?rev=1531970&r1=1531969&r2=1531970&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/CHANGES.txt (original)
+++ zookeeper/bookkeeper/trunk/CHANGES.txt Mon Oct 14 16:32:27 2013
@@ -110,6 +110,8 @@ Trunk (unreleased changes)
BOOKKEEPER-676: Make add asynchrounous in ledger recovery (aniruddha via ivank)
+ BOOKKEEPER-673: Ledger length can be inaccurate in failure case (sijie via ivank)
+
hedwig-server:
BOOKKEEPER-601: readahead cache size isn't updated correctly (sijie via fpj)
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=1531970&r1=1531969&r2=1531970&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 Oct 14 16:32:27 2013
@@ -258,13 +258,14 @@ public class LedgerHandle {
prevLastEntryId = metadata.getLastEntryId();
prevLength = metadata.getLength();
+ // error out pending adds first
+ errorOutPendingAdds(rc);
+
// synchronized on LedgerHandle.this to ensure that
// lastAddPushed can not be updated after the metadata
// is closed.
metadata.setLength(length);
-
metadata.close(lastAddConfirmed);
- errorOutPendingAdds(rc);
lastAddPushed = lastAddConfirmed;
}
@@ -510,7 +511,7 @@ public class LedgerHandle {
public void safeRun() {
ChannelBuffer toSend = macManager.computeDigestAndPackageForSending(
entryId, lastAddConfirmed, currentLength, data, offset, length);
- op.initiate(toSend);
+ op.initiate(toSend, length);
}
});
} catch (RuntimeException e) {
@@ -631,6 +632,7 @@ public class LedgerHandle {
void errorOutPendingAdds(int rc) {
PendingAddOp pendingAddOp;
while ((pendingAddOp = pendingAddOps.poll()) != null) {
+ addToLength(-pendingAddOp.entryLength);
pendingAddOp.submitCallback(rc);
}
}
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=1531970&r1=1531969&r2=1531970&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 Mon Oct 14 16:32:27 2013
@@ -43,6 +43,7 @@ class PendingAddOp implements WriteCallb
AddCallback cb;
Object ctx;
long entryId;
+ int entryLength;
Set<Integer> writeSet;
DistributionSchedule.AckSet ackSet;
@@ -121,8 +122,9 @@ class PendingAddOp implements WriteCallb
sendWriteRequest(bookieIndex);
}
- void initiate(ChannelBuffer toSend) {
+ void initiate(ChannelBuffer toSend, int entryLength) {
this.toSend = toSend;
+ this.entryLength = entryLength;
for (int bookieIndex : writeSet) {
sendWriteRequest(bookieIndex);
}
Modified: zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerCloseTest.java
URL: http://svn.apache.org/viewvc/zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerCloseTest.java?rev=1531970&r1=1531969&r2=1531970&view=diff
==============================================================================
--- zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerCloseTest.java (original)
+++ zookeeper/bookkeeper/trunk/bookkeeper-server/src/test/java/org/apache/bookkeeper/client/LedgerCloseTest.java Mon Oct 14 16:32:27 2013
@@ -23,6 +23,7 @@ import java.nio.ByteBuffer;
import java.util.Set;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
import org.apache.bookkeeper.bookie.Bookie;
import org.apache.bookkeeper.bookie.BookieException;
@@ -57,6 +58,33 @@ public class LedgerCloseTest extends Boo
}
@Test(timeout = 60000)
+ public void testLedgerCloseWithConsistentLength() throws Exception {
+ ClientConfiguration conf = new ClientConfiguration();
+ conf.setZkServers(zkUtil.getZooKeeperConnectString()).setReadTimeout(1);
+
+ BookKeeper bkc = new BookKeeper(conf);
+ LedgerHandle lh = bkc.createLedger(6, 3, DigestType.CRC32, new byte[] {});
+ final CountDownLatch latch = new CountDownLatch(1);
+ stopBKCluster();
+ final AtomicInteger i = new AtomicInteger(0xdeadbeef);
+ AsyncCallback.AddCallback cb = new AsyncCallback.AddCallback() {
+ @Override
+ public void addComplete(int rc, LedgerHandle lh, long entryId, Object ctx) {
+ i.set(rc);
+ latch.countDown();
+ }
+ };
+ lh.asyncAddEntry("Test Entry".getBytes(), cb, null);
+ latch.await();
+ assertEquals(i.get(), BKException.Code.NotEnoughBookiesException);
+ assertEquals(0, lh.getLength());
+ assertEquals(LedgerHandle.INVALID_ENTRY_ID, lh.getLastAddConfirmed());
+ LedgerHandle newLh = bkc.openLedger(lh.getId(), DigestType.CRC32, new byte[] {});
+ assertEquals(0, newLh.getLength());
+ assertEquals(LedgerHandle.INVALID_ENTRY_ID, newLh.getLastAddConfirmed());
+ }
+
+ @Test(timeout = 60000)
public void testLedgerCloseDuringUnrecoverableErrors() throws Exception {
int numEntries = 3;
LedgerHandle lh = bkc.createLedger(3, 3, 3, digestType, "".getBytes());