Skip to content

Commit 644b786

Browse files
committed
RATIS-2343. Replace string reasons with enum constants in log messages.
1 parent 047444b commit 644b786

File tree

3 files changed

+24
-20
lines changed

3 files changed

+24
-20
lines changed

ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java

Lines changed: 6 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -83,7 +83,7 @@
8383
import org.apache.ratis.server.impl.RetryCacheImpl.CacheEntry;
8484
import org.apache.ratis.server.impl.ServerImplUtils.ConsecutiveIndices;
8585
import org.apache.ratis.server.impl.ServerImplUtils.NavigableIndices;
86-
import org.apache.ratis.server.leader.LeaderState;
86+
import org.apache.ratis.server.leader.LeaderState.StepDownReason;
8787
import org.apache.ratis.server.metrics.LeaderElectionMetrics;
8888
import org.apache.ratis.server.metrics.RaftServerMetricsImpl;
8989
import org.apache.ratis.server.protocol.RaftServerAsynchronousProtocol;
@@ -409,7 +409,7 @@ boolean start() throws IOException {
409409
startAsPeer(RaftPeerRole.LISTENER);
410410
} else {
411411
LOG.info("{}: start with initializing state, conf={}", getMemberId(), conf);
412-
setRole(RaftPeerRole.FOLLOWER, "start");
412+
setRole(RaftPeerRole.FOLLOWER, "NOT_IN_CONF");
413413
}
414414

415415
jmxAdapter.registerMBean();
@@ -859,7 +859,7 @@ private CompletableFuture<RaftClientReply> appendTransaction(
859859
cacheEntry.failWithReply(exceptionReply);
860860
// leader will step down here
861861
if (e.leaderShouldStepDown() && getInfo().isLeader()) {
862-
leaderState.submitStepDownEvent(LeaderState.StepDownReason.STATE_MACHINE_EXCEPTION);
862+
leaderState.submitStepDownEvent(StepDownReason.STATE_MACHINE_EXCEPTION);
863863
}
864864
return CompletableFuture.completedFuture(exceptionReply);
865865
}
@@ -909,7 +909,7 @@ private RaftClientReply combineReplies(RaftClientReply reply, RaftClientReply wa
909909
}
910910

911911
void stepDownOnJvmPause() {
912-
role.getLeaderState().ifPresent(leader -> leader.submitStepDownEvent(LeaderState.StepDownReason.JVM_PAUSE));
912+
role.getLeaderState().ifPresent(leader -> leader.submitStepDownEvent(StepDownReason.JVM_PAUSE));
913913
}
914914

915915
private RaftClientRequest filterDataStreamRaftClientRequest(RaftClientRequest request)
@@ -1580,11 +1580,11 @@ leaderId, getMemberId(), currentTerm, followerCommit, state.getNextIndex(),
15801580
AppendResult.NOT_LEADER, callId, RaftLog.INVALID_LOG_INDEX, isHeartbeat));
15811581
}
15821582
try {
1583-
future = changeToFollowerAndPersistMetadata(leaderTerm, true, "appendEntries");
1583+
future = changeToFollowerAndPersistMetadata(leaderTerm, true, Op.APPEND_ENTRIES);
15841584
} catch (IOException e) {
15851585
return JavaUtils.completeExceptionally(e);
15861586
}
1587-
state.setLeader(leaderId, "appendEntries");
1587+
state.setLeader(leaderId, Op.APPEND_ENTRIES);
15881588

15891589
if (!proto.getInitializing() && lifeCycle.compareAndTransition(State.STARTING, State.RUNNING)) {
15901590
role.startFollowerState(this, Op.APPEND_ENTRIES);

ratis-server/src/main/java/org/apache/ratis/server/impl/ServerProtoUtils.java

Lines changed: 5 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -178,8 +178,12 @@ static ServerRpcProto toServerRpcProto(RaftPeer peer, long delay) {
178178
// if no peer information return empty
179179
return ServerRpcProto.getDefaultInstance();
180180
}
181+
return toServerRpcProto(peer.getRaftPeerProto(), delay);
182+
}
183+
184+
static ServerRpcProto toServerRpcProto(RaftPeerProto peer, long delay) {
181185
return ServerRpcProto.newBuilder()
182-
.setId(peer.getRaftPeerProto())
186+
.setId(peer)
183187
.setLastRpcElapsedTimeMs(delay)
184188
.build();
185189
}

ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java

Lines changed: 13 additions & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -29,10 +29,10 @@
2929
import org.apache.ratis.proto.RaftProtos.ServerRpcProto;
3030
import org.apache.ratis.protocol.RaftGroupId;
3131
import org.apache.ratis.protocol.RaftGroupMemberId;
32-
import org.apache.ratis.protocol.RaftPeer;
3332
import org.apache.ratis.protocol.RaftPeerId;
3433
import org.apache.ratis.server.RaftServerConfigKeys;
35-
import org.apache.ratis.server.protocol.RaftServerProtocol;
34+
import org.apache.ratis.server.impl.FollowerState.UpdateType;
35+
import org.apache.ratis.server.protocol.RaftServerProtocol.Op;
3636
import org.apache.ratis.server.protocol.TermIndex;
3737
import org.apache.ratis.server.raftlog.LogProtoUtils;
3838
import org.apache.ratis.server.util.ServerStringUtils;
@@ -173,16 +173,16 @@ private CompletableFuture<InstallSnapshotReplyProto> checkAndInstallSnapshot(Ins
173173
final long lastIncludedIndex = lastIncluded.getIndex();
174174
final CompletableFuture<Void> future;
175175
synchronized (server) {
176-
final boolean recognized = state.recognizeLeader(RaftServerProtocol.Op.INSTALL_SNAPSHOT, leaderId, leaderTerm);
176+
final boolean recognized = state.recognizeLeader(Op.INSTALL_SNAPSHOT, leaderId, leaderTerm);
177177
currentTerm = state.getCurrentTerm();
178178
if (!recognized) {
179179
return CompletableFuture.completedFuture(toInstallSnapshotReplyProto(leaderId, getMemberId(),
180180
currentTerm, snapshotChunkRequest.getRequestIndex(), InstallSnapshotResult.NOT_LEADER));
181181
}
182-
future = server.changeToFollowerAndPersistMetadata(leaderTerm, true, "installSnapshot");
183-
state.setLeader(leaderId, "installSnapshot");
182+
future = server.changeToFollowerAndPersistMetadata(leaderTerm, true, Op.INSTALL_SNAPSHOT);
183+
state.setLeader(leaderId, Op.INSTALL_SNAPSHOT);
184184

185-
server.updateLastRpcTime(FollowerState.UpdateType.INSTALL_SNAPSHOT_START);
185+
server.updateLastRpcTime(UpdateType.INSTALL_SNAPSHOT_START);
186186
long callId = chunk0CallId.get();
187187
// 1. leaderTerm < currentTerm will never come here
188188
// 2. leaderTerm == currentTerm && callId == request.getCallId()
@@ -229,7 +229,7 @@ private CompletableFuture<InstallSnapshotReplyProto> checkAndInstallSnapshot(Ins
229229
chunk0CallId.set(-1);
230230
}
231231
} finally {
232-
server.updateLastRpcTime(FollowerState.UpdateType.INSTALL_SNAPSHOT_COMPLETE);
232+
server.updateLastRpcTime(UpdateType.INSTALL_SNAPSHOT_COMPLETE);
233233
}
234234
}
235235
if (snapshotChunkRequest.getDone()) {
@@ -249,15 +249,15 @@ private CompletableFuture<InstallSnapshotReplyProto> notifyStateMachineToInstall
249249
final long firstAvailableLogIndex = firstAvailableLogTermIndex.getIndex();
250250
final CompletableFuture<Void> future;
251251
synchronized (server) {
252-
final boolean recognized = state.recognizeLeader("notifyInstallSnapshot", leaderId, leaderTerm);
252+
final boolean recognized = state.recognizeLeader(UpdateType.INSTALL_SNAPSHOT_NOTIFICATION, leaderId, leaderTerm);
253253
currentTerm = state.getCurrentTerm();
254254
if (!recognized) {
255255
return CompletableFuture.completedFuture(toInstallSnapshotReplyProto(leaderId, getMemberId(),
256256
currentTerm, InstallSnapshotResult.NOT_LEADER));
257257
}
258-
future = server.changeToFollowerAndPersistMetadata(leaderTerm, true, "installSnapshot");
259-
state.setLeader(leaderId, "installSnapshot");
260-
server.updateLastRpcTime(FollowerState.UpdateType.INSTALL_SNAPSHOT_NOTIFICATION);
258+
future = server.changeToFollowerAndPersistMetadata(leaderTerm, true, UpdateType.INSTALL_SNAPSHOT_NOTIFICATION);
259+
state.setLeader(leaderId, UpdateType.INSTALL_SNAPSHOT_NOTIFICATION);
260+
server.updateLastRpcTime(UpdateType.INSTALL_SNAPSHOT_NOTIFICATION);
261261

262262
if (inProgressInstallSnapshotIndex.compareAndSet(INVALID_LOG_INDEX, firstAvailableLogIndex)) {
263263
LOG.info("{}: Received notification to install snapshot at index {}", getMemberId(), firstAvailableLogIndex);
@@ -292,7 +292,7 @@ private CompletableFuture<InstallSnapshotReplyProto> notifyStateMachineToInstall
292292
// For the cases where RaftConf is empty on newly started peer with empty peer list,
293293
// we retrieve leader info from installSnapShotRequestProto.
294294
final RoleInfoProto proto = leaderProto == null || server.getRaftConf().getPeer(state.getLeaderId()) != null?
295-
server.getRoleInfoProto(): getRoleInfoProto(ProtoUtils.toRaftPeer(leaderProto));
295+
server.getRoleInfoProto(): getRoleInfoProto(leaderProto);
296296
// This is the first installSnapshot notify request for this term and
297297
// index. Notify the state machine to install the snapshot.
298298
LOG.info("{}: notifyInstallSnapshot: nextIndex is {} but the leader's first available index is {}.",
@@ -386,7 +386,7 @@ private CompletableFuture<InstallSnapshotReplyProto> notifyStateMachineToInstall
386386
}
387387
}
388388

389-
private RoleInfoProto getRoleInfoProto(RaftPeer leader) {
389+
private RoleInfoProto getRoleInfoProto(RaftPeerProto leader) {
390390
final RoleInfo role = server.getRole();
391391
final Optional<FollowerState> fs = role.getFollowerState();
392392
final ServerRpcProto leaderInfo = toServerRpcProto(leader,

0 commit comments

Comments
 (0)