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

Siyao Meng edited comment on RATIS-2089 at 5/29/24 8:39 PM:
------------------------------------------------------------

bq. Did you know what Ratis exception is thrown in 
testDatanodeExclusionWithMajorityCommit? Is it TimeoutIOException thrown by the 
client or NotReplicatedException throen by server.

-I have seen {{NotReplicatedException}}. But I also remember seeing 
{{TimeoutIOException}} when I'm debugging 
{{testDatanodeExclusionWithMajorityCommit}} in HDDS-10108. But maybe the 
{{TimeoutIOException}} is somehow caused by {{NotReplicatedException}} and in 
that case when I backport RATIS-2089 (to my local dev branch) and fix Ozone 
client retry it wouldn't happen anymore (so as to improve latency).- Uhh 
actually I realized that the NotReplicatedException I have seen is from my 
retry policy debug print rather than actual NotReplicatedException. So no it 
doesn't look like NotReplicatedException is being returned to the client at 
this point. I will try reducing the timeout to see if it helps.

bq. Additionally, may I know whether the Ozone cluster used changed these 
configurations and whether there are latency improvements?

Good point. But we haven't tried changing these timeout configs any further 
than Ozone defaults at this point.


was (Author: smeng):
bq. Did you know what Ratis exception is thrown in 
testDatanodeExclusionWithMajorityCommit? Is it TimeoutIOException thrown by the 
client or NotReplicatedException throen by server.

-I have seen {{NotReplicatedException}}. But I also remember seeing 
{{TimeoutIOException}} when I'm debugging 
{{testDatanodeExclusionWithMajorityCommit}} in HDDS-10108. But maybe the 
{{TimeoutIOException}} is somehow caused by {{NotReplicatedException}} and in 
that case when I backport RATIS-2089 (to my local dev branch) and fix Ozone 
client retry it wouldn't happen anymore (so as to improve latency).- Actually 
the NotReplicatedException seen from my retry policy debug print rather than 
actual NotReplicatedException. So no it doesn't look like 
NotReplicatedException is being returned to the client at this point. I will 
try reducing the timeout to see if it helps.

bq. Additionally, may I know whether the Ozone cluster used changed these 
configurations and whether there are latency improvements?

Good point. But we haven't tried changing these timeout configs any further 
than Ozone defaults at this point.

> Add CommitInfoProto in NotReplicatedException
> ---------------------------------------------
>
>                 Key: RATIS-2089
>                 URL: https://issues.apache.org/jira/browse/RATIS-2089
>             Project: Ratis
>          Issue Type: Improvement
>          Components: client
>            Reporter: Ivan Andika
>            Assignee: Ivan Andika
>            Priority: Minor
>             Fix For: 3.1.0
>
>          Time Spent: 50m
>  Remaining Estimate: 0h
>
> 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