Skip to content
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 @@ -551,6 +551,12 @@ public void close() {
});
}

void setFirstElection(Object reason) {
if (firstElectionSinceStartup.compareAndSet(true, false)) {
LOG.info("{}: set firstElectionSinceStartup to false for {}", getMemberId(), reason);
}
}

/**
* Change the server state to Follower if this server is in a different role or force is true.
* @param newTerm The new term.
Expand Down Expand Up @@ -579,7 +585,7 @@ private synchronized boolean changeToFollower(
role.shutdownFollowerState();
}
role.startFollowerState(this, reason);
firstElectionSinceStartup.set(false);
setFirstElection(reason);
}
return metadataUpdated;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -69,7 +69,7 @@ class ServerState {
/** local storage for log and snapshot */
private final MemoizedCheckedSupplier<RaftStorageImpl, IOException> raftStorage;
private final SnapshotManager snapshotManager;
private volatile Timestamp lastNoLeaderTime;
private final AtomicReference<Timestamp> lastNoLeaderTime;
private final TimeDuration noLeaderTimeout;

private final ReadRequests readRequests;
Expand Down Expand Up @@ -120,7 +120,7 @@ class ServerState {
stateMachine.getStateMachineStorage());

// On start the leader is null, start the clock now
this.lastNoLeaderTime = Timestamp.currentTime();
this.lastNoLeaderTime = new AtomicReference<>(Timestamp.currentTime());
this.noLeaderTimeout = RaftServerConfigKeys.Notification.noLeaderTimeout(prop);

final LongSupplier getSnapshotIndexFromStateMachine = () -> Optional.ofNullable(stateMachine.getLatestSnapshot())
Expand Down Expand Up @@ -246,15 +246,15 @@ void grantVote(RaftPeerId candidateId) {
void setLeader(RaftPeerId newLeaderId, Object op) {
final RaftPeerId oldLeaderId = leaderId.getAndSet(newLeaderId);
if (!Objects.equals(oldLeaderId, newLeaderId)) {
String suffix;
final String suffix;
if (newLeaderId == null) {
// reset the time stamp when a null leader is assigned
lastNoLeaderTime = Timestamp.currentTime();
lastNoLeaderTime.set(Timestamp.currentTime());
suffix = "";
} else {
Timestamp previous = lastNoLeaderTime;
lastNoLeaderTime = null;
final Timestamp previous = lastNoLeaderTime.getAndSet(null);
suffix = ", leader elected after " + previous.elapsedTimeMs() + "ms";
server.setFirstElection(op);
server.getStateMachine().event().notifyLeaderChanged(getMemberId(), newLeaderId);
}
LOG.info("{}: change Leader from {} to {} at term {} for {}{}",
Expand All @@ -266,14 +266,14 @@ void setLeader(RaftPeerId newLeaderId, Object op) {
}

boolean shouldNotifyExtendedNoLeader() {
return Optional.ofNullable(lastNoLeaderTime)
return Optional.ofNullable(lastNoLeaderTime.get())
.map(Timestamp::elapsedTime)
.filter(t -> t.compareTo(noLeaderTimeout) > 0)
.isPresent();
}

long getLastLeaderElapsedTimeMs() {
return Optional.ofNullable(lastNoLeaderTime).map(Timestamp::elapsedTimeMs).orElse(0L);
return Optional.ofNullable(lastNoLeaderTime.get()).map(Timestamp::elapsedTimeMs).orElse(0L);
}

void becomeLeader() {
Expand Down