-
Notifications
You must be signed in to change notification settings - Fork 3.6k
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
[fix][broker] refactor cursor read entry process to fix dead loop read issue of txn #22944
Open
TakaHiR07
wants to merge
2
commits into
apache:master
Choose a base branch
from
TakaHiR07:refactor_cursor_read_entry_process
base: master
Could not load branches
Branch not found: {{ refName }}
Loading
Could not load tags
Nothing to show
Loading
Are you sure you want to change the base?
Some commits from the old base branch may be removed from the timeline,
and old review comments may become outdated.
Open
Changes from all commits
Commits
Show all changes
2 commits
Select commit
Hold shift + click to select a range
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -162,6 +162,12 @@ public class ManagedCursorImpl implements ManagedCursor { | |
@SuppressWarnings("unused") | ||
private volatile OpReadEntry waitingReadOp = null; | ||
|
||
protected static final AtomicReferenceFieldUpdater<ManagedCursorImpl, OpReadEntry> | ||
WAITING_READ_OP_UPDATER_BY_MAX_READ_POSITION = AtomicReferenceFieldUpdater | ||
.newUpdater(ManagedCursorImpl.class, OpReadEntry.class, "waitingReadOpByMaxReadPosition"); | ||
@SuppressWarnings("unused") | ||
private volatile OpReadEntry waitingReadOpByMaxReadPosition = null; | ||
|
||
public static final int FALSE = 0; | ||
public static final int TRUE = 1; | ||
private static final AtomicIntegerFieldUpdater<ManagedCursorImpl> RESET_CURSOR_IN_PROGRESS_UPDATER = | ||
|
@@ -1017,8 +1023,11 @@ public void asyncReadEntriesWithSkipOrWait(int maxEntries, long maxSizeBytes, Re | |
|
||
int numberOfEntriesToRead = applyMaxSizeCap(maxEntries, maxSizeBytes); | ||
|
||
if (hasMoreEntries() && maxPosition.compareTo(readPosition) >= 0) { | ||
boolean hasMoreEntries = hasMoreEntries(); | ||
boolean hasMoreEntriesByMaxReadPosition = hasMoreEntriesByMaxReadPosition(); | ||
if (hasMoreEntries && hasMoreEntriesByMaxReadPosition) { | ||
// If we have available entries, we can read them immediately | ||
// readPos <= lastConfirmedEntry and readPos <= maxReadPosition | ||
if (log.isDebugEnabled()) { | ||
log.debug("[{}] [{}] Read entries immediately", ledger.getName(), name); | ||
} | ||
|
@@ -1030,25 +1039,54 @@ public void asyncReadEntriesWithSkipOrWait(int maxEntries, long maxSizeBytes, Re | |
OpReadEntry op = OpReadEntry.create(this, readPosition, numberOfEntriesToRead, callback, | ||
ctx, maxPosition, skipCondition); | ||
|
||
if (!WAITING_READ_OP_UPDATER.compareAndSet(this, null, op)) { | ||
op.recycle(); | ||
callback.readEntriesFailed(new ManagedLedgerException.ConcurrentWaitCallbackException(), ctx); | ||
return; | ||
} | ||
if (hasMoreEntries) { | ||
// We have available entries, but we can't read them immediately | ||
// readPos <= lastConfirmedEntry and readPos > maxReadPosition | ||
if (!WAITING_READ_OP_UPDATER_BY_MAX_READ_POSITION.compareAndSet(this, null, op)) { | ||
op.recycle(); | ||
callback.readEntriesFailed(new ManagedLedgerException.ConcurrentWaitCallbackException(), ctx); | ||
return; | ||
} | ||
|
||
if (log.isDebugEnabled()) { | ||
log.debug("[{}] [{}] Deferring retry of read at position {}", ledger.getName(), name, op.readPosition); | ||
} | ||
if (log.isDebugEnabled()) { | ||
log.debug("[{}] [{}] Deferring retry of read at readPosition {} maxPosition {}", | ||
ledger.getName(), name, op.readPosition, op.maxPosition); | ||
} | ||
|
||
// Check again for new entries after the configured time, then if still no entries are available register | ||
// to be notified | ||
if (getConfig().getNewEntriesCheckDelayInMillis() > 0) { | ||
ledger.getScheduledExecutor() | ||
.schedule(() -> checkForNewEntries(op, callback, ctx), | ||
getConfig().getNewEntriesCheckDelayInMillis(), TimeUnit.MILLISECONDS); | ||
// Check again for new entries after the configured time, | ||
// then if still no entries are available register to be notified | ||
if (getConfig().getNewEntriesCheckDelayInMillis() > 0) { | ||
ledger.getScheduledExecutor() | ||
.schedule(() -> checkForNewEntriesByMaxReadPosition(op, callback, ctx), | ||
getConfig().getNewEntriesCheckDelayInMillis(), TimeUnit.MILLISECONDS); | ||
} else { | ||
// If there's no delay, check directly from the same thread | ||
checkForNewEntriesByMaxReadPosition(op, callback, ctx); | ||
} | ||
} else { | ||
// If there's no delay, check directly from the same thread | ||
checkForNewEntries(op, callback, ctx); | ||
// do not have available entries | ||
// readPos > lastConfirmedEntry and readPos > maxReadPosition | ||
if (!WAITING_READ_OP_UPDATER.compareAndSet(this, null, op)) { | ||
op.recycle(); | ||
callback.readEntriesFailed(new ManagedLedgerException.ConcurrentWaitCallbackException(), ctx); | ||
return; | ||
} | ||
|
||
if (log.isDebugEnabled()) { | ||
log.debug("[{}] [{}] Deferring retry of read at position {}", | ||
ledger.getName(), name, op.readPosition); | ||
} | ||
|
||
// Check again for new entries after the configured time, | ||
// then if still no entries are available register to be notified | ||
if (getConfig().getNewEntriesCheckDelayInMillis() > 0) { | ||
ledger.getScheduledExecutor() | ||
.schedule(() -> checkForNewEntries(op, callback, ctx), | ||
getConfig().getNewEntriesCheckDelayInMillis(), TimeUnit.MILLISECONDS); | ||
} else { | ||
// If there's no delay, check directly from the same thread | ||
checkForNewEntries(op, callback, ctx); | ||
} | ||
} | ||
} | ||
} | ||
|
@@ -1108,6 +1146,63 @@ private void checkForNewEntries(OpReadEntry op, ReadEntriesCallback callback, Ob | |
} | ||
} | ||
|
||
private void checkForNewEntriesByMaxReadPosition(OpReadEntry op, ReadEntriesCallback callback, Object ctx) { | ||
try { | ||
if (log.isDebugEnabled()) { | ||
log.debug("[{}] [{}] Re-trying the read at readPosition {} maxPosition {}", | ||
ledger.getName(), name, op.readPosition, op.maxPosition); | ||
} | ||
|
||
if (isClosed()) { | ||
callback.readEntriesFailed(new CursorAlreadyClosedException("Cursor was already closed"), ctx); | ||
return; | ||
} | ||
|
||
if (!hasMoreEntriesByMaxReadPosition()) { | ||
if (log.isDebugEnabled()) { | ||
log.debug("[{}] [{}] Still no entries available by maxReadPosition. Register for notification", | ||
ledger.getName(), name); | ||
} | ||
ledger.addWaitingCursorByMaxReadPosition(this); | ||
} else { | ||
if (log.isDebugEnabled()) { | ||
log.debug("[{}] [{}] Skip notification registering since we do have entries " | ||
+ "available by maxReadPosition", | ||
ledger.getName(), name); | ||
} | ||
} | ||
|
||
// Check again the entries count, since maxReadPosition may be changed between the time we | ||
// checked and the time we've asked to be notified by managed ledger | ||
if (hasMoreEntriesByMaxReadPosition()) { | ||
if (log.isDebugEnabled()) { | ||
log.debug("[{}] [{}] Found more entries by maxReadPosition", ledger.getName(), name); | ||
} | ||
// Try to cancel the notification request | ||
if (WAITING_READ_OP_UPDATER_BY_MAX_READ_POSITION.compareAndSet(this, op, null)) { | ||
if (log.isDebugEnabled()) { | ||
log.debug("[{}] [{}] Cancelled notification by maxReadPosition and scheduled read at {}", | ||
ledger.getName(), name, op.readPosition); | ||
} | ||
PENDING_READ_OPS_UPDATER.incrementAndGet(this); | ||
ledger.asyncReadEntries(op); | ||
} else { | ||
if (log.isDebugEnabled()) { | ||
log.debug("[{}] [{}] notification by maxReadPosition was already cancelled", | ||
ledger.getName(), name); | ||
} | ||
} | ||
} else if (ledger.isTerminated()) { | ||
// At this point we registered for notification and still there were no more available | ||
// entries by maxReadPosition. | ||
// If the managed ledger was indeed terminated, we need to notify the cursor | ||
callback.readEntriesFailed(new NoMoreEntriesToReadException("Topic was terminated"), ctx); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. need to release ReadEntryOp? |
||
} | ||
} catch (Throwable t) { | ||
callback.readEntriesFailed(new ManagedLedgerException(t), ctx); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. need to release ReadEntryOp? |
||
} | ||
} | ||
|
||
@Override | ||
public boolean isClosed() { | ||
return state == State.Closed || state == State.Closing; | ||
|
@@ -1119,14 +1214,21 @@ public boolean cancelPendingReadRequest() { | |
log.debug("[{}] [{}] Cancel pending read request", ledger.getName(), name); | ||
} | ||
final OpReadEntry op = WAITING_READ_OP_UPDATER.getAndSet(this, null); | ||
final OpReadEntry opByMaxReadPosition = | ||
WAITING_READ_OP_UPDATER_BY_MAX_READ_POSITION.getAndSet(this, null); | ||
if (op != null) { | ||
op.recycle(); | ||
} | ||
return op != null; | ||
if (opByMaxReadPosition != null) { | ||
opByMaxReadPosition.recycle(); | ||
} | ||
// use || here because sometimes only exist one of the two waitingReadOp | ||
return op != null || opByMaxReadPosition != null; | ||
} | ||
|
||
public boolean hasPendingReadRequest() { | ||
return WAITING_READ_OP_UPDATER.get(this) != null; | ||
return WAITING_READ_OP_UPDATER.get(this) != null | ||
|| WAITING_READ_OP_UPDATER_BY_MAX_READ_POSITION.get(this) != null; | ||
} | ||
|
||
@Override | ||
|
@@ -1148,6 +1250,22 @@ public boolean hasMoreEntries() { | |
} | ||
} | ||
|
||
private boolean hasMoreEntriesByMaxReadPosition() { | ||
Position maxReadPosition = ledger.getMaxReadPosition(); | ||
if (maxReadPosition.getEntryId() != -1) { | ||
return readPosition.compareTo(maxReadPosition) <= 0; | ||
} else { | ||
// Fall back to checking the number of entries to ensure we are at the last entry in ledger and no ledgers | ||
// are in the middle | ||
Position maxReadPositionNext = PositionFactory.create(maxReadPosition.getLedgerId(), 0); | ||
if (readPosition.compareTo(maxReadPositionNext) > 0) { | ||
return false; | ||
} else { | ||
return getNumberOfEntries(Range.closedOpen(readPosition, maxReadPositionNext)) > 0; | ||
} | ||
} | ||
} | ||
|
||
@Override | ||
public long getNumberOfEntries() { | ||
if (readPosition.compareTo(ledger.getLastPosition().getNext()) > 0) { | ||
|
@@ -3441,6 +3559,40 @@ void notifyEntriesAvailable() { | |
} | ||
} | ||
|
||
/** | ||
* | ||
* @return Whether the cursor responded to the notification | ||
*/ | ||
void notifyEntriesAvailableByMaxReadPosition() { | ||
if (log.isDebugEnabled()) { | ||
log.debug("[{}] [{}] Received ml notification by maxReadPosition", ledger.getName(), name); | ||
} | ||
OpReadEntry opReadEntry = WAITING_READ_OP_UPDATER_BY_MAX_READ_POSITION.getAndSet(this, null); | ||
|
||
if (opReadEntry != null) { | ||
if (log.isDebugEnabled()) { | ||
log.debug("[{}] [{}] Received notification of maxReadPosition change, reading at {} -- mPos: {}", | ||
ledger.getName(), name, opReadEntry.readPosition, ledger.maxReadPosition); | ||
log.debug("[{}] Consumer {} cursor notification: other counters: consumed {} mdPos {} rdPos {}", | ||
ledger.getName(), name, messagesConsumedCounter, markDeletePosition, readPosition); | ||
} | ||
|
||
PENDING_READ_OPS_UPDATER.incrementAndGet(this); | ||
opReadEntry.readPosition = getReadPosition(); | ||
// Only ml.maxReadPosition change would trigger notifyEntriesAvailableByMaxReadPosition. | ||
// Also update op.maxPosition since ml.maxReadPosition has been changed. | ||
// This can avoid that opReadEntry directly finish and request readEntry once again | ||
opReadEntry.maxPosition = ledger.maxReadPosition; | ||
ledger.asyncReadEntries(opReadEntry); | ||
} else { | ||
// No one is waiting to be notified. Ignore | ||
if (log.isDebugEnabled()) { | ||
log.debug("[{}] [{}] Received notification by maxReadPosition but had no pending read operation", | ||
ledger.getName(), name); | ||
} | ||
} | ||
} | ||
|
||
void asyncCloseCursorLedger(final AsyncCallbacks.CloseCallback callback, final Object ctx) { | ||
LedgerHandle lh = cursorLedger; | ||
ledger.mbean.startCursorLedgerCloseOp(); | ||
|
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
need to release ReadEntryOp here?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think so. need to recycle the op first. But this is better fixed by another pr.