Skip to content

[fix][ml] Avoid blocking the factory-shutdown thread on serial managed-ledger close#26063

Open
merlimat wants to merge 1 commit into
apache:masterfrom
merlimat:mmerli/fix-mledger-factory-shutdown-blocking
Open

[fix][ml] Avoid blocking the factory-shutdown thread on serial managed-ledger close#26063
merlimat wants to merge 1 commit into
apache:masterfrom
merlimat:mmerli/fix-mledger-factory-shutdown-blocking

Conversation

@merlimat

Copy link
Copy Markdown
Contributor

Motivation

ManagedLedgerFactoryImpl.shutdownAsync first initiates an async close (asyncClose) for every managed ledger known at the start of shutdown. A second pass, inside bookkeeperFuture.thenRun(...), then closes any ledgers that were added after that snapshot — but it did so with a serial, blocking managedLedger.close() (which waits on a CountDownLatch), blocking the thread that completes the shutdown future.

Modifications

Change that second pass from thenRun to thenCompose, and replace the blocking close() with asyncClose(), collecting the per-ledger futures and awaiting them with Futures.waitForAll. In the common case this pass is empty (the first pass has already drained the ledgers map), so it is a no-op; when ledgers were added concurrently during shutdown they now close in parallel without blocking the continuation thread. The first pass and the rest of the shutdown sequence are unchanged.

Verifying this change

Covered by ManagedLedgerFactoryShutdownTest.openEncounteredShutdown (shutdown while a ledger open is in progress) — passes.

Does this pull request potentially affect one of the following parts:

If the box was checked, please highlight the changes

  • Dependencies (add or upgrade a dependency)
  • The public API
  • The schema
  • The default values of configurations
  • The threading model
  • The binary protocol
  • The REST endpoints
  • The admin CLI options
  • The metrics
  • Anything that affects deployment

…d-ledger close

ManagedLedgerFactoryImpl.shutdownAsync first initiates an async close
(asyncClose) for every managed ledger known at the start of shutdown. A second
pass, inside bookkeeperFuture.thenRun(...), then closes any ledgers that were
added after that snapshot -- but it did so with a serial, blocking
managedLedger.close() (which waits on a CountDownLatch), blocking the thread that
completes the shutdown future.

Change that second pass from thenRun to thenCompose, and replace the blocking
close() with asyncClose(), collecting the per-ledger futures and awaiting them
with Futures.waitForAll. In the common case this pass is empty (the first pass
has already drained the ledgers map), so it is a no-op; when ledgers were added
concurrently during shutdown they now close in parallel without blocking the
continuation thread. The first pass and the rest of the shutdown sequence are
unchanged.
}, null);
}
}));
return Futures.waitForAll(remainingFutures);

Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Previously this path used managedLedger.close(), which bounded the wait with AsyncOperationTimeoutSeconds before continuing shutdown. With the new asyncClose() + waitForAll flow, do we still want to keep a bounded wait here?

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

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

2 participants