You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ratis.apache.org by "Lokesh Jain (JIRA)" <ji...@apache.org> on 2017/11/12 11:51:00 UTC
[jira] [Comment Edited] (RATIS-113) Add Async send interface to
RaftClient
[ https://issues.apache.org/jira/browse/RATIS-113?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16248829#comment-16248829 ]
Lokesh Jain edited comment on RATIS-113 at 11/12/17 11:50 AM:
--------------------------------------------------------------
Thanks for the review [~szetszwo]. v3 patch addresses the above comments. I have removed the implementation for configuration and used semaphore instead of ArrayBlockingQueue.
getCause() in RaftClientRpc#sendRequestAsync is a requirement as Hadoop Rpc throws Remote exception. And code logic in RaftClientImpl#sendRequestAsync checks the exception class.
{code:java}
if (cause instanceof RaftException) {
return new RaftClientReply(request, (RaftException) cause);
} else if (cause instanceof IOException) {
handleIOException(request, (IOException) cause, null);
}
{code}
The send api throws StateMachineException and GroupMismatchException as in RaftClientImpl#sendRequest.
{code:java}
try {
reply = clientRpc.sendRequest(request);
} catch (GroupMismatchException gme) {
throw gme;
{code}
and
{code:java}
} else if (reply.hasStateMachineException()) {
throw reply.getStateMachineException();
{code}
Therefore I have changed the logic of sendAsync to complete exceptionally in case of
StateMachineException and GroupMismatchException.
was (Author: ljain):
Thanks for the review [~szetszwo]. v3 patch addresses the above comments. I have removed the implementation for configuration and used semaphore instead of ArrayBlockingQueue.
RaftClientRpc#sendRequestAsync is a requirement as Hadoop Rpc throws Remote exception. And code logic in RaftClientImpl#sendRequestAsync checks the exception class.
{code:java}
if (cause instanceof RaftException) {
return new RaftClientReply(request, (RaftException) cause);
} else if (cause instanceof IOException) {
handleIOException(request, (IOException) cause, null);
}
{code}
The send api throws StateMachineException and GroupMismatchException as in RaftClientImpl#sendRequest.
{code:java}
try {
reply = clientRpc.sendRequest(request);
} catch (GroupMismatchException gme) {
throw gme;
{code}
and
{code:java}
} else if (reply.hasStateMachineException()) {
throw reply.getStateMachineException();
{code}
Therefore I have changed the logic of sendAsync to complete exceptionally in case of
StateMachineException and GroupMismatchException.
> Add Async send interface to RaftClient
> --------------------------------------
>
> Key: RATIS-113
> URL: https://issues.apache.org/jira/browse/RATIS-113
> Project: Ratis
> Issue Type: Bug
> Reporter: Mukul Kumar Singh
> Assignee: Lokesh Jain
> Attachments: RATIS-113.001.patch, RATIS-113.002.patch, RATIS-113.003.patch
>
>
> Raft Client currently only has a sync interface, an sync interface is needed for ozone
--
This message was sent by Atlassian JIRA
(v6.4.14#64029)