-
Notifications
You must be signed in to change notification settings - Fork 893
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
AutoRecovery supports batch read #4211
Merged
merlimat
merged 3 commits into
apache:master
from
hangc0276:chenhang/autorecovery_support_batch_read
Feb 20, 2024
Merged
Changes from 2 commits
Commits
Show all changes
3 commits
Select commit
Hold shift + click to select a range
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -171,29 +171,41 @@ private void replicateFragmentInternal(final LedgerHandle lh, | |
return; | ||
} | ||
|
||
/* | ||
* Add all the entries to entriesToReplicate list from | ||
* firstStoredEntryId to lastStoredEntryID. | ||
*/ | ||
List<Long> entriesToReplicate = new LinkedList<Long>(); | ||
long lastStoredEntryId = lf.getLastStoredEntryId(); | ||
for (long i = lf.getFirstStoredEntryId(); i <= lastStoredEntryId; i++) { | ||
entriesToReplicate.add(i); | ||
} | ||
/* | ||
* Now asynchronously replicate all of the entries for the ledger | ||
* fragment that were on the dead bookie. | ||
*/ | ||
int entriesToReplicateCnt = (int) (endEntryId - startEntryId + 1); | ||
MultiCallback ledgerFragmentEntryMcb = new MultiCallback( | ||
entriesToReplicate.size(), ledgerFragmentMcb, null, BKException.Code.OK, | ||
entriesToReplicateCnt, ledgerFragmentMcb, null, BKException.Code.OK, | ||
BKException.Code.LedgerRecoveryException); | ||
if (this.replicationThrottle != null) { | ||
this.replicationThrottle.resetRate(this.conf.getReplicationRateByBytes()); | ||
} | ||
for (final Long entryId : entriesToReplicate) { | ||
recoverLedgerFragmentEntry(entryId, lh, ledgerFragmentEntryMcb, | ||
|
||
if (conf.isRecoveryBatchReadEnabled() | ||
&& conf.getUseV2WireProtocol() | ||
&& conf.isBatchReadEnabled() | ||
&& lh.getLedgerMetadata().getEnsembleSize() == lh.getLedgerMetadata().getWriteQuorumSize()) { | ||
batchRecoverLedgerFragmentEntry(startEntryId, endEntryId, lh, ledgerFragmentEntryMcb, | ||
newBookies, onReadEntryFailureCallback); | ||
|
||
} else { | ||
/* | ||
* Add all the entries to entriesToReplicate list from | ||
* firstStoredEntryId to lastStoredEntryID. | ||
*/ | ||
List<Long> entriesToReplicate = new LinkedList<Long>(); | ||
long lastStoredEntryId = lf.getLastStoredEntryId(); | ||
for (long i = lf.getFirstStoredEntryId(); i <= lastStoredEntryId; i++) { | ||
entriesToReplicate.add(i); | ||
} | ||
for (final Long entryId : entriesToReplicate) { | ||
recoverLedgerFragmentEntry(entryId, lh, ledgerFragmentEntryMcb, | ||
newBookies, onReadEntryFailureCallback); | ||
} | ||
} | ||
|
||
} | ||
|
||
/** | ||
|
@@ -433,6 +445,109 @@ public void readComplete(int rc, LedgerHandle lh, | |
}, null); | ||
} | ||
|
||
void batchRecoverLedgerFragmentEntry(final long startEntryId, | ||
final long endEntryId, | ||
final LedgerHandle lh, | ||
final AsyncCallback.VoidCallback ledgerFragmentMcb, | ||
final Set<BookieId> newBookies, | ||
final BiConsumer<Long, Long> onReadEntryFailureCallback) | ||
throws InterruptedException { | ||
int entriesToReplicateCnt = (int) (endEntryId - startEntryId + 1); | ||
int maxBytesToReplicate = conf.getReplicationRateByBytes(); | ||
if (replicationThrottle != null) { | ||
if (maxBytesToReplicate != -1 && maxBytesToReplicate > averageEntrySize.get() * entriesToReplicateCnt) { | ||
maxBytesToReplicate = averageEntrySize.get() * entriesToReplicateCnt; | ||
} | ||
replicationThrottle.acquire(maxBytesToReplicate); | ||
} | ||
|
||
lh.asyncBatchReadEntries(startEntryId, entriesToReplicateCnt, maxBytesToReplicate, | ||
new ReadCallback() { | ||
@Override | ||
public void readComplete(int rc, LedgerHandle lh, Enumeration<LedgerEntry> seq, Object ctx) { | ||
if (rc != BKException.Code.OK) { | ||
LOG.error("BK error reading ledger entries: {} - {}", | ||
startEntryId, endEntryId, BKException.create(rc)); | ||
onReadEntryFailureCallback.accept(lh.getId(), startEntryId); | ||
ledgerFragmentMcb.processResult(rc, null, null); | ||
return; | ||
} | ||
long lastEntryId = startEntryId; | ||
while (seq.hasMoreElements()) { | ||
LedgerEntry entry = seq.nextElement(); | ||
lastEntryId = entry.getEntryId(); | ||
byte[] data = entry.getEntry(); | ||
final long dataLength = data.length; | ||
numEntriesRead.inc(); | ||
numBytesRead.registerSuccessfulValue(dataLength); | ||
|
||
ReferenceCounted toSend = lh.getDigestManager() | ||
.computeDigestAndPackageForSending(entry.getEntryId(), | ||
lh.getLastAddConfirmed(), entry.getLength(), | ||
Unpooled.wrappedBuffer(data, 0, data.length), | ||
lh.getLedgerKey(), | ||
BookieProtocol.FLAG_RECOVERY_ADD); | ||
if (replicationThrottle != null) { | ||
if (toSend instanceof ByteBuf) { | ||
updateAverageEntrySize(((ByteBuf) toSend).readableBytes()); | ||
} else if (toSend instanceof ByteBufList) { | ||
updateAverageEntrySize(((ByteBufList) toSend).readableBytes()); | ||
} | ||
} | ||
AtomicInteger numCompleted = new AtomicInteger(0); | ||
AtomicBoolean completed = new AtomicBoolean(false); | ||
|
||
WriteCallback multiWriteCallback = new WriteCallback() { | ||
@Override | ||
public void writeComplete(int rc, long ledgerId, long entryId, BookieId addr, Object ctx) { | ||
if (rc != BKException.Code.OK) { | ||
LOG.error("BK error writing entry for ledgerId: {}, entryId: {}, bookie: {}", | ||
ledgerId, entryId, addr, BKException.create(rc)); | ||
if (completed.compareAndSet(false, true)) { | ||
ledgerFragmentMcb.processResult(rc, null, null); | ||
} | ||
} else { | ||
numEntriesWritten.inc(); | ||
if (ctx instanceof Long) { | ||
numBytesWritten.registerSuccessfulValue((Long) ctx); | ||
} | ||
if (LOG.isDebugEnabled()) { | ||
LOG.debug("Success writing ledger id {}, entry id {} to a new bookie {}!", | ||
ledgerId, entryId, addr); | ||
} | ||
if (numCompleted.incrementAndGet() == newBookies.size() | ||
&& completed.compareAndSet(false, true)) { | ||
ledgerFragmentMcb.processResult(rc, null, null); | ||
} | ||
} | ||
} | ||
}; | ||
|
||
for (BookieId newBookie : newBookies) { | ||
long startWriteEntryTime = MathUtils.nowInNano(); | ||
bkc.getBookieClient().addEntry(newBookie, lh.getId(), | ||
lh.getLedgerKey(), entry.getEntryId(), toSend, | ||
multiWriteCallback, dataLength, BookieProtocol.FLAG_RECOVERY_ADD, | ||
false, WriteFlag.NONE); | ||
writeDataLatency.registerSuccessfulEvent( | ||
MathUtils.elapsedNanos(startWriteEntryTime), TimeUnit.NANOSECONDS); | ||
} | ||
toSend.release(); | ||
} | ||
if (lastEntryId == endEntryId) { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. When There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Good point! Updated |
||
ledgerFragmentMcb.processResult(BKException.Code.OK, null, null); | ||
} else { | ||
try { | ||
batchRecoverLedgerFragmentEntry(lastEntryId + 1, endEntryId, lh, | ||
ledgerFragmentMcb, newBookies, onReadEntryFailureCallback); | ||
} catch (InterruptedException e) { | ||
ledgerFragmentMcb.processResult(BKException.Code.InterruptedException, null, null); | ||
} | ||
} | ||
} | ||
}, null); | ||
} | ||
|
||
private void updateAverageEntrySize(int toSendSize) { | ||
averageEntrySize.updateAndGet(value -> (int) (value * AVERAGE_ENTRY_SIZE_RATIO | ||
+ (1 - AVERAGE_ENTRY_SIZE_RATIO) * toSendSize)); | ||
|
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The
ledgerFragmentMcb
isMultiCallback
, when batch read fails, theprocessResult
invoke count may be not enough.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Good point. Updated.