-
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
[bugfix] ManagedLedger: move to FENCED state in case of BadVersionException #17736
Changes from 1 commit
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -431,6 +431,7 @@ public void operationComplete(ManagedLedgerInfo mlInfo, Stat stat) { | |
|
||
@Override | ||
public void operationFailed(MetaStoreException e) { | ||
handleBadVersion(e); | ||
if (e instanceof MetadataNotFoundException) { | ||
callback.initializeFailed(new ManagedLedgerNotFoundException(e)); | ||
} else { | ||
|
@@ -481,6 +482,7 @@ public void operationComplete(Void v, Stat stat) { | |
|
||
@Override | ||
public void operationFailed(MetaStoreException e) { | ||
handleBadVersion(e); | ||
callback.initializeFailed(new ManagedLedgerException(e)); | ||
} | ||
}; | ||
|
@@ -1022,6 +1024,7 @@ public void operationComplete(Void result, Stat stat) { | |
|
||
@Override | ||
public void operationFailed(MetaStoreException e) { | ||
handleBadVersion(e); | ||
callback.deleteCursorFailed(e, ctx); | ||
} | ||
|
||
|
@@ -1312,6 +1315,7 @@ public void operationComplete(Void result, Stat stat) { | |
@Override | ||
public void operationFailed(MetaStoreException e) { | ||
log.error("[{}] Failed to terminate managed ledger: {}", name, e.getMessage()); | ||
handleBadVersion(e); | ||
callback.terminateFailed(new ManagedLedgerException(e), ctx); | ||
} | ||
}); | ||
|
@@ -1396,6 +1400,7 @@ public void closeFailed(ManagedLedgerException exception, Object ctx) { | |
public synchronized void asyncClose(final CloseCallback callback, final Object ctx) { | ||
State state = STATE_UPDATER.get(this); | ||
if (state == State.Fenced) { | ||
cancelScheduledTasks(); | ||
factory.close(this); | ||
callback.closeFailed(new ManagedLedgerFencedException(), ctx); | ||
return; | ||
|
@@ -1519,6 +1524,7 @@ public void operationComplete(Void v, Stat stat) { | |
@Override | ||
public void operationFailed(MetaStoreException e) { | ||
log.warn("[{}] Error updating meta data with the new list of ledgers: {}", name, e.getMessage()); | ||
handleBadVersion(e); | ||
mbean.startDataLedgerDeleteOp(); | ||
bookKeeper.asyncDeleteLedger(lh.getId(), (rc1, ctx1) -> { | ||
mbean.endDataLedgerDeleteOp(); | ||
|
@@ -1527,14 +1533,12 @@ public void operationFailed(MetaStoreException e) { | |
BKException.getMessage(rc1)); | ||
} | ||
}, null); | ||
|
||
if (e instanceof BadVersionException) { | ||
synchronized (ManagedLedgerImpl.this) { | ||
log.error( | ||
"[{}] Failed to update ledger list. z-node version mismatch. Closing managed ledger", | ||
name); | ||
lastLedgerCreationFailureTimestamp = clock.millis(); | ||
STATE_UPDATER.set(ManagedLedgerImpl.this, State.Fenced); | ||
// Return ManagedLedgerFencedException to addFailed callback | ||
// to indicate that the ledger is now fenced and topic needs to be closed | ||
clearPendingAddEntries(new ManagedLedgerFencedException(e)); | ||
|
@@ -1557,6 +1561,12 @@ public void operationFailed(MetaStoreException e) { | |
updateLedgersListAfterRollover(cb, newLedger); | ||
} | ||
} | ||
|
||
private void handleBadVersion(Throwable e) { | ||
if (e instanceof BadVersionException) { | ||
setFenced(); | ||
} | ||
} | ||
private void updateLedgersListAfterRollover(MetaStoreCallback<Void> callback, LedgerInfo newLedger) { | ||
if (!metadataMutex.tryLock()) { | ||
// Defer update for later | ||
|
@@ -2463,12 +2473,19 @@ void internalTrimLedgers(boolean isTruncate, CompletableFuture<?> promise) { | |
log.debug("[{}] Start TrimConsumedLedgers. ledgers={} totalSize={}", name, ledgers.keySet(), | ||
TOTAL_SIZE_UPDATER.get(this)); | ||
} | ||
if (STATE_UPDATER.get(this) == State.Closed) { | ||
State currentState = STATE_UPDATER.get(this); | ||
if (currentState == State.Closed) { | ||
log.debug("[{}] Ignoring trimming request since the managed ledger was already closed", name); | ||
trimmerMutex.unlock(); | ||
promise.completeExceptionally(new ManagedLedgerAlreadyClosedException("Can't trim closed ledger")); | ||
return; | ||
} | ||
if (currentState == State.Fenced) { | ||
log.debug("[{}] Ignoring trimming request since the managed ledger was already fenced", name); | ||
trimmerMutex.unlock(); | ||
promise.completeExceptionally(new ManagedLedgerFencedException("Can't trim fenced ledger")); | ||
return; | ||
} | ||
|
||
long slowestReaderLedgerId = -1; | ||
if (!cursors.hasDurableCursors()) { | ||
|
@@ -2557,7 +2574,7 @@ void internalTrimLedgers(boolean isTruncate, CompletableFuture<?> promise) { | |
return; | ||
} | ||
|
||
if (STATE_UPDATER.get(this) == State.CreatingLedger // Give up now and schedule a new trimming | ||
if (currentState == State.CreatingLedger // Give up now and schedule a new trimming | ||
|| !metadataMutex.tryLock()) { // Avoid deadlocks with other operations updating the ledgers list | ||
scheduleDeferredTrimming(isTruncate, promise); | ||
trimmerMutex.unlock(); | ||
|
@@ -2624,6 +2641,7 @@ public void operationFailed(MetaStoreException e) { | |
log.warn("[{}] Failed to update the list of ledgers after trimming", name, e); | ||
metadataMutex.unlock(); | ||
trimmerMutex.unlock(); | ||
handleBadVersion(e); | ||
|
||
promise.completeExceptionally(e); | ||
} | ||
|
@@ -2708,7 +2726,7 @@ public void deleteLedgerFailed(ManagedLedgerException e, Object ctx) { | |
public void asyncDelete(final DeleteLedgerCallback callback, final Object ctx) { | ||
// Delete the managed ledger without closing, since we are not interested in gracefully closing cursors and | ||
// ledgers | ||
STATE_UPDATER.set(this, State.Fenced); | ||
setFenced(); | ||
cancelScheduledTasks(); | ||
|
||
List<ManagedCursor> cursors = Lists.newArrayList(this.cursors); | ||
|
@@ -2957,7 +2975,7 @@ public void asyncOffloadPrefix(Position pos, OffloadCallback callback, Object ct | |
promise.whenComplete((result, exception) -> { | ||
offloadMutex.unlock(); | ||
if (exception != null) { | ||
callback.offloadFailed(new ManagedLedgerException(exception), ctx); | ||
callback.offloadFailed(ManagedLedgerException.getManagedLedgerException(exception), ctx); | ||
} else { | ||
callback.offloadComplete(result, ctx); | ||
} | ||
|
@@ -2971,11 +2989,17 @@ public void asyncOffloadPrefix(Position pos, OffloadCallback callback, Object ct | |
|
||
private void offloadLoop(CompletableFuture<PositionImpl> promise, Queue<LedgerInfo> ledgersToOffload, | ||
PositionImpl firstUnoffloaded, Optional<Throwable> firstError) { | ||
if (getState() == State.Closed) { | ||
State currentState = getState(); | ||
if (currentState == State.Closed) { | ||
promise.completeExceptionally(new ManagedLedgerAlreadyClosedException( | ||
String.format("managed ledger [%s] has already closed", name))); | ||
return; | ||
} | ||
if (currentState == State.Fenced) { | ||
promise.completeExceptionally(new ManagedLedgerFencedException( | ||
String.format("managed ledger [%s] is fenced", name))); | ||
return; | ||
} | ||
LedgerInfo info = ledgersToOffload.poll(); | ||
if (info == null) { | ||
if (firstError.isPresent()) { | ||
|
@@ -3117,6 +3141,7 @@ public void operationComplete(Void result, Stat stat) { | |
|
||
@Override | ||
public void operationFailed(MetaStoreException e) { | ||
handleBadVersion(e); | ||
unlockingPromise.completeExceptionally(e); | ||
} | ||
}); | ||
|
@@ -3639,6 +3664,7 @@ private void checkManagedLedgerIsOpen() throws ManagedLedgerException { | |
} | ||
|
||
synchronized void setFenced() { | ||
log.info("{} Moving to Fenced state", name); | ||
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. Do we need to change the log level to 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. This is not a "problem". we should log "WARN" or "ERROR" when there is something bad, and you have to take extra care |
||
STATE_UPDATER.set(this, State.Fenced); | ||
} | ||
|
||
|
@@ -3842,12 +3868,21 @@ private void scheduleTimeoutTask() { | |
? Math.max(config.getAddEntryTimeoutSeconds(), config.getReadEntryTimeoutSeconds()) | ||
: timeoutSec; | ||
this.timeoutTask = this.scheduledExecutor.scheduleAtFixedRate(safeRun(() -> { | ||
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. Suggest change
to |
||
checkAddTimeout(); | ||
checkReadTimeout(); | ||
checkTimeouts(); | ||
}), timeoutSec, timeoutSec, TimeUnit.SECONDS); | ||
} | ||
} | ||
|
||
private void checkTimeouts() { | ||
final State state = STATE_UPDATER.get(this); | ||
if (state == State.Closed | ||
|| state == State.Fenced) { | ||
return; | ||
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. Do we need to add a 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. there are already logs that say that we fenced or closed the topic. |
||
} | ||
checkAddTimeout(); | ||
checkReadTimeout(); | ||
} | ||
|
||
private void checkAddTimeout() { | ||
long timeoutSec = config.getAddEntryTimeoutSeconds(); | ||
if (timeoutSec < 1) { | ||
|
@@ -4004,6 +4039,7 @@ public void operationComplete(Void result, Stat version) { | |
@Override | ||
public void operationFailed(MetaStoreException e) { | ||
log.error("[{}] Update managedLedger's properties failed", name, e); | ||
handleBadVersion(e); | ||
callback.updatePropertiesFailed(e, ctx); | ||
metadataMutex.unlock(); | ||
} | ||
|
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.
Maybe we don't need to unlock the
trimmerMutex
.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 am not sure.
I prefer to keep the ML in a clean status.
I did the same it works for a "closed" ML.
it is very like to being "Closed" in this point
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.
Sorry, after checking the code, I think we should release the lock, or else the scheduled task will always try to get the lock per 100 milliseconds.