Skip to content

Commit

Permalink
Issue apache#1791: Read Submission should bypass OSE Threads
Browse files Browse the repository at this point in the history
This change executes read submissions (PendingReadOp) on read-only ledger handles directly inside the client thread instead of submitting them to Ordered Executors.

Tests with a prototype have shown significant improvements in both overall CPU consumption as well as read latency. The additional work client threads have to do (the dispatch of the read requests to netty) is roughly the same as the (saved) dispatch cost to OSE, so the change turns out to be neutral for CPU consumption of client threads. In some experiments, the savings even exceed the additional work, and client threads consume less cpu even though they "do more". It also frees up lots of resources in OSE threads. Since it eliminates one context-switch in read submission and also avoids serialization of reads to the same ledger (or ledgers hashing to the same OSE), it also reduces read latency. For a mixed read-write workload (14,000 reads/sec on read-only ledgers, 4,000 writes/sec on another set of ledgers), this change has reduced CPU consumption of OSE threads by 25%, kept CPU consumption of client (and Netty) threads the same, and yielded a 6% improvement of read latency (as measured by BK Client).
  • Loading branch information
nicmichael committed Nov 6, 2018
1 parent 1087471 commit fa26921
Showing 1 changed file with 8 additions and 1 deletion.
Expand Up @@ -840,7 +840,14 @@ CompletableFuture<LedgerEntries> readEntriesInternalAsync(long firstEntry,
ws.recycle();
}

clientCtx.getMainWorkerPool().executeOrdered(ledgerId, op);
if (isHandleWritable()) {
// Ledger handle in read/write mode: submit to OSE for ordered execution.
clientCtx.getMainWorkerPool().executeOrdered(ledgerId, op);
} else {
// Read-only ledger handle: bypass OSE and execute read directly in client thread.
// This avoids a context-switch to OSE thread and thus reduces latency.
op.run();
}
} else {
op.future().completeExceptionally(BKException.create(ClientClosedException));
}
Expand Down

0 comments on commit fa26921

Please sign in to comment.