[
https://issues.apache.org/jira/browse/RATIS-563?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16848789#comment-16848789
]
Mukul Kumar Singh commented on RATIS-563:
-----------------------------------------
Thanks for working on this [~szetszwo]. The patch generally looks good to me.
Some comments as following.
a) RaftLog.java, lets add a precondition with a log message to check that index
is always >= lastPurge.
b) RaftLog.java:340, the condition should be only "<" and not "<=" as we should
purge when it is equal to purge gap.
c) StateMachineUpdater.java:233, lets also add a precondition that
minCommitIndex should be lesser that lastAppliedIndex
d) StateMachineUpdater:233, CommitInfoCache.minCommitIndex can return a log
index which has still not been applied, purge index should be
min(minCommitIndex, snapshotIndex).
e) StateMachineUpdater:229 condition should be > 0, as taking snapshot at 0 is
unnecessary.
f) Lets also add a precondition in SegmentRaftLogCache.java:268 & 260 to assert
that isOpen on segment returns false.
> 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
>
>
> 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)