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

[ISSUE #85] fix preferred leader bug #106

Merged
merged 1 commit into from
Mar 22, 2022
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -88,6 +88,7 @@ public class DLedgerConfig {
private boolean isEnableBatchPush = false;
private int maxBatchPushSize = 4 * 1024;

private long leadershipTransferWaitTimeout = 1000;

public String getDefaultPath() {
return storeBaseDir + File.separator + "dledger-" + selfId;
Expand Down Expand Up @@ -398,4 +399,12 @@ public int getMaxBatchPushSize() {
public void setMaxBatchPushSize(int maxBatchPushSize) {
this.maxBatchPushSize = maxBatchPushSize;
}

public long getLeadershipTransferWaitTimeout() {
return leadershipTransferWaitTimeout;
}

public void setLeadershipTransferWaitTimeout(long leadershipTransferWaitTimeout) {
this.leadershipTransferWaitTimeout = leadershipTransferWaitTimeout;
}
}
19 changes: 19 additions & 0 deletions src/main/java/io/openmessaging/storage/dledger/DLedgerServer.java
Original file line number Diff line number Diff line change
Expand Up @@ -297,6 +297,25 @@ public CompletableFuture<LeadershipTransferResponse> handleLeadershipTransfer(Le
// It's the transferee received the take leadership command.
PreConditions.check(request.getTransferId().equals(memberState.getLeaderId()), DLedgerResponseCode.INCONSISTENT_LEADER, "transfer=%s is not leader", request.getTransferId());

long costTime = 0L;
long startTime = System.currentTimeMillis();
long fallBehind = request.getTakeLeadershipLedgerIndex() - memberState.getLedgerEndIndex();

while (fallBehind > 0) {

if (costTime > dLedgerConfig.getLeadershipTransferWaitTimeout()) {
throw new DLedgerException(DLedgerResponseCode.TAKE_LEADERSHIP_FAILED,
"transferee fall behind, wait timeout. timeout = {}, diff = {}",
dLedgerConfig.getLeadershipTransferWaitTimeout(), fallBehind);
}

logger.warn("transferee fall behind, diff = {}", fallBehind);
Thread.sleep(10);

fallBehind = request.getTakeLeadershipLedgerIndex() - memberState.getLedgerEndIndex();
costTime = System.currentTimeMillis() - startTime;
}

return dLedgerLeaderElector.handleTakeLeadership(request);
} else {
return CompletableFuture.completedFuture(new LeadershipTransferResponse().term(memberState.currTerm()).code(DLedgerResponseCode.UNEXPECTED_ARGUMENT.getCode()));
Expand Down