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

LedgerHandle: do not complete metadata operation on the ZookKeeper/Metadata callback thread #3516

Merged
merged 6 commits into from Oct 6, 2022

Conversation

eolivelli
Copy link
Contributor

Motivation

We are currently completing the creation and opening (also with recovery) of the LedgerHandle in the main ZookKeeper (or metadata driver) callback thread.

This leads to unpredictable code to be executed on that thread:

  • possible deadlocks (that cannot be prevented fully from BK code)
  • execute heavy weight operations, like opening BK connections

This is a good example of a something that tries to open a connection to a bookie because the application executes a "read" after a "openLedger" operation.

at org.apache.bookkeeper.proto.PerChannelBookieClient.connect(PerChannelBookieClient.java:532)
	at org.apache.bookkeeper.proto.PerChannelBookieClient.connectIfNeededAndDoOp(PerChannelBookieClient.java:658)
	at org.apache.bookkeeper.proto.DefaultPerChannelBookieClientPool.initialize(DefaultPerChannelBookieClientPool.java:92)
	at org.apache.bookkeeper.proto.BookieClientImpl.lookupClient(BookieClientImpl.java:217)
	at org.apache.bookkeeper.proto.BookieClientImpl.isWritable(BookieClientImpl.java:170)
	at org.apache.bookkeeper.client.LedgerHandle.isWriteSetWritable(LedgerHandle.java:1227)
	at org.apache.bookkeeper.client.LedgerHandle.waitForWritable(LedgerHandle.java:1249)
	at org.apache.bookkeeper.client.LedgerHandle.readEntriesInternalAsync(LedgerHandle.java:883)
	at org.apache.bookkeeper.client.LedgerHandle.asyncReadEntriesInternal(LedgerHandle.java:800)
	at org.apache.bookkeeper.client.LedgerHandle.asyncReadEntries(LedgerHandle.java:694)
	at org.apache.pulsar.broker.service.schema.BookkeeperSchemaStorage$Functions.getLedgerEntry(BookkeeperSchemaStorage.java:646)
	at org.apache.pulsar.broker.service.schema.BookkeeperSchemaStorage.lambda$readSchemaEntry$33(BookkeeperSchemaStorage.java:524)
	at org.apache.pulsar.broker.service.schema.BookkeeperSchemaStorage$$Lambda$820/0x00000008007e5840.apply(Unknown Source)
	at java.util.concurrent.CompletableFuture$UniCompose.tryFire(java.base@11.0.15.0.1/CompletableFuture.java:1072)
	at java.util.concurrent.CompletableFuture.postComplete(java.base@11.0.15.0.1/CompletableFuture.java:506)
	at java.util.concurrent.CompletableFuture.complete(java.base@11.0.15.0.1/CompletableFuture.java:2073)
	at org.apache.pulsar.broker.service.schema.BookkeeperSchemaStorage.lambda$openLedger$40(BookkeeperSchemaStorage.java:601)
	at org.apache.pulsar.broker.service.schema.BookkeeperSchemaStorage$$Lambda$819/0x00000008007e5440.openComplete(Unknown Source)
	at org.apache.bookkeeper.client.LedgerOpenOp.openComplete(LedgerOpenOp.java:248)
	at org.apache.bookkeeper.client.LedgerOpenOp.openWithMetadata(LedgerOpenOp.java:201)
	at org.apache.bookkeeper.client.LedgerOpenOp.lambda$initiate$0(LedgerOpenOp.java:119)
	at org.apache.bookkeeper.client.LedgerOpenOp$$Lambda$621/0x0000000800715040.accept(Unknown Source)

The full story is here, but this patch does not focus on the BookieAddressResolver blocking calls to the metadata storage. The scope is this patch is to prevent application from running code on the metadata store main thread.
apache/pulsar#17913

It is unfortunate that CompletableFuture doesn't allow you to have full control on the thread which will execute the completion tasks

Changes

(Describe: what changes you have made)

Master Issue: #


In order to uphold a high standard for quality for code contributions, Apache BookKeeper runs various precommit
checks for pull requests. A pull request can only be merged when it passes precommit checks.


Be sure to do all of the following to help us incorporate your contribution
quickly and easily:

If this PR is a BookKeeper Proposal (BP):

  • Make sure the PR title is formatted like:
    <BP-#>: Description of bookkeeper proposal
    e.g. BP-1: 64 bits ledger is support
  • Attach the master issue link in the description of this PR.
  • Attach the google doc link if the BP is written in Google Doc.

Otherwise:

  • Make sure the PR title is formatted like:
    <Issue #>: Description of pull request
    e.g. Issue 123: Description ...
  • Make sure tests pass via mvn clean apache-rat:check install spotbugs:check.
  • Replace <Issue #> in the title with the actual Issue number.

Copy link
Contributor

@nicoloboschi nicoloboschi left a comment

Choose a reason for hiding this comment

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

LGTM

@eolivelli eolivelli self-assigned this Oct 4, 2022
@eolivelli eolivelli added this to the 4.16.0 milestone Oct 4, 2022
@dlg99
Copy link
Contributor

dlg99 commented Oct 4, 2022

overall LGTM.

Client tests failed with

Error:  Errors: 
Error:  org.apache.bookkeeper.client.BookieWriteLedgerTest.testLedgerCreateAdvWithLedgerIdInLoop
Error:    Run 1: BookieWriteLedgerTest.testLedgerCreateAdvWithLedgerIdInLoop:759 » TestTimedOut
Error:    Run 2: BookieWriteLedgerTest.testLedgerCreateAdvWithLedgerIdInLoop:759 » TestTimedOut
Error:    Run 3: BookieWriteLedgerTest.testLedgerCreateAdvWithLedgerIdInLoop:759 » TestTimedOut
[INFO] 
Warning:  Flakes: 
Warning:  org.apache.bookkeeper.client.SlowBookieTest.testSlowBookieAndFastFailOn
Error:    Run 1: SlowBookieTest.testSlowBookieAndFastFailOn:198->doBackPressureTest:277 write error expected:<true> but was:<false>
[INFO]   Run 2: PASS

The job is re-running, but fwiw I don't remember BookieWriteLedgerTest being flaky.

@dlg99
Copy link
Contributor

dlg99 commented Oct 4, 2022

Looks like the change broke BookieWriteLedgerTest.testLedgerCreateAdvWithLedgerIdInLoop

@eolivelli
Copy link
Contributor Author

@dlg99 @shoothzj @StevenLuMT
I have fixed the test
BookieWriteLedgerTest.testLedgerCreateAdvWithLedgerIdInLoop

basically the test was writing to one ledger at a time because it was "blocking" the main zookeeper thread.

Now createLedger completes in a BK thread (pinned to the ledgerId) and not on the ZK thread, so doing a blocking call that waits for writes on the same ledger leads to a deadlock

this is an interesting behaviour change, but I believe that it is better that BK works this way, otherwise the application will execute code in the main zk thread without knowning.

cc @merlimat @michaeljmarshall @lhotari @rdhabalia @RaulGracia @fpj

@eolivelli eolivelli merged commit 2135708 into apache:master Oct 6, 2022
@eolivelli eolivelli deleted the impl/complete-out-of-metadata branch October 6, 2022 12:00
Copy link
Contributor

@StevenLuMT StevenLuMT left a comment

Choose a reason for hiding this comment

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

LGTM
Great work

hangc0276 pushed a commit to hangc0276/bookkeeper that referenced this pull request Nov 7, 2022
hangc0276 pushed a commit to hangc0276/bookkeeper that referenced this pull request Nov 7, 2022
nicoloboschi pushed a commit to datastax/bookkeeper that referenced this pull request Jan 11, 2023
…tadata callback thread (apache#3516)

(cherry picked from commit 2135708)
(cherry picked from commit ca03223)
zymap pushed a commit that referenced this pull request Feb 16, 2023
…tadata callback thread (#3516)

(cherry picked from commit 2135708)
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

7 participants