Skip to content

Commit

Permalink
Tune the BP-62 doc.
Browse files Browse the repository at this point in the history
  • Loading branch information
horizonzy committed Mar 12, 2024
1 parent 34bd4b3 commit 78a9311
Show file tree
Hide file tree
Showing 2 changed files with 13 additions and 18 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -81,9 +81,9 @@ protected void submitCallback(int code) {
long latencyNanos = MathUtils.elapsedNanos(requestTimeNanos);
if (code != BKException.Code.OK) {
LOG.error(
"Read of ledger entry failed: L{} E{}-E{}, Sent to {}, "
"Batch read of ledger entry failed: L{} E{}-E{}, Sent to {}, "
+ "Heard from {} : bitset = {}, Error = '{}'. First unread entry is ({}, rc = {})",
lh.getId(), startEntryId, endEntryId, sentToHosts, heardFromHosts, heardFromHostsBitSet,
lh.getId(), startEntryId, startEntryId + maxCount, sentToHosts, heardFromHosts, heardFromHostsBitSet,
BKException.getMessage(code), startEntryId, code);
clientCtx.getClientStats().getReadOpLogger().registerFailedEvent(latencyNanos, TimeUnit.NANOSECONDS);
// release the entries
Expand Down
27 changes: 11 additions & 16 deletions site3/website/src/pages/bps/BP-62-new-API-for-batched-reads.md
Original file line number Diff line number Diff line change
Expand Up @@ -44,24 +44,19 @@ that takes into account the expected count and size of entries.
## BookKeeper Client API
1. The new APIs will be added to BookieClient.java
```java
default void readEntries(BookieId address, long ledgerId, long startEntryId,
default void batchReadEntries(BookieId address, long ledgerId, long startEntryId,
int maxCount, long maxSize, BatchedReadEntryCallback cb, Object ctx,
int flags) {
readEntries(address, ledgerId, startEntryId, maxCount, maxSize, cb, ctx, flags, null);
batchReadEntries(address, ledgerId, startEntryId, maxCount, maxSize, cb, ctx, flags, null);
}
default void readEntries(BookieId address, long ledgerId, long startEntryId,
default void batchReadEntries(BookieId address, long ledgerId, long startEntryId,
int maxCount, long maxSize, BatchedReadEntryCallback cb, Object ctx,
int flags, byte[] masterKey) {
readEntries(address, ledgerId, startEntryId, maxCount, maxSize, cb, ctx, flags, masterKey, false);
batchReadEntries(address, ledgerId, startEntryId, maxCount, maxSize, cb, ctx, flags, masterKey, false);
}

void readEntries(BookieId address, long ledgerId, long startEntryId,
void batchReadEntries(BookieId address, long ledgerId, long startEntryId,
int maxCount, long maxSize, BatchedReadEntryCallback cb, Object ctx,
int flags, byte[] masterKey, boolean allowFastFail);

void readEntiesWithFallback(BookieId address, long ledgerId, long startEntryId,
int maxCount, long maxSize, BatchedReadEntryCallback cb, Object ctx,
int flags, byte[] masterKey, boolean allowFastFail)
```
2. The new class BatchedReadEntryCallback will be added to BookkeeperInternalCallbacks.java
```java
Expand All @@ -71,17 +66,17 @@ public interface BatchedReadEntryCallback {
```
3. The new APIs will be added to ReadHandle.java
```java
CompletableFuture<LedgerEntries> readAsync(long startEntry, int maxCount, long maxSize);

default LedgerEntries read(long startEntry, int maxCount, long maxSize) throws BKException, InterruptedException {
return FutureUtils.result(readAsync(startEntry, maxCount, maxSize),
BKException.HANDLER);
default CompletableFuture<LedgerEntries> batchReadAsync(long startEntry, int maxCount, long maxSize);
default LedgerEntries batchRead(long startEntry, int maxCount, long maxSize)
throws BKException, InterruptedException {
return FutureUtils.result(batchReadAsync(startEntry, maxCount, maxSize), BKException.HANDLER);
}
```
## Wire protocol changes

In BookKeeper, the V2 protocol uses a custom encoding format. So we need to handle the data encoding and decoding.
The V3 protocol uses the ProtoBuf for encoding and decoding.
The V3 protocol uses the ProtoBuf for encoding and decoding.(Not support now)

### V2 Protocol

Expand Down

0 comments on commit 78a9311

Please sign in to comment.