Skip to content

Commit

Permalink
RATIS-761. Handle writeStateMachineData failure in leader.
Browse files Browse the repository at this point in the history
  • Loading branch information
szetszwo committed Sep 25, 2023
1 parent d461a01 commit 711cb5a
Showing 1 changed file with 17 additions and 3 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@
import org.apache.ratis.server.RaftConfiguration;
import org.apache.ratis.server.RaftServerConfigKeys;
import org.apache.ratis.server.protocol.TermIndex;
import org.apache.ratis.server.util.ServerStringUtils;
import org.apache.ratis.statemachine.TransactionContext;
import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
import org.apache.ratis.util.AutoCloseableLock;
Expand Down Expand Up @@ -178,14 +179,27 @@ private long appendImpl(long term, TransactionContext operation) throws StateMac
}

// build the log entry after calling the StateMachine
final LogEntryProto e = operation.initLogEntry(term, nextIndex);
final LogEntryProto entry = operation.initLogEntry(term, nextIndex);

int entrySize = e.getSerializedSize();
final int entrySize = entry.getSerializedSize();
if (entrySize > maxBufferSize) {
throw new StateMachineException(memberId, new RaftLogIOException(
"Log entry size " + entrySize + " exceeds the max buffer limit of " + maxBufferSize));
}
appendEntry(e);
appendEntry(entry).whenComplete((index, e) -> {
if (index != nextIndex) {
LOG.error("{}: Indices mismatched: returned index={} but nextIndex={} for log entry {}",
name, index, nextIndex, LogProtoUtils.toLogEntryString(entry));
if (e != null) {
LOG.error("Failed to write log entry " + TermIndex.valueOf(entry), e);
}
try {
close();
} catch (IOException ioe) {
LOG.error("Failed to close " + name, ioe);
}
}
});
return nextIndex;
}
}
Expand Down

0 comments on commit 711cb5a

Please sign in to comment.