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/incubator-ratis.git
The following commit(s) were added to refs/heads/master by this push:
new b5b0ff2 RATIS-600. Better logging for installSnapshot.
b5b0ff2 is described below
commit b5b0ff2d1eed13045c069239ceb5f676e21736e8
Author: Tsz Wo Nicholas Sze <[email protected]>
AuthorDate: Wed Jul 3 13:11:13 2019 +0800
RATIS-600. Better logging for installSnapshot.
---
.../apache/ratis/protocol/RaftGroupMemberId.java | 71 +++++++++++++++++
.../apache/ratis/server/impl/RaftServerImpl.java | 88 ++++++++++++----------
.../apache/ratis/server/impl/ServerProtoUtils.java | 70 ++++++++++++++---
3 files changed, 178 insertions(+), 51 deletions(-)
diff --git
a/ratis-common/src/main/java/org/apache/ratis/protocol/RaftGroupMemberId.java
b/ratis-common/src/main/java/org/apache/ratis/protocol/RaftGroupMemberId.java
new file mode 100644
index 0000000..b4a0b50
--- /dev/null
+++
b/ratis-common/src/main/java/org/apache/ratis/protocol/RaftGroupMemberId.java
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ratis.protocol;
+
+import java.util.Objects;
+
+/**
+ * A {@link RaftGroupMemberId} consists of a {@link RaftPeerId} and a {@link
RaftGroupId}.
+ *
+ * This is a value-based class.
+ */
+public final class RaftGroupMemberId {
+ public static RaftGroupMemberId valueOf(RaftPeerId peerId, RaftGroupId
groupId) {
+ return new RaftGroupMemberId(peerId, groupId);
+ }
+
+ private final RaftPeerId peerId;
+ private final RaftGroupId groupId;
+ private final String name;
+
+ private RaftGroupMemberId(RaftPeerId peerId, RaftGroupId groupId) {
+ this.peerId = Objects.requireNonNull(peerId, "peerId == null");
+ this.groupId = Objects.requireNonNull(groupId, "groupId == null");
+ this.name = peerId + "@" + groupId;
+ }
+
+ public RaftPeerId getPeerId() {
+ return peerId;
+ }
+
+ public RaftGroupId getGroupId() {
+ return groupId;
+ }
+
+ @Override
+ public String toString() {
+ return name;
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (obj == this) {
+ return true;
+ } else if (!(obj instanceof RaftGroupMemberId)) {
+ return false;
+ }
+
+ final RaftGroupMemberId that = (RaftGroupMemberId)obj;
+ return this.peerId.equals(that.peerId) &&
this.groupId.equals(that.groupId);
+ }
+
+ @Override
+ public int hashCode() {
+ return peerId.hashCode() ^ groupId.hashCode();
+ }
+}
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 d7843d5..b819e29 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
@@ -78,6 +78,7 @@ public class RaftServerImpl implements RaftServerProtocol,
RaftServerAsynchronou
private final LifeCycle lifeCycle;
private final ServerState state;
private final RaftGroupId groupId;
+ private final RaftGroupMemberId memberId; // TODO: move it to ServerState;
see RATIS-605
private final Supplier<RaftPeer> peerSupplier = JavaUtils.memoize(() -> new
RaftPeer(getId(), getServerRpc().getInetSocketAddress()));
private final RoleInfo role;
@@ -92,6 +93,7 @@ public class RaftServerImpl implements RaftServerProtocol,
RaftServerAsynchronou
final RaftPeerId id = proxy.getId();
LOG.info("{}: new RaftServerImpl for {} with {}", id, group, stateMachine);
this.groupId = group.getGroupId();
+ this.memberId = RaftGroupMemberId.valueOf(id, groupId);
this.lifeCycle = new LifeCycle(id);
this.stateMachine = stateMachine;
this.role = new RoleInfo(id);
@@ -224,6 +226,10 @@ public class RaftServerImpl implements RaftServerProtocol,
RaftServerAsynchronou
return state;
}
+ public RaftGroupMemberId getMemberId() {
+ return memberId;
+ }
+
public RaftPeerId getId() {
return getState().getSelfId();
}
@@ -1004,14 +1010,29 @@ public class RaftServerImpl implements
RaftServerProtocol, RaftServerAsynchronou
}
@Override
- public InstallSnapshotReplyProto installSnapshot(
- InstallSnapshotRequestProto request) throws IOException {
+ public InstallSnapshotReplyProto installSnapshot(InstallSnapshotRequestProto
request) throws IOException {
+ if (LOG.isInfoEnabled()) {
+ LOG.info("{}: receive installSnapshot: {}", getMemberId(),
ServerProtoUtils.toString(request));
+ }
+ final InstallSnapshotReplyProto reply;
+ try {
+ reply = installSnapshotImpl(request);
+ } catch (Throwable t) {
+ LOG.error("{}: installSnapshot failed", getMemberId(), t);
+ throw t;
+ }
+ if (LOG.isInfoEnabled()) {
+ LOG.info("{}: reply installSnapshot: {}", getMemberId(),
ServerProtoUtils.toString(reply));
+ }
+ return reply;
+ }
+
+ private InstallSnapshotReplyProto
installSnapshotImpl(InstallSnapshotRequestProto request) throws IOException {
final RaftRpcRequestProto r = request.getServerRequest();
final RaftPeerId leaderId = RaftPeerId.valueOf(r.getRequestorId());
final RaftGroupId leaderGroupId =
ProtoUtils.toRaftGroupId(r.getRaftGroupId());
CodeInjectionForTesting.execute(INSTALL_SNAPSHOT, getId(),
leaderId, request);
- LOG.debug("{}: receive installSnapshot({})", getId(), request);
assertLifeCycleState(STARTING, RUNNING);
assertGroup(leaderId, leaderGroupId);
@@ -1029,11 +1050,10 @@ public class RaftServerImpl implements
RaftServerProtocol, RaftServerAsynchronou
}
}
// There is a mismatch between configurations on leader and follower.
- final InstallSnapshotReplyProto reply = ServerProtoUtils
- .toInstallSnapshotReplyProto(leaderId, getId(), groupId,
- InstallSnapshotResult.CONF_MISMATCH);
+ final InstallSnapshotReplyProto reply =
ServerProtoUtils.toInstallSnapshotReplyProto(
+ leaderId, getMemberId(), InstallSnapshotResult.CONF_MISMATCH);
LOG.error("{}: Configuration Mismatch ({}): Leader {} has it set to {} but
follower {} has it set to {}",
- getId(),
RaftServerConfigKeys.Log.Appender.INSTALL_SNAPSHOT_ENABLED_KEY,
+ getMemberId(),
RaftServerConfigKeys.Log.Appender.INSTALL_SNAPSHOT_ENABLED_KEY,
leaderId, request.hasSnapshotChunk(), getId(), installSnapshotEnabled);
return reply;
}
@@ -1045,22 +1065,19 @@ public class RaftServerImpl implements
RaftServerProtocol, RaftServerAsynchronou
InstallSnapshotRequestProto.SnapshotChunkProto snapshotChunkRequest =
request.getSnapshotChunk();
final TermIndex lastTermIndex =
ServerProtoUtils.toTermIndex(snapshotChunkRequest.getTermIndex());
final long lastIncludedIndex = lastTermIndex.getIndex();
- final Optional<FollowerState> followerState;
synchronized (this) {
final boolean recognized = state.recognizeLeader(leaderId, leaderTerm);
currentTerm = state.getCurrentTerm();
if (!recognized) {
- final InstallSnapshotReplyProto reply = ServerProtoUtils
- .toInstallSnapshotReplyProto(leaderId, getId(), groupId,
currentTerm,
- snapshotChunkRequest.getRequestIndex(),
InstallSnapshotResult.NOT_LEADER);
- LOG.debug("{}: do not recognize leader for installing snapshot." +
- " Reply: {}", getId(), reply);
+ final InstallSnapshotReplyProto reply =
ServerProtoUtils.toInstallSnapshotReplyProto(leaderId, getMemberId(),
+ currentTerm, snapshotChunkRequest.getRequestIndex(),
InstallSnapshotResult.NOT_LEADER);
+ LOG.warn("{}: Failed to recognize leader for installSnapshot chunk.
Reply: {}", getMemberId(), reply);
return reply;
}
changeToFollowerAndPersistMetadata(leaderTerm, "installSnapshot");
state.setLeader(leaderId, "installSnapshot");
- followerState =
updateLastRpcTime(FollowerState.UpdateType.INSTALL_SNAPSHOT_START);
+ updateLastRpcTime(FollowerState.UpdateType.INSTALL_SNAPSHOT_START);
try {
// Check and append the snapshot chunk. We simply put this in lock
// considering a follower peer requiring a snapshot installation does
not
@@ -1083,10 +1100,9 @@ public class RaftServerImpl implements
RaftServerProtocol, RaftServerAsynchronou
}
}
if (snapshotChunkRequest.getDone()) {
- LOG.info("{}:{} successfully install the whole snapshot-{}", getId(),
getGroupId(),
- lastIncludedIndex);
+ LOG.info("{}: successfully install the entire snapshot-{}",
getMemberId(), lastIncludedIndex);
}
- return ServerProtoUtils.toInstallSnapshotReplyProto(leaderId, getId(),
groupId,
+ return ServerProtoUtils.toInstallSnapshotReplyProto(leaderId,
getMemberId(),
currentTerm, snapshotChunkRequest.getRequestIndex(),
InstallSnapshotResult.SUCCESS);
}
@@ -1102,11 +1118,9 @@ public class RaftServerImpl implements
RaftServerProtocol, RaftServerAsynchronou
final boolean recognized = state.recognizeLeader(leaderId, leaderTerm);
currentTerm = state.getCurrentTerm();
if (!recognized) {
- final InstallSnapshotReplyProto reply = ServerProtoUtils
- .toInstallSnapshotReplyProto(leaderId, getId(), groupId,
currentTerm,
- InstallSnapshotResult.NOT_LEADER, -1);
- LOG.debug("{}: do not recognize leader for installing snapshot." +
- " Reply: {}", getId(), reply);
+ final InstallSnapshotReplyProto reply =
ServerProtoUtils.toInstallSnapshotReplyProto(leaderId, getMemberId(),
+ currentTerm, InstallSnapshotResult.NOT_LEADER, -1);
+ LOG.warn("{}: Failed to recognize leader for installSnapshot
notification. Reply: {}", getMemberId(), reply);
return reply;
}
changeToFollowerAndPersistMetadata(leaderTerm, "installSnapshot");
@@ -1125,24 +1139,21 @@ public class RaftServerImpl implements
RaftServerProtocol, RaftServerAsynchronou
inProgressInstallSnapshotRequest.compareAndSet(firstAvailableLogTermIndex,
null);
final InstallSnapshotReplyProto reply =
ServerProtoUtils.toInstallSnapshotReplyProto(
- leaderId, getId(), groupId, currentTerm,
- InstallSnapshotResult.ALREADY_INSTALLED, snapshotIndex);
- LOG.info("{}: StateMachine latest installed snapshot index: {}.
Reply: {}",
- getId(), snapshotIndex, reply);
+ leaderId, getMemberId(), currentTerm,
InstallSnapshotResult.ALREADY_INSTALLED, snapshotIndex);
+ LOG.info("{}: StateMachine snapshotIndex: {}. Reply: {}",
getMemberId(), snapshotIndex, reply);
return reply;
}
// This is the first installSnapshot notify request for this term and
// index. Notify the state machine to install the snapshot.
- LOG.debug("{}: notifying state machine to install snapshot. Next log "
+
- "index is {} but the leader's first available log index is
{}.",
- getId(), state.getLog().getNextIndex(), firstAvailableLogIndex);
+ LOG.info("{}: notifyInstallSnapshot: nextIndex is {} but the leader's
first available index is {}.",
+ getMemberId(), state.getLog().getNextIndex(),
firstAvailableLogIndex);
stateMachine.notifyInstallSnapshotFromLeader(getRoleInfoProto(),
firstAvailableLogTermIndex)
.whenComplete((reply, exception) -> {
if (exception != null) {
- LOG.error(getId() + ": State Machine failed to install
snapshot", exception);
+ LOG.error("{}: State Machine failed to install snapshot",
getMemberId(), exception);
inProgressInstallSnapshotRequest.compareAndSet(firstAvailableLogTermIndex,
null);
return;
}
@@ -1153,17 +1164,15 @@ public class RaftServerImpl implements
RaftServerProtocol, RaftServerAsynchronou
state.updateInstalledSnapshotIndex(reply);
}
inProgressInstallSnapshotRequest.compareAndSet(firstAvailableLogTermIndex,
null);
- return;
});
- LOG.info("{}: StateMachine notified to install snapshot, Request: {}");
- return ServerProtoUtils.toInstallSnapshotReplyProto(leaderId, getId(),
groupId,
+ return ServerProtoUtils.toInstallSnapshotReplyProto(leaderId,
getMemberId(),
currentTerm, InstallSnapshotResult.SUCCESS, -1);
}
- LOG.debug("{}: StateMachine snapshot installation is in progress. " +
- "InProgress Request: {}", getId(),
inProgressInstallSnapshotRequest.get());
- return ServerProtoUtils.toInstallSnapshotReplyProto(leaderId, getId(),
groupId,
+ LOG.info("{}: StateMachine installSnapshot is in progress: {}",
+ getMemberId(), inProgressInstallSnapshotRequest.get());
+ return ServerProtoUtils.toInstallSnapshotReplyProto(leaderId,
getMemberId(),
currentTerm, InstallSnapshotResult.IN_PROGRESS, -1);
}
}
@@ -1174,17 +1183,16 @@ public class RaftServerImpl implements
RaftServerProtocol, RaftServerAsynchronou
OptionalLong totalSize = snapshot.getFiles().stream()
.mapToLong(FileInfo::getFileSize).reduce(Long::sum);
assert totalSize.isPresent();
- return ServerProtoUtils.toInstallSnapshotRequestProto(getId(), targetId,
groupId,
+ return ServerProtoUtils.toInstallSnapshotRequestProto(getMemberId(),
targetId,
requestId, requestIndex, state.getCurrentTerm(),
snapshot.getTermIndex(),
chunks, totalSize.getAsLong(), done);
}
synchronized InstallSnapshotRequestProto createInstallSnapshotRequest(
RaftPeerId targetId, TermIndex firstAvailableLogTermIndex) {
-
assert (firstAvailableLogTermIndex.getIndex() > 0);
- return ServerProtoUtils.toInstallSnapshotRequestProto(getId(),
- targetId, groupId, state.getCurrentTerm(), firstAvailableLogTermIndex);
+ return ServerProtoUtils.toInstallSnapshotRequestProto(getMemberId(),
targetId,
+ state.getCurrentTerm(), firstAvailableLogTermIndex);
}
synchronized RequestVoteRequestProto createRequestVoteRequest(
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 ff1e368..5d94929 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
@@ -23,6 +23,7 @@ import
org.apache.ratis.proto.RaftProtos.AppendEntriesReplyProto.AppendResult;
import org.apache.ratis.protocol.ClientId;
import org.apache.ratis.protocol.RaftClientRequest;
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.protocol.TermIndex;
@@ -119,6 +120,44 @@ public interface ServerProtoUtils {
return ProtoUtils.toString(proto.getServerReply()) + "-t" +
proto.getTerm();
}
+ static String toString(InstallSnapshotRequestProto proto) {
+ if (proto == null) {
+ return null;
+ }
+ final String s;
+ switch (proto.getInstallSnapshotRequestBodyCase()) {
+ case SNAPSHOTCHUNK:
+ final InstallSnapshotRequestProto.SnapshotChunkProto chunk =
proto.getSnapshotChunk();
+ s = "chunk:" + chunk.getRequestId() + "," + chunk.getRequestIndex();
+ break;
+ case NOTIFICATION:
+ final InstallSnapshotRequestProto.NotificationProto notification =
proto.getNotification();
+ s = "notify:" +
toTermIndexString(notification.getFirstAvailableTermIndex());
+ break;
+ default:
+ throw new IllegalStateException("Unexpected body case in " + proto);
+ }
+ return ProtoUtils.toString(proto.getServerRequest()) + "-t" +
proto.getLeaderTerm() + "," + s;
+ }
+
+ static String toString(InstallSnapshotReplyProto proto) {
+ if (proto == null) {
+ return null;
+ }
+ final String s;
+ switch (proto.getInstallSnapshotReplyBodyCase()) {
+ case REQUESTINDEX:
+ s = "requestIndex=" + proto.getRequestIndex();
+ break;
+ case SNAPSHOTINDEX:
+ s = "snapshotIndex=" + proto.getSnapshotIndex();
+ break;
+ default:
+ throw new IllegalStateException("Unexpected body case in " + proto);
+ }
+ return ProtoUtils.toString(proto.getServerReply()) + "-t" +
proto.getTerm() + "," + s;
+ }
+
static RaftConfigurationProto.Builder
toRaftConfigurationProto(RaftConfiguration conf) {
return RaftConfigurationProto.newBuilder()
.addAllPeers(ProtoUtils.toRaftPeerProtos(conf.getPeersInConf()))
@@ -250,6 +289,11 @@ public interface ServerProtoUtils {
}
static RaftRpcReplyProto.Builder toRaftRpcReplyProtoBuilder(
+ RaftPeerId requestorId, RaftGroupMemberId replyId, boolean success) {
+ return toRaftRpcReplyProtoBuilder(requestorId, replyId.getPeerId(),
replyId.getGroupId(), success);
+ }
+
+ static RaftRpcReplyProto.Builder toRaftRpcReplyProtoBuilder(
RaftPeerId requestorId, RaftPeerId replyId, RaftGroupId groupId, boolean
success) {
return ClientProtoUtils.toRaftRpcReplyProtoBuilder(
requestorId.toByteString(), replyId.toByteString(), groupId,
DEFAULT_CALLID, success);
@@ -266,6 +310,11 @@ public interface ServerProtoUtils {
}
static RaftRpcRequestProto.Builder toRaftRpcRequestProtoBuilder(
+ RaftGroupMemberId requestorId, RaftPeerId replyId) {
+ return toRaftRpcRequestProtoBuilder(requestorId.getPeerId(), replyId,
requestorId.getGroupId());
+ }
+
+ static RaftRpcRequestProto.Builder toRaftRpcRequestProtoBuilder(
RaftPeerId requestorId, RaftPeerId replyId, RaftGroupId groupId) {
return ClientProtoUtils.toRaftRpcRequestProtoBuilder(
requestorId.toByteString(), replyId.toByteString(), groupId,
DEFAULT_CALLID, null);
@@ -283,10 +332,10 @@ public interface ServerProtoUtils {
}
static InstallSnapshotReplyProto toInstallSnapshotReplyProto(
- RaftPeerId requestorId, RaftPeerId replyId, RaftGroupId groupId,
+ RaftPeerId requestorId, RaftGroupMemberId replyId,
long currentTerm, int requestIndex, InstallSnapshotResult result) {
final RaftRpcReplyProto.Builder rb =
toRaftRpcReplyProtoBuilder(requestorId,
- replyId, groupId, result == InstallSnapshotResult.SUCCESS);
+ replyId, result == InstallSnapshotResult.SUCCESS);
final InstallSnapshotReplyProto.Builder builder = InstallSnapshotReplyProto
.newBuilder().setServerReply(rb).setTerm(currentTerm).setResult(result)
.setRequestIndex(requestIndex);
@@ -294,10 +343,10 @@ public interface ServerProtoUtils {
}
static InstallSnapshotReplyProto toInstallSnapshotReplyProto(
- RaftPeerId requestorId, RaftPeerId replyId, RaftGroupId groupId,
+ RaftPeerId requestorId, RaftGroupMemberId replyId,
long currentTerm, InstallSnapshotResult result, long
installedSnapshotIndex) {
final RaftRpcReplyProto.Builder rb =
toRaftRpcReplyProtoBuilder(requestorId,
- replyId, groupId, result == InstallSnapshotResult.SUCCESS);
+ replyId, result == InstallSnapshotResult.SUCCESS);
final InstallSnapshotReplyProto.Builder builder = InstallSnapshotReplyProto
.newBuilder().setServerReply(rb).setTerm(currentTerm).setResult(result);
if (installedSnapshotIndex > 0) {
@@ -307,17 +356,17 @@ public interface ServerProtoUtils {
}
static InstallSnapshotReplyProto toInstallSnapshotReplyProto(
- RaftPeerId requestorId, RaftPeerId replyId, RaftGroupId groupId,
+ RaftPeerId requestorId, RaftGroupMemberId replyId,
InstallSnapshotResult result) {
final RaftRpcReplyProto.Builder rb =
toRaftRpcReplyProtoBuilder(requestorId,
- replyId, groupId, result == InstallSnapshotResult.SUCCESS);
+ replyId, result == InstallSnapshotResult.SUCCESS);
final InstallSnapshotReplyProto.Builder builder = InstallSnapshotReplyProto
.newBuilder().setServerReply(rb).setResult(result);
return builder.build();
}
static InstallSnapshotRequestProto toInstallSnapshotRequestProto(
- RaftPeerId requestorId, RaftPeerId replyId, RaftGroupId groupId, String
requestId, int requestIndex,
+ RaftGroupMemberId requestorId, RaftPeerId replyId, String requestId, int
requestIndex,
long term, TermIndex lastTermIndex, List<FileChunkProto> chunks,
long totalSize, boolean done) {
final InstallSnapshotRequestProto.SnapshotChunkProto.Builder
snapshotChunkProto =
@@ -329,7 +378,7 @@ public interface ServerProtoUtils {
.setTotalSize(totalSize)
.setDone(done);
return InstallSnapshotRequestProto.newBuilder()
- .setServerRequest(toRaftRpcRequestProtoBuilder(requestorId, replyId,
groupId))
+ .setServerRequest(toRaftRpcRequestProtoBuilder(requestorId, replyId))
// .setRaftConfiguration() TODO: save and pass RaftConfiguration
.setLeaderTerm(term)
.setSnapshotChunk(snapshotChunkProto)
@@ -337,13 +386,12 @@ public interface ServerProtoUtils {
}
static InstallSnapshotRequestProto toInstallSnapshotRequestProto(
- RaftPeerId requestorId, RaftPeerId replyId, RaftGroupId groupId,
- long leaderTerm, TermIndex firstAvailable) {
+ RaftGroupMemberId requestorId, RaftPeerId replyId, long leaderTerm,
TermIndex firstAvailable) {
final InstallSnapshotRequestProto.NotificationProto.Builder
notificationProto =
InstallSnapshotRequestProto.NotificationProto.newBuilder()
.setFirstAvailableTermIndex(toTermIndexProto(firstAvailable));
return InstallSnapshotRequestProto.newBuilder()
- .setServerRequest(toRaftRpcRequestProtoBuilder(requestorId, replyId,
groupId))
+ .setServerRequest(toRaftRpcRequestProtoBuilder(requestorId, replyId))
// .setRaftConfiguration() TODO: save and pass RaftConfiguration
.setLeaderTerm(leaderTerm)
.setNotification(notificationProto)