Skip to content

Conversation

dao-jun
Copy link
Member

@dao-jun dao-jun commented Jul 10, 2025

Motivation

In our pulsar cluster, we have encountered an extremely strange problem.
A topic has been running for a long time, but at a time, it starts fenced for a long time(more than 3 hours, until we unload it manually).

After we analysis the heap dump and broker log, we find the root cause:

For some reason the CAS operation in OpAddEntry.initiate() failed, but it logs initiate with unexpected state OPEN, expect OPEN state.
image
Which means the state is OPEN, but the CAS operation still fails.
Because of the CAS operation failed, the request did not send to bookies.

The next request passed the CAS operation and sent to bookies and received the response, when doing callback, it will check the ml.pendingAddEntries.
image
It polls the 1st entry, but the 1st entry did not send to bookies, so it will firstInQueue.failed(...) the 1st entry, and the topic will be fenced.
image

But the 2nd entry will always hang there, because it only fails the 1st entry but do nothing to itself. So the topic keeps fencing and never be unfenced.

The PR change maybe could causes message out of ordering, so do not merge.

Modifications

Verifying this change

  • Make sure that the change passes the CI checks.

(Please pick either of the following options)

This change is a trivial rework / code cleanup without any test coverage.

(or)

This change is already covered by existing tests, such as (please describe tests).

(or)

This change added tests and can be verified as follows:

(example:)

  • Added integration tests for end-to-end deployment with large payloads (10MB)
  • Extended integration test for recovery after broker failure

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

Documentation

  • doc
  • doc-required
  • doc-not-needed
  • doc-complete

@github-actions github-actions bot added the doc-required Your PR changes impact docs and you will update later. label Jul 10, 2025
@dao-jun
Copy link
Member Author

dao-jun commented Jul 10, 2025

More context:

  1. No new ledger created at the time for the ML, so OpAddEntry.initiate() only called in ManagedLedger.internalAsyncAddEntry
  2. The topic is fenced and PersistentTopic.pendingWriteOps is 1
    image
  3. The OpAddEntry's state = COMPLETED and entryId was set, it means it was sent to bookies and received response, but it still in the queue
    image

@dao-jun
Copy link
Member Author

dao-jun commented Jul 10, 2025

@jiazhai jiazhai marked this pull request as draft July 11, 2025 00:49
dao-jun added 2 commits July 28, 2025 18:29
# Conflicts:
#	managed-ledger/src/main/java/org/apache/bookkeeper/mledger/impl/OpAddEntry.java
@dao-jun dao-jun changed the title [DONOTMERGE][fix][ml] Fix the CAS operation in initiate() failure [fix][ml] Fix the CAS operation in initiate() failure Jul 29, 2025
@dao-jun dao-jun closed this Jul 29, 2025
@dao-jun dao-jun reopened this Jul 29, 2025
@dao-jun dao-jun marked this pull request as ready for review July 29, 2025 03:38
@dao-jun dao-jun self-assigned this Jul 29, 2025
@dao-jun dao-jun closed this Jul 29, 2025
@dao-jun dao-jun reopened this Jul 29, 2025
Comment on lines 174 to 179
if (STATE_UPDATER.compareAndSet(OpAddEntry.this, State.OPEN, State.INITIATED)) {
// Fail the add operation if the managed ledger is in a state that prevents adding entries.
ManagedLedgerException exbw = ml.interceptorException;
if (exbw != null) {
ml.pendingAddEntries.remove(this);
Copy link
Member

Choose a reason for hiding this comment

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

It looks unnecessary.

return;
}

ByteBuf duplicateBuffer = data.retainedDuplicate();
Copy link
Member

Choose a reason for hiding this comment

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

You can directly set the ManagedLedgerException.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
area/ML doc-required Your PR changes impact docs and you will update later. ready-to-test
Projects
None yet
Development

Successfully merging this pull request may close these issues.

2 participants