@Override public void readEntryComplete(int rc, long ledgerId, final long entryId, final ByteBuf buffer, Object ctx) { final ReadContext rctx = (ReadContext) ctx; final LedgerEntryRequest entry = rctx.entry; if (rc != BKException.Code.OK) { entry.logErrorAndReattemptRead(rctx.bookieIndex, rctx.to, "Error: " + BKException.getMessage(rc), rc); return; } heardFromHosts.add(rctx.to); heardFromHostsBitSet.set(rctx.bookieIndex, true); buffer.retain(); if (entry.complete(rctx.bookieIndex, rctx.to, buffer)) { if (!isRecoveryRead) { // do not advance LastAddConfirmed for recovery reads lh.updateLastConfirmed(rctx.getLastAddConfirmed(), 0L); } submitCallback(BKException.Code.OK); } else { buffer.release(); } if (numPendingEntries < 0) { LOG.error("Read too many values for ledger {} : [{}, {}].", ledgerId, startEntryId, endEntryId); } }
Integer firstRc = null; for (LedgerEntryRequest req : seq) { if (!req.isComplete()) { firstUnread = req.eId; firstRc = req.rc;
@Override public Boolean call() throws Exception { if (!isComplete() && null != maybeSendSpeculativeRead(heardFromHostsBitSet)) { if (LOG.isDebugEnabled()) { LOG.debug("Send speculative read for {}. Hosts sent are {}, " + " Hosts heard are {}, ensemble is {}.", this, sentToHosts, heardFromHostsBitSet, ensemble); } return true; } return false; } });
} catch (BKDigestMatchException e) { clientCtx.getClientStats().getReadOpDmCounter().inc(); logErrorAndReattemptRead(bookieIndex, host, "Mac mismatch", BKException.Code.DigestMatchException); return false;
void initiate() { long nextEnsembleChange = startEntryId, i = startEntryId; this.requestTimeNanos = MathUtils.nowInNano(); List<BookieSocketAddress> ensemble = null; do { if (i == nextEnsembleChange) { ensemble = getLedgerMetadata().getEnsembleAt(i); nextEnsembleChange = LedgerMetadataUtils.getNextEnsembleChange(getLedgerMetadata(), i); } LedgerEntryRequest entry; if (parallelRead) { entry = new ParallelReadRequest(ensemble, lh.ledgerId, i); } else { entry = new SequenceReadRequest(ensemble, lh.ledgerId, i); } seq.add(entry); i++; } while (i <= endEntryId); // read the entries. for (LedgerEntryRequest entry : seq) { entry.read(); if (!parallelRead && clientCtx.getConf().readSpeculativeRequestPolicy.isPresent()) { speculativeTask = clientCtx.getConf().readSpeculativeRequestPolicy.get() .initiateSpeculativeRequest(clientCtx.getScheduler(), entry); } } }
@Override protected void submitCallback(int code) { LedgerEntryRequest request; while (!seq.isEmpty() && (request = seq.get(0)) != null) { if (!request.isComplete()) { return; } seq.remove(0); long latencyNanos = MathUtils.elapsedNanos(requestTimeNanos); LedgerEntry entry; if (BKException.Code.OK == request.getRc()) { clientCtx.getClientStats().getReadOpLogger() .registerSuccessfulEvent(latencyNanos, TimeUnit.NANOSECONDS); // callback with completed entry entry = new LedgerEntry(request.entryImpl); } else { clientCtx.getClientStats().getReadOpLogger() .registerFailedEvent(latencyNanos, TimeUnit.NANOSECONDS); entry = null; } request.close(); listener.onEntryComplete(request.getRc(), lh, entry, ctx); } // if all entries are already completed. cancelSpeculativeTask(true); }
@Override synchronized void logErrorAndReattemptRead(int bookieIndex, BookieSocketAddress host, String errMsg, int rc) { super.logErrorAndReattemptRead(bookieIndex, host, errMsg, rc); int replica = writeSet.indexOf(bookieIndex); if (replica == NOT_FOUND) { LOG.error("Received error from a host which is not in the ensemble {} {}.", host, ensemble); return; } erroredReplicas.set(replica); if (isRecoveryRead && (numBookiesMissingEntry >= requiredBookiesMissingEntryForRecovery)) { /* For recovery, report NoSuchEntry as soon as wQ-aQ+1 bookies report that they do not * have the entry */ fail(BKException.Code.NoSuchEntryException); return; } if (!readsOutstanding()) { sendNextRead(); } }
@Override boolean complete(int bookieIndex, BookieSocketAddress host, ByteBuf buffer) { boolean completed = super.complete(bookieIndex, host, buffer); if (completed) { int numReplicasTried = getNextReplicaIndexToReadFrom(); // Check if any speculative reads were issued and mark any slow bookies before // the first successful speculative read as "slow" for (int i = 0; i < numReplicasTried - 1; i++) { int slowBookieIndex = writeSet.get(i); BookieSocketAddress slowBookieSocketAddress = ensemble.get(slowBookieIndex); clientCtx.getPlacementPolicy().registerSlowBookie(slowBookieSocketAddress, eId); } } return completed; } }
@Override synchronized void logErrorAndReattemptRead(int bookieIndex, BookieSocketAddress host, String errMsg, int rc) { super.logErrorAndReattemptRead(bookieIndex, host, errMsg, rc); // if received all responses or this entry doesn't meet quorum write, complete the request. --numPendings; if (isRecoveryRead && numBookiesMissingEntry >= requiredBookiesMissingEntryForRecovery) { /* For recovery, report NoSuchEntry as soon as wQ-aQ+1 bookies report that they do not * have the entry */ fail(BKException.Code.NoSuchEntryException); } else if (numPendings == 0) { // if received all responses, complete the request. fail(firstError); } }