in ratis-client/src/main/java/org/apache/ratis/client/impl/ClientProtoUtils.java [392:438]
static RaftClientReply toRaftClientReply(RaftClientReplyProto replyProto) {
final RaftRpcReplyProto rp = replyProto.getRpcReply();
final RaftGroupMemberId serverMemberId = ProtoUtils.toRaftGroupMemberId(rp.getReplyId(), rp.getRaftGroupId());
final RaftException e;
if (replyProto.getExceptionDetailsCase().equals(NOTLEADEREXCEPTION)) {
NotLeaderExceptionProto nleProto = replyProto.getNotLeaderException();
final RaftPeer suggestedLeader = nleProto.hasSuggestedLeader() ?
ProtoUtils.toRaftPeer(nleProto.getSuggestedLeader()) : null;
final List<RaftPeer> peers = ProtoUtils.toRaftPeers(nleProto.getPeersInConfList());
e = new NotLeaderException(serverMemberId, suggestedLeader, peers);
} else if (replyProto.getExceptionDetailsCase() == NOTREPLICATEDEXCEPTION) {
final NotReplicatedExceptionProto nre = replyProto.getNotReplicatedException();
e = new NotReplicatedException(nre.getCallId(), nre.getReplication(), nre.getLogIndex(),
replyProto.getCommitInfosList());
} else if (replyProto.getExceptionDetailsCase().equals(STATEMACHINEEXCEPTION)) {
e = toStateMachineException(serverMemberId, replyProto.getStateMachineException());
} else if (replyProto.getExceptionDetailsCase().equals(DATASTREAMEXCEPTION)) {
e = ProtoUtils.toThrowable(replyProto.getDataStreamException(), DataStreamException.class);
} else if (replyProto.getExceptionDetailsCase().equals(LEADERNOTREADYEXCEPTION)) {
LeaderNotReadyExceptionProto lnreProto = replyProto.getLeaderNotReadyException();
e = new LeaderNotReadyException(ProtoUtils.toRaftGroupMemberId(lnreProto.getServerId()));
} else if (replyProto.getExceptionDetailsCase().equals(ALREADYCLOSEDEXCEPTION)) {
e = toAlreadyClosedException(replyProto.getAlreadyClosedException());
} else if (replyProto.getExceptionDetailsCase().equals(LEADERSTEPPINGDOWNEXCEPTION)) {
e = ProtoUtils.toThrowable(replyProto.getLeaderSteppingDownException(), LeaderSteppingDownException.class);
} else if (replyProto.getExceptionDetailsCase().equals(TRANSFERLEADERSHIPEXCEPTION)) {
e = ProtoUtils.toThrowable(replyProto.getTransferLeadershipException(), TransferLeadershipException.class);
} else if (replyProto.getExceptionDetailsCase().equals(READEXCEPTION)) {
e = ProtoUtils.toThrowable(replyProto.getReadException(), ReadException.class);
} else if (replyProto.getExceptionDetailsCase().equals(READINDEXEXCEPTION)) {
e = ProtoUtils.toThrowable(replyProto.getReadIndexException(), ReadIndexException.class);
} else {
e = null;
}
return RaftClientReply.newBuilder()
.setClientId(ClientId.valueOf(rp.getRequestorId()))
.setServerId(serverMemberId)
.setCallId(rp.getCallId())
.setSuccess(rp.getSuccess())
.setMessage(toMessage(replyProto.getMessage()))
.setException(e)
.setLogIndex(replyProto.getLogIndex())
.setCommitInfos(replyProto.getCommitInfosList())
.build();
}