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

Ivan Andika edited comment on RATIS-2089 at 5/27/24 6:11 AM:
-------------------------------------------------------------

[~smeng] Thank you for checking this out, glad to see that this might be 
useful. 

I thought of this improvement while analyzing the Ratis block write process and 
the upper bound latency for block write in case of failure (e.g. 1 datanode is 
down/unreachable). We found three relevant watch configurations that might be 
too high:
 * Client-side watch timeout: hdds.ratis.raft.client.rpc.watch.request.timeout
 ** Ratis equivalent: raft.client.rpc.watch.request.timeout
 ** Ratis default: 10s
 ** Ozone default: 180s
 * Server-side watch timeout (This is the one that throws 
NotReplicatedException): hdds.ratis.raft.server.watch.timeout
 ** Ratis equivalent: raft.server.watch.timeout
 ** Ratis default: 10s
 ** Ozone default: 180s
 * Cleint-side watch retry timeout: hdds.ratis.client.request.watch.timeout
 ** No Ratis equivalent. This is used in RequestTypeDependentRetryPolicyCreator 
to decide whether to throw RaftRetryFailureException
 ** I don't think this configuration is used frequently since WATCH request 
retry policy is RetryPolicies.noRetry(), since NotReplicatedException have 
NO_RETRY policy (only ResourceUnavailableException will be retried).
 ** Ozone default: 3m

All of them have the same values. Most likely the client-side watch timeout 
will always be triggered since the client-side watch timeout starts first. This 
might cause NotReplicatedException to never be caught by the client since 
client timeout happened before server timeout. I believe we need to reduce the 
server-side watch timeout significantly (e.g. 30s or 10s) so that 
NotReplicatedException can be thrown faster. Only then, the chance of 
NotReplicatedException caught will be higher and we can use the 
CommitInfoProto. Additionally, I believe the server-side watch timeout should 
always be shorter than the client-side watch timeout since if not, the pending 
watch request in Ratis leader might build up and cause 
ResourceUnavailableException. 

[~smeng] Did you know what Ratis exception is thrown in 
testDatanodeExclusionWithMajorityCommit? Is it TimeoutIOException thrown by the 
client or NotReplicatedException throen by server. Additionally, may I know 
whether the Ozone cluster used changed these configurations and whether there 
are latency improvements? I will try to come up with a simple patch for 
RATIS-2089 soon. Will take a look at HDDS-10108.

[~szetszwo] [~smeng] Another general question, may I ask that was the reason of 
the ALL_COMMITTED watch requests on the block write in the first place? 
Alternatively, can we instead release the client buffer immediately after 
successful WriteChunk/PutBlock request instead? My understanding is that the 
reply from Ratis server implies MAJORITY_COMMITTED and therefore there is no 
need to keep the buffer since it's going to be replicated to all the Datanode 
Ratis server eventually. Is it to ensure that when client (XceiverClientGrpc) 
read from any datanode, all the data would have been replicated to all the 
datanodes and therefore prevent BCSID_MISMATCH during BlockManagerImpl#getBlock?


was (Author: JIRAUSER298977):
Thank you for checking this out, glad to see that this might be useful. 

I thought of this improvement while analyzing the Ratis block write process and 
the upper bound latency for block write in case of failure (e.g. 1 datanode is 
down/unreachable). We found three relevant watch configurations that might be 
too high:
 * Client-side watch timeout: hdds.ratis.raft.client.rpc.watch.request.timeout
 ** Ratis equivalent: raft.client.rpc.watch.request.timeout
 ** Ratis default: 10s
 ** Ozone default: 180s
 * Server-side watch timeout (This is the one that throws 
NotReplicatedException): hdds.ratis.raft.server.watch.timeout
 ** Ratis equivalent: raft.server.watch.timeout
 ** Ratis default: 10s
 ** Ozone default: 180s
 * Cleint-side watch retry timeout: hdds.ratis.client.request.watch.timeout
 ** No Ratis equivalent. This is used in RequestTypeDependentRetryPolicyCreator 
to decide whether to throw RaftRetryFailureException
 ** I don't think this configuration is used frequently since WATCH request 
retry policy is RetryPolicies.noRetry(), since NotReplicatedException have 
NO_RETRY policy (only ResourceUnavailableException will be retried).
 ** Ozone default: 3m

All of them have the same values. Most likely the client-side watch timeout 
will always be triggered since the client-side watch timeout starts first. This 
might cause NotReplicatedException to never be caught by the client since 
client timeout happened before server timeout. I believe we need to reduce the 
server-side watch timeout significantly (e.g. 30s or 10s) so that 
NotReplicatedException can be thrown faster. Only then, the chance of 
NotReplicatedException caught will be higher and we can use the 
CommitInfoProto. Additionally, I believe the server-side watch timeout should 
always be shorter than the client-side watch timeout since if not, the pending 
watch request in Ratis leader might build up and cause 
ResourceUnavailableException. 

[~smeng] Did you know what Ratis exception is thrown in 
testDatanodeExclusionWithMajorityCommit? Is it TimeoutIOException thrown by the 
client or NotReplicatedException throen by server. Additionally, may I know 
whether the Ozone cluster used changed these configurations and whether there 
are latency improvements? I will try to come up with a simple patch for 
RATIS-2089 soon. Will take a look at HDDS-10108.

[~szetszwo] [~smeng] Another general question, may I ask that was the reason of 
the ALL_COMMITTED watch requests on the block write in the first place? 
Alternatively, can we instead release the client buffer immediately after 
successful WriteChunk/PutBlock request instead? My understanding is that the 
reply from Ratis server implies MAJORITY_COMMITTED and therefore there is no 
need to keep the buffer since it's going to be replicated to all the Datanode 
Ratis server eventually. Is it to ensure that when client (XceiverClientGrpc) 
read from any datanode, all the data would have been replicated to all the 
datanodes and therefore prevent BCSID_MISMATCH during BlockManagerImpl#getBlock?

> 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