[
https://issues.apache.org/jira/browse/RATIS-563?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16849624#comment-16849624
]
Tsz Wo Nicholas Sze commented on RATIS-563:
-------------------------------------------
In RaftLog, commitIndex is initialized by a parameter.
{code}
protected RaftLog(RaftPeerId selfId, long commitIndex, RaftProperties
properties) {
...
this.commitIndex = new RaftLogIndex("commitIndex", commitIndex);
...
}
{code}
and the value of parameter is returned from ServerState.initStatemachine(..)
{code}
private long initStatemachine(StateMachine sm, RaftGroupId gid)
throws IOException {
...
return snapshot.getIndex();
}
{code}
{code}
ServerState(RaftPeerId id, RaftGroup group, RaftProperties prop,
RaftServerImpl server, StateMachine stateMachine)
throws IOException {
...
long lastApplied = initStatemachine(stateMachine, group.getGroupId());
...
log = initLog(id, prop, lastApplied, this::setRaftConf);
...
}
{code}
> Purge logs after snapshot only after all the nodes have catught up with the
> index
> ---------------------------------------------------------------------------------
>
> Key: RATIS-563
> URL: https://issues.apache.org/jira/browse/RATIS-563
> Project: Ratis
> Issue Type: Bug
> Components: snapshot
> Affects Versions: 0.3.0
> Reporter: Mukul Kumar Singh
> Assignee: Tsz Wo Nicholas Sze
> Priority: Major
> Attachments: r563_20190522.patch, r563_20190524.patch,
> r563_20190528.patch, r563_20190528b.patch
>
>
> Leader will purge the snapshot after applying a transaction. A transaction is
> applied after a majority has been reached. This means that a slow follower
> will request for a transaction which has already been purged and for this
> transaction to catchup the leader has to transfer the snapshot. This can be
> detrimental for the performance while replicating.
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)