[ 
https://issues.apache.org/jira/browse/RATIS-2089?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17849619#comment-17849619
 ] 

Ivan Andika commented on RATIS-2089:
------------------------------------

[~szetszwo] Thank you for the explanation

> The reply from Ratis server is a just MAJORITY, which means "replicated to a 
> majority of peers" (the peers except for the leader may not be able to apply 
> the log). MAJORITY_COMMITTED is more strict. It means "committed at a 
> majority of peers" (i.e. they can apply the log locally).

I see, I thought it was MAJORITY_COMMITTED which I assumed from the Raft paper.

> btw, the hsync feature (HDDS-7593) is going to improve the latency and write 
> performance

Sure, let me take a look for latency-related patches.  

> Add CommitInfoProto in NotReplicatedException
> ---------------------------------------------
>
>                 Key: RATIS-2089
>                 URL: https://issues.apache.org/jira/browse/RATIS-2089
>             Project: Ratis
>          Issue Type: Improvement
>            Reporter: Ivan Andika
>            Assignee: Ivan Andika
>            Priority: Minor
>
> In Ozone's XceiverClientRatis#watchForCommit, there are two watch commits 
> request with different ReplicationLevel
>  # Watch for ALL_COMMITTED 
>  # Watch for MAJORITY_COMMITTED (If the previous watch threw an exception)
> Based on the second watch request, the client will remove some failed 
> datanode UUID from the commitInfoMap.
> The second watch might not be necessary since the entries in 
> AbstractCommitWatcher.commitIndexMap implies that the PutBlock request has 
> been committed to the majority of the servers. Therefore, another 
> MAJORITY_COMMITTED watch might not be necessary. From my understanding, the 
> second MAJORITY_COMMITTED only serves to gain information to remove entries 
> from commitInfoMap.
> If the first watch failed with NotReplicatedException, we might be able to 
> remove the need to a second watch request. Since NotReplicatedException is a 
> Raft server exception, we can include the CommitInfoProtos in the 
> NotReplicatedException. The client can use this CommitInfoProtos to remove 
> the entry from commitInfoMap without sending another WATCH request. 
> This CommitInfoProto is returned for every RaftClientReply 
> (RaftClientReply.commitInfos), but if there is an exception, it seems the 
> RaftClientReply is not accessible to the client.
> However, if the exception is a client exception (e.g. due to Raft client 
> watch timeout configuration), the client might have no choice but to send 
> another watch request.
> So in this patch, I propose to include CommitInfoProto into 
> NotReplicatedException.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to