Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

ISSUE #1390 Ensemble change on delayed write error #1395

Closed
wants to merge 3 commits into from
Closed
Show file tree
Hide file tree
Changes from 2 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,7 @@
import java.util.Arrays;
import java.util.EnumSet;
import java.util.Enumeration;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
Expand Down Expand Up @@ -117,6 +118,7 @@ public class LedgerHandle implements WriteHandle {
ScheduledFuture<?> timeoutFuture = null;

final long waitForWriteSetMs;
private Map<Integer, BookieSocketAddress> delayedWriteFailedBookies = new HashMap<Integer, BookieSocketAddress>();
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: final?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Sure, I can do that; but would wait to see if you have any other comments.


/**
* Invalid entry id. This value is returned from methods which
Expand Down Expand Up @@ -151,6 +153,10 @@ public class LedgerHandle implements WriteHandle {
}
}

public Map<Integer, BookieSocketAddress> getDelayedWriteFailedBookies() {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why do we have to expose as 'public' ?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

took care of it.

return delayedWriteFailedBookies;
}

LedgerHandle(BookKeeper bk, long ledgerId, LedgerMetadata metadata,
BookKeeper.DigestType digestType, byte[] password, EnumSet<WriteFlag> writeFlags)
throws GeneralSecurityException, NumberFormatException {
Expand Down Expand Up @@ -1749,6 +1755,42 @@ EnsembleInfo replaceBookieInMetadata(final Map<Integer, BookieSocketAddress> fai
return new EnsembleInfo(newEnsemble, failedBookies, replacedBookies);
}

void handleDelayedWriteBookieFailure() {
int curBlockAddCompletions = blockAddCompletions.get();
if (bk.getDisableEnsembleChangeFeature().isAvailable()) {
if (LOG.isDebugEnabled()) {
LOG.debug("Ensemble change is disabled. Failed bookies {} for ledger {}.",
delayedWriteFailedBookies, ledgerId);
}
return;
}
int curNumEnsembleChanges = numEnsembleChanges.incrementAndGet();
if (curNumEnsembleChanges > maxAllowedEnsembleChanges) {
if (LOG.isDebugEnabled()) {
LOG.debug("Exceeding maxAllowedEnsembeChanges {}. Failed bookies {} for ledger {}.",
maxAllowedEnsembleChanges, delayedWriteFailedBookies, ledgerId);
}
return;
}
synchronized (metadata) {
try {
EnsembleInfo ensembleInfo = replaceBookieInMetadata(delayedWriteFailedBookies, curNumEnsembleChanges);
if (ensembleInfo.replacedBookies.isEmpty()) {
return;
}
if (LOG.isDebugEnabled()) {
LOG.debug("[EnsembleChange-L{}-{}] : writing new ensemble info = {}",
getId(), curNumEnsembleChanges, ensembleInfo);
}
writeLedgerConfig(new ChangeEnsembleCb(ensembleInfo, curBlockAddCompletions,
curNumEnsembleChanges, false));
} catch (BKException.BKNotEnoughBookiesException e) {
LOG.error("Could not get additional bookie to remake ensemble: {}", ledgerId);
}
delayedWriteFailedBookies.clear();
}
}

void handleBookieFailure(final Map<Integer, BookieSocketAddress> failedBookies) {
int curBlockAddCompletions = blockAddCompletions.incrementAndGet();
if (bk.getDisableEnsembleChangeFeature().isAvailable()) {
Expand All @@ -1764,7 +1806,7 @@ void handleBookieFailure(final Map<Integer, BookieSocketAddress> failedBookies)
int curNumEnsembleChanges = numEnsembleChanges.incrementAndGet();

// when the ensemble changes are too frequent, close handle
if (curNumEnsembleChanges > maxAllowedEnsembleChanges){
if (curNumEnsembleChanges > maxAllowedEnsembleChanges) {
if (LOG.isDebugEnabled()) {
LOG.debug("Ledger {} reaches max allowed ensemble change number {}",
ledgerId, maxAllowedEnsembleChanges);
Expand All @@ -1783,7 +1825,10 @@ void handleBookieFailure(final Map<Integer, BookieSocketAddress> failedBookies)
LOG.debug("[EnsembleChange-L{}-{}] : writing new ensemble info = {}, block add completions = {}",
getId(), curNumEnsembleChanges, ensembleInfo, curBlockAddCompletions);
}
writeLedgerConfig(new ChangeEnsembleCb(ensembleInfo, curBlockAddCompletions, curNumEnsembleChanges));
writeLedgerConfig(new ChangeEnsembleCb(ensembleInfo, curBlockAddCompletions,
curNumEnsembleChanges, true));
// clear if there are any delayed write failures were recorded.
delayedWriteFailedBookies.clear();
} catch (BKException.BKNotEnoughBookiesException e) {
LOG.error("Could not get additional bookie to remake ensemble, closing ledger: {}", ledgerId);
handleUnrecoverableErrorDuringAdd(e.getCode());
Expand Down Expand Up @@ -1825,14 +1870,17 @@ private final class ChangeEnsembleCb extends OrderedGenericCallback<Void> {
private final EnsembleInfo ensembleInfo;
private final int curBlockAddCompletions;
private final int ensembleChangeIdx;
private final boolean addEntryFailureRecovery;

ChangeEnsembleCb(EnsembleInfo ensembleInfo,
int curBlockAddCompletions,
int ensembleChangeIdx) {
int ensembleChangeIdx,
boolean addEntryFailureRecovery) {
super(bk.getMainWorkerPool(), ledgerId);
this.ensembleInfo = ensembleInfo;
this.curBlockAddCompletions = curBlockAddCompletions;
this.ensembleChangeIdx = ensembleChangeIdx;
this.addEntryFailureRecovery = addEntryFailureRecovery;
}

@Override
Expand All @@ -1853,10 +1901,17 @@ public void safeOperationComplete(final int rc, Void result) {
} else if (rc != BKException.Code.OK) {
LOG.error("[EnsembleChange-L{}-{}] : could not persist ledger metadata : info = {}, "
+ "closing ledger : {}.", getId(), ensembleChangeIdx, ensembleInfo, rc);
handleUnrecoverableErrorDuringAdd(rc);
if (addEntryFailureRecovery) {
handleUnrecoverableErrorDuringAdd(rc);
}
return;
}
int newBlockAddCompletions = blockAddCompletions.decrementAndGet();
int newBlockAddCompletions;
if (addEntryFailureRecovery) {
newBlockAddCompletions = blockAddCompletions.decrementAndGet();
} else {
newBlockAddCompletions = blockAddCompletions.get();
}

if (LOG.isDebugEnabled()) {
LOG.info("[EnsembleChange-L{}-{}] : completed ensemble change, block add completion {} => {}",
Expand All @@ -1867,8 +1922,10 @@ public void safeOperationComplete(final int rc, Void result) {
ensembleChangeCounter.inc();
LOG.info("New Ensemble: {} for ledger: {}", ensembleInfo.newEnsemble, ledgerId);

// the failed bookie has been replaced
unsetSuccessAndSendWriteRequest(ensembleInfo.replacedBookies);
if (addEntryFailureRecovery) {
// the failed bookie has been replaced
unsetSuccessAndSendWriteRequest(ensembleInfo.replacedBookies);
}
}

@Override
Expand Down Expand Up @@ -2041,7 +2098,8 @@ private boolean updateMetadataIfPossible(LedgerMetadata newMeta) {
// merge ensemble infos from new meta except last ensemble
// since they might be modified by recovery tool.
metadata.mergeEnsembles(newMeta.getEnsembles());
writeLedgerConfig(new ChangeEnsembleCb(ensembleInfo, curBlockAddCompletions, ensembleChangeIdx));
writeLedgerConfig(new ChangeEnsembleCb(ensembleInfo, curBlockAddCompletions,
ensembleChangeIdx, true));
return true;
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -241,6 +241,12 @@ public void safeRun() {
entryId, lh.lastAddConfirmed, currentLedgerLength,
payload);

// We are about to send. Check if we need to make an ensemble change
// becasue of delayed write errors
Map <Integer, BookieSocketAddress> delayedWriteFailedBookies = lh.getDelayedWriteFailedBookies();
if (!delayedWriteFailedBookies.isEmpty()) {
lh.handleDelayedWriteBookieFailure();
}
// Iterate over set and trigger the sendWriteRequests
DistributionSchedule.WriteSet writeSet = lh.distributionSchedule.getWriteSet(entryId);
try {
Expand Down Expand Up @@ -276,6 +282,9 @@ public void writeComplete(int rc, long ledgerId, long entryId, BookieSocketAddre
// Got an error after satisfying AQ. This means we are under replicated at the create itself.
// Update the stat to reflect it.
addOpUrCounter.inc();
if (!lh.bk.getDisableEnsembleChangeFeature().isAvailable() && !lh.bk.delayEnsembleChange) {
lh.getDelayedWriteFailedBookies().putIfAbsent(bookieIndex, addr);
}
}
// even the add operation is completed, but because we don't reset completed flag back to false when
// #unsetSuccessAndSendWriteRequest doesn't break ack quorum constraint. we still have current pending
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@
import static org.apache.bookkeeper.client.BookKeeperClientStats.CLIENT_SCOPE;
import static org.apache.bookkeeper.client.BookKeeperClientStats.READ_OP_DM;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import io.netty.buffer.ByteBuf;
Expand Down Expand Up @@ -262,7 +263,75 @@ public void testWriteAndReadStats() throws Exception {
.get() > 0);
lh.close();
}
/**
* Verty delayedWriteError causes ensemble changes.
*/
@Test
public void testDelayedWriteEnsembleChange() throws Exception {
// Create a ledger
lh = bkc.createLedger(3, 3, 2, digestType, ledgerPassword);
baseClientConf.setAddEntryTimeout(1);

int numEntriesToWrite = 10;
// write-batch-1
for (int i = 0; i < numEntriesToWrite; i++) {
ByteBuffer entry = ByteBuffer.allocate(4);
entry.putInt(rng.nextInt(maxInt));
entry.position(0);

entries1.add(entry.array());
lh.addEntry(entry.array());
}

CountDownLatch sleepLatch1 = new CountDownLatch(1);

// get bookie at index-0
BookieSocketAddress bookie1 = lh.getLedgerMetadata().currentEnsemble.get(0);
sleepBookie(bookie1, sleepLatch1);

int i = numEntriesToWrite;
numEntriesToWrite = numEntriesToWrite + 10;

// write-batch-2

for (; i < numEntriesToWrite; i++) {
ByteBuffer entry = ByteBuffer.allocate(4);
entry.putInt(rng.nextInt(maxInt));
entry.position(0);

entries1.add(entry.array());
lh.addEntry(entry.array());
}
// Sleep to receive delayed error on the write directed to the sleeping bookie
Thread.sleep(baseClientConf.getAddEntryTimeout() * 1000 * 2);
assertTrue(
"Stats should have captured a new UnderReplication during write",
bkc.getTestStatsProvider().getCounter(
CLIENT_SCOPE + "." + ADD_OP_UR)
.get() > 0);

i = numEntriesToWrite;
numEntriesToWrite = numEntriesToWrite + 10;

// write-batch-3
for (; i < numEntriesToWrite; i++) {
ByteBuffer entry = ByteBuffer.allocate(4);
entry.putInt(rng.nextInt(maxInt));
entry.position(0);

entries1.add(entry.array());
lh.addEntry(entry.array());
}

sleepLatch1.countDown();
// get the bookie at index-0 again, this must be different.
BookieSocketAddress bookie2 = lh.getLedgerMetadata().currentEnsemble.get(0);

assertFalse(
"Delayed write error must have forced ensemble change",
bookie1.equals(bookie2));
lh.close();
}
/**
* Verify the functionality Ledgers with different digests.
*
Expand Down