[
https://issues.apache.org/jira/browse/RATIS-979?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17223353#comment-17223353
]
runzhiwang commented on RATIS-979:
----------------------------------
[~szetszwo] Hi, Can several DataStreamOutput belongs to one
NettyClientStreamRpc work at the same time ? If so, there maybe exist a
potential bug.
NettyServerStreamRpc only guarantee the reply of the same DataStreamOutput is
in order by the previous.get().thenCombineAsync..., if the reply belongs to
different DataStreamOutputs, we can not guarantee the reply order. But
NettyClientStreamRpc#replies implemented by queue store all the reply future
belongs to different DataStreamOutputs, so the client maybe happen
DataStreamReply out of order when several DataStreamOutput work at the same
time.
{code:java}
final AtomicReference<CompletableFuture<?>> previous = info.getPrevious();
final CompletableFuture<?> current = previous.get()
.thenCombineAsync(JavaUtils.allOf(remoteWrites), (u, v) -> null)
.thenCombineAsync(localWrite, (v, bytesWritten) -> {
buf.release();
sendReply(remoteWrites, request, bytesWritten, ctx);
return null;
});
{code}
> Ratis streaming
> ---------------
>
> Key: RATIS-979
> URL: https://issues.apache.org/jira/browse/RATIS-979
> Project: Ratis
> Issue Type: New Feature
> Components: Streaming
> Reporter: Tsz-wo Sze
> Assignee: Tsz-wo Sze
> Priority: Major
> Attachments: RatisStreaming20200929.pdf
>
>
> In this JIRA, we design and implement Ratis Streaming with zero buffer
> copying and asynchronous event driven.
--
This message was sent by Atlassian Jira
(v8.3.4#803005)