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

Siyao Meng commented on RATIS-2089:
-----------------------------------

Thanks a lot for filing this one [~ivanandika] . I am +1 on this. I am in the 
same boat when working on HDDS-10108.

A little bit background on my side:
1. In HDDS-10108, I'm trying to utilize the improvement made in RATIS-1994.
2. The goal of HDDS-10108 is to use {{send(ALL_COMMITTED)}} ONLY in Ozone to 
achieve the same level of consistency as {{send(MAJORITY)}} + 
{{watch(ALL_COMMITTED)}}. The latter is the existing Ozone implementation.
3. So effectively, the goal of HDDS-10108 is to remove explicit watch() calls 
in Ozone in order to eliminate this extra {{watch()}} round trip.
4. Then I 
[observed|https://github.com/apache/ozone/pull/6014#issuecomment-2102763285] 
the same issue as you mentioned, an Ozone test case 
`testDatanodeExclusionWithMajorityCommit` failed because {{failedServers}} 
(dead datanodes list) relies on the "retry" logic in 
{{XceiverClientRatis#watchForCommit}} to be populated.
5. And as a result, I will only get null reply (with non-null exception) from 
the future returned from {{XceiverClientRatis#sendRequestAsync}}. So I won't 
get CommitInfo that will be very useful in this case. If RATIS-2089 can be 
done, I won't have to do the "retry hack" like {{watchForCommit()}} did :)

> 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