介绍
具体来说,StateMachineUpdater 在 Raft 中扮演以下角色:
-
日志应用:Raft 算法通过日志的复制和应用来确保一致性。Leader 节点将客户端的请求转换为日志条目,并将这些日志条目复制到其他节点。一旦大多数节点都确认接收并持久化了日志,Leader 就可以提交这些日志。StateMachineUpdater 接收提交的日志,将其应用到本地状态机中,从而实现状态机的状态更新。
-
确保一致性:Raft 算法保证了复制日志的一致性,这意味着所有节点最终会在相同的日志条目上达成共识。StateMachineUpdater 确保在所有节点上应用相同的日志序列,从而使状态机保持一致。
-
StateMachineUpdater会根据当前的日志情况创建snapshot文件,将当前状态记录为snapshot文件,成功后会purge掉不需要的log文件。
看代码
StatemachineUpdater的线程负责检查当前是否需要制作snapshot和获取已经提交到follower并成功返回的commit信息应用到状态机中。
public void run() {
for(; state != State.STOP; ) {
try {
waitForCommit();
if (state == State.RELOAD) {
reload();
}
final MemoizedSupplier<List<CompletableFuture<Message>>> futures = applyLog();
checkAndTakeSnapshot(futures);
if (shouldStop()) {
checkAndTakeSnapshot(futures);
stop();
}
} catch (Throwable t) {
if (t instanceof InterruptedException && state == State.STOP) {
LOG.info("{} was interrupted. Exiting ...", this);
} else {
state = State.EXCEPTION;
LOG.error(this + " caught a Throwable.", t);
server.close();
}
}
}
}
其中的applyLog方法负责向上层StateMachine提交已经成功的日志。
private MemoizedSupplier<List<CompletableFuture<Message>>> applyLog() throws RaftLogIOException {
final MemoizedSupplier<List<CompletableFuture<Message>>> futures = MemoizedSupplier.valueOf(ArrayList::new);
final long committed = raftLog.getLastCommittedIndex();
for(long applied; (applied = getLastAppliedIndex()) < committed && state == State.RUNNING && !shouldStop(); ) {
final long nextIndex = applied + 1;
final LogEntryProto next = raftLog.get(nextIndex);
if (next != null) {
if (LOG.isTraceEnabled()) {
LOG.trace("{}: applying nextIndex={}, nextLog={}", this, nextIndex, LogProtoUtils.toLogEntryString(next));
} else {
LOG.debug("{}: applying nextIndex={}", this, nextIndex);
}
final CompletableFuture<Message> f = server.applyLogToStateMachine(next);
final long incremented = appliedIndex.incrementAndGet(debugIndexChange);
Preconditions.assertTrue(incremented == nextIndex);
if (f != null) {
futures.get().add(f);
f.thenAccept(m -> notifyAppliedIndex(incremented));
} else {
notifyAppliedIndex(incremented);
}
} else {
LOG.debug("{}: logEntry {} is null. There may be snapshot to load. state:{}",
this, nextIndex, state);
break;
}
}
return futures;
}
stateMachineUpdater还有一个重要的作用就是制作snapshot,在制作snapshot之前会判断当前是否应该制造snapshot, 如果不是手动生成snapshot则需要log日志的条数大于400000条(默认)才会生成snapshot.
private boolean shouldTakeSnapshot() {
if (state == State.RUNNING && server.getSnapshotRequestHandler().shouldTriggerTakingSnapshot()) {
return true;
}
if (autoSnapshotThreshold == null) {
return false;
} else if (shouldStop()) {
return getLastAppliedIndex() - snapshotIndex.get() > 0;
}
return state == State.RUNNING &&
getStateMachineLastAppliedIndex() - snapshotIndex.get() >= autoSnapshotThreshold;
}
在判断需要生成snapshot时候会调用takeSnapshot开始生成文件
private void takeSnapshot() {
final long i;
try {
// 调用上层的stateMachine开始生成snapshot文件
try(UncheckedAutoCloseable ignored = Timekeeper.start(stateMachineMetrics.get().getTakeSnapshotTimer())) {
i = stateMachine.takeSnapshot();
}
server.getSnapshotRequestHandler().completeTakingSnapshot(i);
final long lastAppliedIndex = getLastAppliedIndex();
if (i > lastAppliedIndex) {
throw new StateMachineException(
"Bug in StateMachine: snapshot index = " + i + " > appliedIndex = " + lastAppliedIndex
+ "; StateMachine class=" + stateMachine.getClass().getName() + ", stateMachine=" + stateMachine);
}
// 清理老的snapshot文件
stateMachine.getStateMachineStorage().cleanupOldSnapshots(snapshotRetentionPolicy);
} catch (IOException e) {
LOG.error(name + ": Failed to take snapshot", e);
return;
}
if (i >= 0) {
LOG.info("{}: Took a snapshot at index {}", name, i);
snapshotIndex.updateIncreasingly(i, infoIndexChange);
final long purgeIndex;
if (purgeUptoSnapshotIndex) {
// We can purge up to snapshot index even if all the peers do not have
// commitIndex up to this snapshot index.
purgeIndex = i;
} else {
final LongStream commitIndexStream = server.getCommitInfos().stream().mapToLong(
CommitInfoProto::getCommitIndex);
purgeIndex = LongStream.concat(LongStream.of(i), commitIndexStream).min().orElse(i);
}
// purge掉生成snapshot的index之前的所有log日志
raftLog.purge(purgeIndex);
}
}