Skip to content
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

Issue #1791: Read Submission should bypass OSE Threads #1792

Merged
merged 1 commit into from
Nov 8, 2018

Conversation

nicmichael
Copy link
Contributor

Motivation

Profiling of our Bookkeeper Client code for read requests shows that client threads spend half of their time in dispatching requests to OrderedExecutors (just the dispatch itself, not the execution inside OSE): 54% of their CPU time is spent in OrderedExecutor.executeOrdered() (called by LedgerHandle.readEntriesInternalAsync()). The high time spend in request submission to OSE is largely caused by Linux scheduling cost, that is the cost of dispatching the OSE thread to CPU: 42% of total time (3/4th of executeOrdered() time), threads spend in Unsafe.unpark(), which is essentially Linux scheduling/dispatching of another thread.

Changes

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).

Master Issue: #1791: Read Submission should bypass OSE Threads

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).
Copy link
Contributor

@eolivelli eolivelli left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can we add some test case? We already have mockito based tests for PendingAppOp.

I am not sure we can able this by default, what about having a configuration flag?

@sijie
Copy link
Member

sijie commented Nov 6, 2018

@eolivelli

Can we add some test case?

don't think we need test cases. read op is a common enough operation, which should be covered by all existing test cases.

I am not sure we can able this by default,

I think the change is enabling only on readonly handles. that seems to be fine.

@nicmichael
Copy link
Contributor Author

@eolivelli I looked at the existing test cases. As sijie pointed out, there's quite a few existing ones that test reads on both read-only as well as read-write ledger handles. I hope that would be sufficient. Or were you thinking of any kind of special tests?

@eolivelli
Copy link
Contributor

I was talking about adding a test about the fact that if the handlr is read-only the operation is not submitted to the OSE, otherwise it uses the OSE.

Not blocker

Copy link
Contributor

@eolivelli eolivelli left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Nice improvement.

Thank you @nicmichael

@eolivelli
Copy link
Contributor

No need for a flag

@sijie
Copy link
Member

sijie commented Nov 7, 2018

@merlimat please review this since you have comment at #1791

@sijie sijie added this to the 4.9.0 milestone Nov 8, 2018
@sijie sijie merged commit 6b99ff7 into apache:master Nov 8, 2018
sijie pushed a commit that referenced this pull request Nov 8, 2018
Profiling of our Bookkeeper Client code for read requests shows that client threads spend half of their time in dispatching requests to OrderedExecutors (just the dispatch itself, not the execution inside OSE): 54% of their CPU time is spent in OrderedExecutor.executeOrdered() (called by LedgerHandle.readEntriesInternalAsync()). The high time spend in request submission to OSE is largely caused by Linux scheduling cost, that is the cost of dispatching the OSE thread to CPU: 42% of total time (3/4th of executeOrdered() time), threads spend in Unsafe.unpark(), which is essentially Linux scheduling/dispatching of another thread.

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).

Master Issue: #1791: Read Submission should bypass OSE Threads

Reviewers: Enrico Olivelli <eolivelli@gmail.com>, Andrey Yegorov <None>, Sijie Guo <sijie@apache.org>, Matteo Merli <mmerli@apache.org>

This closes #1792 from nicmichael/DirectRead, closes #1791
@lhotari
Copy link
Member

lhotari commented Mar 13, 2022

In Pulsar, apache/pulsar#14436 seems to be caused by a thread safety issue in Bookkeeper client. It looks like it is caused by the changes in this PR.

The reason why there's a thread safety issue is that a network request-response call doesn't ensure happens-before. Happens-before visibility guarantee is needed since there's no synchronization in the PendingReadOp handling.

@lhotari
Copy link
Member

lhotari commented Mar 13, 2022

BookKeeper uses LinkedBlockingQueue for executors by default which isn't very efficient.

A better Queue implementation would be something that Jetty uses for it's thread pools https://github.com/eclipse/jetty.project/blob/jetty-10.0.x/jetty-util/src/main/java/org/eclipse/jetty/util/BlockingArrayQueue.java .

By improving the queue implementation, there would be better performance without breaking thread safety.

The BookKeeper code includes https://github.com/apache/bookkeeper/blob/master/bookkeeper-server/src/main/java/org/apache/bookkeeper/util/collections/GrowableArrayBlockingQueue.java queue implementation.

@lhotari
Copy link
Member

lhotari commented Mar 14, 2022

I created #3104 about the thread safety issue.

@lhotari
Copy link
Member

lhotari commented Mar 15, 2022

#3104 turned out to be a clear state handling issue in PendingReadOp class (and included embedded classes), fix is #3110

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

None yet

6 participants