lokeshj1703 commented on a change in pull request #88:
URL: https://github.com/apache/incubator-ratis/pull/88#discussion_r421367469
##########
File path:
ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java
##########
@@ -222,6 +222,10 @@ private void increaseNextIndex(AppendEntriesRequestProto
request) {
}
}
+ private void increaseNextIndex(final long installedSnapshotIndex) {
+ getFollower().increaseNextIndex(installedSnapshotIndex + 1);
Review comment:
FollowerInfo#increaseNextIndex will throw error if current next index is
greater than installedSnapshotIndex + 1. I think we should do something like
updateCommitIndex here.
##########
File path:
ratis-server/src/main/java/org/apache/ratis/server/raftlog/segmented/SegmentedRaftLog.java
##########
@@ -492,6 +492,13 @@ public void syncWithSnapshot(long lastSnapshotIndex) {
// if the last index in snapshot is larger than the index of the last
// log entry, we should delete all the log entries and their cache to avoid
// gaps between log segments.
+
+ // Close open log segment if entries are already included in snapshot
+ LogSegment openSegment = cache.getOpenSegment();
+ if (openSegment != null && openSegment.getEndIndex() <= lastSnapshotIndex)
{
+ fileLogWorker.closeLogSegment(openSegment);
+ }
+ cache.clear();
Review comment:
We can potentially clear closed log segments with endIndex > snapshot
index? Maybe we should move it inside the if statement?
##########
File path:
ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java
##########
@@ -369,13 +381,19 @@ public void onNext(InstallSnapshotReplyProto reply) {
switch (reply.getResult()) {
case SUCCESS:
Review comment:
Do we need to update commitIndex and nextIndex here as well?
##########
File path:
ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java
##########
@@ -394,17 +412,24 @@ public void onNext(InstallSnapshotReplyProto reply) {
@Override
public void onError(Throwable t) {
if (!isAppenderRunning()) {
- LOG.info("{} is stopped", this);
+ LOG.info("{} is stopped", GrpcLogAppender.this);
return;
}
- LOG.error("{}: Failed installSnapshot: {}", this, t);
+ GrpcUtil.warn(LOG, () -> this + ": Failed InstallSnapshot", t);
+ grpcServerMetrics.onRequestRetry(); // Update try counter
resetClient(null);
close();
}
@Override
public void onCompleted() {
- LOG.info("{}: follower responses installSnapshot COMPLETED", this);
+ if (!isNotificationOnly) {
+ LOG.info("{}: follower responded installSnapshot COMPLETED", this);
+ } else {
+ if (LOG.isDebugEnabled()) {
+ LOG.info("{}: follower responded installSnapshot COMPLETED", this);
+ }
+ }
Review comment:
We can combine the if statements using below condition.
`!isNotificationOnly || LOG.isDebugEnabled()`
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]