This is an automated email from the ASF dual-hosted git repository.
szetszwo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ratis.git
The following commit(s) were added to refs/heads/master by this push:
new 0c2b3fa2b RATIS-2343. Replace string reasons with enum constants in
log messages. (#1298)
0c2b3fa2b is described below
commit 0c2b3fa2b9793a5c6795f115cccc328ccadd713e
Author: Tsz-Wo Nicholas Sze <[email protected]>
AuthorDate: Thu Oct 9 19:21:27 2025 -0700
RATIS-2343. Replace string reasons with enum constants in log messages.
(#1298)
---
.../apache/ratis/server/impl/RaftServerImpl.java | 19 ++++++++--------
.../apache/ratis/server/impl/ServerProtoUtils.java | 6 ++++-
.../org/apache/ratis/server/impl/ServerState.java | 9 ++++----
.../server/impl/SnapshotInstallationHandler.java | 26 +++++++++++-----------
4 files changed, 33 insertions(+), 27 deletions(-)
diff --git
a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java
b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java
index 3c10e103b..689bb8cef 100644
---
a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java
+++
b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java
@@ -83,7 +83,7 @@ import org.apache.ratis.server.impl.LeaderElection.Phase;
import org.apache.ratis.server.impl.RetryCacheImpl.CacheEntry;
import org.apache.ratis.server.impl.ServerImplUtils.ConsecutiveIndices;
import org.apache.ratis.server.impl.ServerImplUtils.NavigableIndices;
-import org.apache.ratis.server.leader.LeaderState;
+import org.apache.ratis.server.leader.LeaderState.StepDownReason;
import org.apache.ratis.server.metrics.LeaderElectionMetrics;
import org.apache.ratis.server.metrics.RaftServerMetricsImpl;
import org.apache.ratis.server.protocol.RaftServerAsynchronousProtocol;
@@ -139,6 +139,7 @@ import java.util.function.Supplier;
import java.util.stream.Collectors;
import java.util.stream.Stream;
+import static org.apache.ratis.server.impl.LeaderElection.Result.NOT_IN_CONF;
import static org.apache.ratis.server.impl.ServerImplUtils.assertEntries;
import static org.apache.ratis.server.impl.ServerImplUtils.assertGroup;
import static
org.apache.ratis.server.impl.ServerImplUtils.effectiveCommitIndex;
@@ -409,7 +410,7 @@ class RaftServerImpl implements RaftServer.Division,
startAsPeer(RaftPeerRole.LISTENER);
} else {
LOG.info("{}: start with initializing state, conf={}", getMemberId(),
conf);
- setRole(RaftPeerRole.FOLLOWER, "start");
+ setRole(RaftPeerRole.FOLLOWER, NOT_IN_CONF);
}
jmxAdapter.registerMBean();
@@ -560,12 +561,12 @@ class RaftServerImpl implements RaftServer.Division,
try {
ConcurrentUtils.shutdownAndWait(clientExecutor);
} catch (Exception e) {
- LOG.warn(getMemberId() + ": Failed to shutdown clientExecutor", e);
+ LOG.warn("{}: Failed to shutdown clientExecutor", getMemberId(), e);
}
try {
ConcurrentUtils.shutdownAndWait(serverExecutor);
} catch (Exception e) {
- LOG.warn(getMemberId() + ": Failed to shutdown serverExecutor", e);
+ LOG.warn("{}: Failed to shutdown serverExecutor", getMemberId(), e);
}
closeFinishedLatch.countDown();
});
@@ -859,7 +860,7 @@ class RaftServerImpl implements RaftServer.Division,
cacheEntry.failWithReply(exceptionReply);
// leader will step down here
if (e.leaderShouldStepDown() && getInfo().isLeader()) {
-
leaderState.submitStepDownEvent(LeaderState.StepDownReason.STATE_MACHINE_EXCEPTION);
+
leaderState.submitStepDownEvent(StepDownReason.STATE_MACHINE_EXCEPTION);
}
return CompletableFuture.completedFuture(exceptionReply);
}
@@ -909,7 +910,7 @@ class RaftServerImpl implements RaftServer.Division,
}
void stepDownOnJvmPause() {
- role.getLeaderState().ifPresent(leader ->
leader.submitStepDownEvent(LeaderState.StepDownReason.JVM_PAUSE));
+ role.getLeaderState().ifPresent(leader ->
leader.submitStepDownEvent(StepDownReason.JVM_PAUSE));
}
private RaftClientRequest
filterDataStreamRaftClientRequest(RaftClientRequest request)
@@ -1529,7 +1530,7 @@ class RaftServerImpl implements RaftServer.Division,
static void logAppendEntries(boolean isHeartbeat, Supplier<String> message) {
if (isHeartbeat) {
if (LOG.isTraceEnabled()) {
- LOG.trace("HEARTBEAT: " + message.get());
+ LOG.trace("HEARTBEAT: {}", message.get());
}
} else {
if (LOG.isDebugEnabled()) {
@@ -1580,11 +1581,11 @@ class RaftServerImpl implements RaftServer.Division,
AppendResult.NOT_LEADER, callId, RaftLog.INVALID_LOG_INDEX,
isHeartbeat));
}
try {
- future = changeToFollowerAndPersistMetadata(leaderTerm, true,
"appendEntries");
+ future = changeToFollowerAndPersistMetadata(leaderTerm, true,
Op.APPEND_ENTRIES);
} catch (IOException e) {
return JavaUtils.completeExceptionally(e);
}
- state.setLeader(leaderId, "appendEntries");
+ state.setLeader(leaderId, Op.APPEND_ENTRIES);
if (!proto.getInitializing() &&
lifeCycle.compareAndTransition(State.STARTING, State.RUNNING)) {
role.startFollowerState(this, Op.APPEND_ENTRIES);
diff --git
a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerProtoUtils.java
b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerProtoUtils.java
index e6a29189a..f491aaee2 100644
---
a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerProtoUtils.java
+++
b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerProtoUtils.java
@@ -178,8 +178,12 @@ final class ServerProtoUtils {
// if no peer information return empty
return ServerRpcProto.getDefaultInstance();
}
+ return toServerRpcProto(peer.getRaftPeerProto(), delay);
+ }
+
+ static ServerRpcProto toServerRpcProto(RaftPeerProto peer, long delay) {
return ServerRpcProto.newBuilder()
- .setId(peer.getRaftPeerProto())
+ .setId(peer)
.setLastRpcElapsedTimeMs(delay)
.build();
}
diff --git
a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java
b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java
index dc05c63e3..05afc0975 100644
--- a/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java
+++ b/ratis-server/src/main/java/org/apache/ratis/server/impl/ServerState.java
@@ -24,6 +24,7 @@ import
org.apache.ratis.protocol.exceptions.StateMachineException;
import org.apache.ratis.server.RaftConfiguration;
import org.apache.ratis.server.RaftServerConfigKeys;
import org.apache.ratis.server.impl.LeaderElection.Phase;
+import org.apache.ratis.server.protocol.RaftServerProtocol.Op;
import org.apache.ratis.server.protocol.TermIndex;
import org.apache.ratis.server.raftlog.LogProtoUtils;
import org.apache.ratis.server.raftlog.RaftLog;
@@ -257,7 +258,7 @@ class ServerState {
*/
void grantVote(RaftPeerId candidateId) {
votedFor = candidateId;
- setLeader(null, "grantVote");
+ setLeader(null, Op.REQUEST_VOTE);
}
void setLeader(RaftPeerId newLeaderId, Object op) {
@@ -431,7 +432,7 @@ class ServerState {
if (e instanceof InterruptedException) {
Thread.currentThread().interrupt();
}
- LOG.warn(getMemberId() + ": Failed to join " + getStateMachineUpdater(),
e);
+ LOG.warn("{}: Failed to join {}", getMemberId(),
getStateMachineUpdater(), e);
}
try {
@@ -439,7 +440,7 @@ class ServerState {
getLog().close();
}
} catch (Throwable e) {
- LOG.warn(getMemberId() + ": Failed to close raft log " + getLog(), e);
+ LOG.warn("{}: Failed to close raft log {}", getMemberId(), getLog(), e);
}
try {
@@ -447,7 +448,7 @@ class ServerState {
getStorage().close();
}
} catch (Throwable e) {
- LOG.warn(getMemberId() + ": Failed to close raft storage " +
getStorage(), e);
+ LOG.warn("{}: Failed to close raft storage {}", getMemberId(),
getStorage(), e);
}
}
diff --git
a/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java
b/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java
index 2a2e9e2b5..faefaaa38 100644
---
a/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java
+++
b/ratis-server/src/main/java/org/apache/ratis/server/impl/SnapshotInstallationHandler.java
@@ -29,10 +29,10 @@ import org.apache.ratis.proto.RaftProtos.RoleInfoProto;
import org.apache.ratis.proto.RaftProtos.ServerRpcProto;
import org.apache.ratis.protocol.RaftGroupId;
import org.apache.ratis.protocol.RaftGroupMemberId;
-import org.apache.ratis.protocol.RaftPeer;
import org.apache.ratis.protocol.RaftPeerId;
import org.apache.ratis.server.RaftServerConfigKeys;
-import org.apache.ratis.server.protocol.RaftServerProtocol;
+import org.apache.ratis.server.impl.FollowerState.UpdateType;
+import org.apache.ratis.server.protocol.RaftServerProtocol.Op;
import org.apache.ratis.server.protocol.TermIndex;
import org.apache.ratis.server.raftlog.LogProtoUtils;
import org.apache.ratis.server.util.ServerStringUtils;
@@ -173,16 +173,16 @@ class SnapshotInstallationHandler {
final long lastIncludedIndex = lastIncluded.getIndex();
final CompletableFuture<Void> future;
synchronized (server) {
- final boolean recognized =
state.recognizeLeader(RaftServerProtocol.Op.INSTALL_SNAPSHOT, leaderId,
leaderTerm);
+ final boolean recognized = state.recognizeLeader(Op.INSTALL_SNAPSHOT,
leaderId, leaderTerm);
currentTerm = state.getCurrentTerm();
if (!recognized) {
return
CompletableFuture.completedFuture(toInstallSnapshotReplyProto(leaderId,
getMemberId(),
currentTerm, snapshotChunkRequest.getRequestIndex(),
InstallSnapshotResult.NOT_LEADER));
}
- future = server.changeToFollowerAndPersistMetadata(leaderTerm, true,
"installSnapshot");
- state.setLeader(leaderId, "installSnapshot");
+ future = server.changeToFollowerAndPersistMetadata(leaderTerm, true,
Op.INSTALL_SNAPSHOT);
+ state.setLeader(leaderId, Op.INSTALL_SNAPSHOT);
-
server.updateLastRpcTime(FollowerState.UpdateType.INSTALL_SNAPSHOT_START);
+ server.updateLastRpcTime(UpdateType.INSTALL_SNAPSHOT_START);
long callId = chunk0CallId.get();
// 1. leaderTerm < currentTerm will never come here
// 2. leaderTerm == currentTerm && callId == request.getCallId()
@@ -229,7 +229,7 @@ class SnapshotInstallationHandler {
chunk0CallId.set(-1);
}
} finally {
-
server.updateLastRpcTime(FollowerState.UpdateType.INSTALL_SNAPSHOT_COMPLETE);
+ server.updateLastRpcTime(UpdateType.INSTALL_SNAPSHOT_COMPLETE);
}
}
if (snapshotChunkRequest.getDone()) {
@@ -249,15 +249,15 @@ class SnapshotInstallationHandler {
final long firstAvailableLogIndex = firstAvailableLogTermIndex.getIndex();
final CompletableFuture<Void> future;
synchronized (server) {
- final boolean recognized =
state.recognizeLeader("notifyInstallSnapshot", leaderId, leaderTerm);
+ final boolean recognized =
state.recognizeLeader(UpdateType.INSTALL_SNAPSHOT_NOTIFICATION, leaderId,
leaderTerm);
currentTerm = state.getCurrentTerm();
if (!recognized) {
return
CompletableFuture.completedFuture(toInstallSnapshotReplyProto(leaderId,
getMemberId(),
currentTerm, InstallSnapshotResult.NOT_LEADER));
}
- future = server.changeToFollowerAndPersistMetadata(leaderTerm, true,
"installSnapshot");
- state.setLeader(leaderId, "installSnapshot");
-
server.updateLastRpcTime(FollowerState.UpdateType.INSTALL_SNAPSHOT_NOTIFICATION);
+ future = server.changeToFollowerAndPersistMetadata(leaderTerm, true,
UpdateType.INSTALL_SNAPSHOT_NOTIFICATION);
+ state.setLeader(leaderId, UpdateType.INSTALL_SNAPSHOT_NOTIFICATION);
+ server.updateLastRpcTime(UpdateType.INSTALL_SNAPSHOT_NOTIFICATION);
if (inProgressInstallSnapshotIndex.compareAndSet(INVALID_LOG_INDEX,
firstAvailableLogIndex)) {
LOG.info("{}: Received notification to install snapshot at index {}",
getMemberId(), firstAvailableLogIndex);
@@ -292,7 +292,7 @@ class SnapshotInstallationHandler {
// For the cases where RaftConf is empty on newly started peer with
empty peer list,
// we retrieve leader info from installSnapShotRequestProto.
final RoleInfoProto proto = leaderProto == null ||
server.getRaftConf().getPeer(state.getLeaderId()) != null?
- server.getRoleInfoProto():
getRoleInfoProto(ProtoUtils.toRaftPeer(leaderProto));
+ server.getRoleInfoProto(): getRoleInfoProto(leaderProto);
// This is the first installSnapshot notify request for this term and
// index. Notify the state machine to install the snapshot.
LOG.info("{}: notifyInstallSnapshot: nextIndex is {} but the leader's
first available index is {}.",
@@ -386,7 +386,7 @@ class SnapshotInstallationHandler {
}
}
- private RoleInfoProto getRoleInfoProto(RaftPeer leader) {
+ private RoleInfoProto getRoleInfoProto(RaftPeerProto leader) {
final RoleInfo role = server.getRole();
final Optional<FollowerState> fs = role.getFollowerState();
final ServerRpcProto leaderInfo = toServerRpcProto(leader,