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

Rui Wang commented on RATIS-1115:
---------------------------------

We might can leave the what are errors cases later, but discuss how to deal 
with not success reply,

To me there are two types of "not success", one is recoverable, another is not. 

The easiest way for error handling, is just fail the stream whenever see a not 
success reply. E.g. do not retry.

But seems that is not smart enough. For a stream that has streamed GBs data, 
just because of a chunk fail to write then we abort the stream sounds not 
right. In this case, we might need to introduce retries on client side. This 
will introduce challenges for ordering. OrderedStreamAsync only guarantee 
sending requests ordering. We have introduced ordering in stream server by 
thenCombineAsync and let this depends on previous.

Then how to support reties in both client side and server side? The following 
is a high level thought

For client side, two options
a. OrderedStreamAsync can only send one request a time. It does not send 
another request before seeing previous reply, and also wait for client to ack 
whether retry or send next request. The biggest drawback is this will slow 
throughput significantly. 

b. OrderedStreamAsync allows insert requests in the head of queue. So instead 
of sending requests sequentially, it allows sends retries before other normal 
firs time requests.

 

For server side, also two options
a. assume StateMachine stream implementation is able to locate a previous write 
by a id (e.g. for file it might be file name + offset), then can overwrite 
previous write. In that case, upon receiving retries, server just apply the 
retries to StateMachine and offer information that StateMachine needs. 

b. When server sees a error, it waits for retries before apply next request. We 
might depends on a unique request id within stream to identify retries.

 

Lastly, server will need to support clean up unfinish steam data in case of the 
whole stream fail to finish.

 

> NettyServerStreamRpc should handle duplicate headers
> ----------------------------------------------------
>
>                 Key: RATIS-1115
>                 URL: https://issues.apache.org/jira/browse/RATIS-1115
>             Project: Ratis
>          Issue Type: Sub-task
>            Reporter: Rui Wang
>            Assignee: Rui Wang
>            Priority: Major
>
> Current server does 
>  
> {code:java}
> if (Header) {    
>   streams.computeIfAbsent(request.getStreamId(), id ->createStream());
>   ...  
> }
> {code}
>  
>  
> It is possible that clients send multiple headers so the entry already 
> exists. We might choose to send a reply with error message without accepting 
> duplicated headers. 
>  



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to