@Override public void readComplete(int rc, LedgerHandle lh, Enumeration<LedgerEntry> entries, Object ctx) { if (BKException.Code.NoSuchEntryException == rc) { callback.operationComplete(BKException.Code.OK, resultList); } else if (BKException.Code.OK == rc) { while (entries.hasMoreElements()) { resultList.add(entries.nextElement()); } long entryId = (Long) ctx; ++entryId; PendingReadOp readOp = new PendingReadOp(lh, lh.bk.scheduler, entryId, entryId, this, entryId); readOp.initiate(); } else { callback.operationComplete(rc, resultList); } } };
private void readNext(long entryId) { PendingReadOp op = new PendingReadOp(lh, clientCtx, entryId, entryId, false); op.future().whenComplete(this); op.submit(); }
CompletableFuture<LedgerEntries> readEntriesInternalAsync(long firstEntry, long lastEntry, boolean isRecoveryRead) { PendingReadOp op = new PendingReadOp(this, clientCtx, firstEntry, lastEntry, isRecoveryRead); if (!clientCtx.isClientClosed()) { try { if (!waitForWritable(ws, firstEntry, ws.size() - 1, clientCtx.getConf().waitForWriteSetMs)) { op.allowFailFastOnUnwritableChannel(); op.run(); op.future().completeExceptionally(BKException.create(ClientClosedException)); return op.future();
@Override public void safeRun() { initiate(); }
synchronized BookieSocketAddress sendNextRead() { if (nextReplicaIndexToReadFrom >= getLedgerMetadata().getWriteQuorumSize()) { // we are done, the read has failed from all replicas, just fail the // read fail(firstError); return null; } // ToDo: pick replica with writable PCBC. ISSUE #1239 // https://github.com/apache/bookkeeper/issues/1239 int replica = nextReplicaIndexToReadFrom; int bookieIndex = writeSet.get(nextReplicaIndexToReadFrom); nextReplicaIndexToReadFrom++; try { BookieSocketAddress to = ensemble.get(bookieIndex); sendReadTo(bookieIndex, to, this); sentToHosts.add(to); sentReplicas.set(replica); return to; } catch (InterruptedException ie) { LOG.error("Interrupted reading entry " + this, ie); Thread.currentThread().interrupt(); fail(BKException.Code.InterruptedException); return null; } }
PendingReadOp(LedgerHandle lh, ClientContext clientCtx, long startEntryId, long endEntryId, boolean isRecoveryRead) { this.seq = new ArrayList<>((int) ((endEntryId + 1) - startEntryId)); this.future = new CompletableFuture<>(); this.lh = lh; this.clientCtx = clientCtx; this.startEntryId = startEntryId; this.endEntryId = endEntryId; this.isRecoveryRead = isRecoveryRead; this.allowFailFast = false; numPendingEntries = endEntryId - startEntryId + 1; requiredBookiesMissingEntryForRecovery = getLedgerMetadata().getWriteQuorumSize() - getLedgerMetadata().getAckQuorumSize() + 1; heardFromHosts = new HashSet<>(); heardFromHostsBitSet = new BitSet(getLedgerMetadata().getEnsembleSize()); }
@Override void read() { for (int i = 0; i < writeSet.size(); i++) { BookieSocketAddress to = ensemble.get(writeSet.get(i)); try { sendReadTo(writeSet.get(i), to, this); } catch (InterruptedException ie) { LOG.error("Interrupted reading entry {} : ", this, ie); Thread.currentThread().interrupt(); fail(BKException.Code.InterruptedException); return; } } }
PendingReadOp op = new PendingReadOp(lh, clientCtx, entryId, entryId, false); op.future().whenComplete(this); op.submit(); PendingReadOp op = new PendingReadOp(lh, clientCtx, entryId, entryId, false); op.future().whenComplete(readListener); op.submit(); };
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 public void readLastConfirmedDataComplete(int rc, DigestManager.RecoveryData recoveryData) { if (BKException.Code.OK != rc) { callback.operationComplete(rc, resultList); return; } if (LedgerHandle.INVALID_ENTRY_ID >= recoveryData.lastAddConfirmed) { callback.operationComplete(BKException.Code.OK, resultList); return; } long entryId = recoveryData.lastAddConfirmed; PendingReadOp readOp = new PendingReadOp(lh, lh.bk.scheduler, entryId, entryId, readCallback, entryId); try { readOp.initiate(); } catch (Throwable t) { logger.error("Failed to initialize pending read entry {} for ledger {} : ", new Object[] { entryId, lh.getLedgerMetadata(), t }); } } };
/** * Send to next replica speculatively, if required and possible. * This returns the host we may have sent to for unit testing. * @return host we sent to if we sent. null otherwise. */ @Override synchronized BookieSocketAddress maybeSendSpeculativeRead(BitSet heardFrom) { if (nextReplicaIndexToReadFrom >= getLedgerMetadata().getWriteQuorumSize()) { return null; } BitSet sentTo = getSentToBitSet(); sentTo.and(heardFrom); // only send another read if we have had no successful response at all // (even for other entries) from any of the other bookies we have sent the // request to if (sentTo.cardinality() == 0) { clientCtx.getClientStats().getSpeculativeReadCounter().inc(); return sendNextRead(); } else { return null; } }