in ratis-netty/src/main/java/org/apache/ratis/netty/server/NettyRpcService.java [184:303]
RaftNettyServerReplyProto handle(RaftNettyServerRequestProto proto) {
RaftRpcRequestProto rpcRequest = null;
try {
switch (proto.getRaftNettyServerRequestCase()) {
case REQUESTVOTEREQUEST:
final RequestVoteRequestProto request = proto.getRequestVoteRequest();
rpcRequest = request.getServerRequest();
final RequestVoteReplyProto reply = server.requestVote(request);
return RaftNettyServerReplyProto.newBuilder()
.setRequestVoteReply(reply)
.build();
case TRANSFERLEADERSHIPREQUEST:
final TransferLeadershipRequestProto transferLeadershipRequest = proto.getTransferLeadershipRequest();
rpcRequest = transferLeadershipRequest.getRpcRequest();
final RaftClientReply transferLeadershipReply = server.transferLeadership(
ClientProtoUtils.toTransferLeadershipRequest(transferLeadershipRequest));
return RaftNettyServerReplyProto.newBuilder()
.setRaftClientReply(ClientProtoUtils.toRaftClientReplyProto(transferLeadershipReply))
.build();
case STARTLEADERELECTIONREQUEST:
final StartLeaderElectionRequestProto startLeaderElectionRequest = proto.getStartLeaderElectionRequest();
rpcRequest = startLeaderElectionRequest.getServerRequest();
final StartLeaderElectionReplyProto startLeaderElectionReply =
server.startLeaderElection(startLeaderElectionRequest);
return RaftNettyServerReplyProto.newBuilder().setStartLeaderElectionReply(startLeaderElectionReply).build();
case SNAPSHOTMANAGEMENTREQUEST:
final SnapshotManagementRequestProto snapshotManagementRequest = proto.getSnapshotManagementRequest();
rpcRequest = snapshotManagementRequest.getRpcRequest();
final RaftClientReply snapshotManagementReply = server.snapshotManagement(
ClientProtoUtils.toSnapshotManagementRequest(snapshotManagementRequest));
return RaftNettyServerReplyProto.newBuilder()
.setRaftClientReply(ClientProtoUtils.toRaftClientReplyProto(snapshotManagementReply))
.build();
case LEADERELECTIONMANAGEMENTREQUEST:
final LeaderElectionManagementRequestProto leaderElectionManagementRequest =
proto.getLeaderElectionManagementRequest();
rpcRequest = leaderElectionManagementRequest.getRpcRequest();
final RaftClientReply leaderElectionManagementReply = server.leaderElectionManagement(
ClientProtoUtils.toLeaderElectionManagementRequest(leaderElectionManagementRequest));
return RaftNettyServerReplyProto.newBuilder()
.setRaftClientReply(ClientProtoUtils.toRaftClientReplyProto(leaderElectionManagementReply))
.build();
case APPENDENTRIESREQUEST:
final AppendEntriesRequestProto appendEntriesRequest = proto.getAppendEntriesRequest();
rpcRequest = appendEntriesRequest.getServerRequest();
final AppendEntriesReplyProto appendEntriesReply = server.appendEntries(appendEntriesRequest);
return RaftNettyServerReplyProto.newBuilder()
.setAppendEntriesReply(appendEntriesReply)
.build();
case INSTALLSNAPSHOTREQUEST:
final InstallSnapshotRequestProto installSnapshotRequest = proto.getInstallSnapshotRequest();
rpcRequest = installSnapshotRequest.getServerRequest();
final InstallSnapshotReplyProto installSnapshotReply = server.installSnapshot(installSnapshotRequest);
return RaftNettyServerReplyProto.newBuilder()
.setInstallSnapshotReply(installSnapshotReply)
.build();
case RAFTCLIENTREQUEST:
final RaftClientRequestProto raftClientRequest = proto.getRaftClientRequest();
rpcRequest = raftClientRequest.getRpcRequest();
final RaftClientReply raftClientReply = server.submitClientRequest(
ClientProtoUtils.toRaftClientRequest(raftClientRequest));
return RaftNettyServerReplyProto.newBuilder()
.setRaftClientReply(ClientProtoUtils.toRaftClientReplyProto(raftClientReply))
.build();
case SETCONFIGURATIONREQUEST:
final SetConfigurationRequestProto configurationRequest = proto.getSetConfigurationRequest();
rpcRequest = configurationRequest.getRpcRequest();
final RaftClientReply configurationReply = server.setConfiguration(
ClientProtoUtils.toSetConfigurationRequest(configurationRequest));
return RaftNettyServerReplyProto.newBuilder()
.setRaftClientReply(ClientProtoUtils.toRaftClientReplyProto(configurationReply))
.build();
case GROUPMANAGEMENTREQUEST:
final GroupManagementRequestProto groupManagementRequest = proto.getGroupManagementRequest();
rpcRequest = groupManagementRequest.getRpcRequest();
final RaftClientReply groupManagementReply = server.groupManagement(
ClientProtoUtils.toGroupManagementRequest(groupManagementRequest));
return RaftNettyServerReplyProto.newBuilder()
.setRaftClientReply(ClientProtoUtils.toRaftClientReplyProto(groupManagementReply))
.build();
case GROUPLISTREQUEST:
final GroupListRequestProto groupListRequest = proto.getGroupListRequest();
rpcRequest = groupListRequest.getRpcRequest();
final GroupListReply groupListReply = server.getGroupList(
ClientProtoUtils.toGroupListRequest(groupListRequest));
return RaftNettyServerReplyProto.newBuilder()
.setGroupListReply(ClientProtoUtils.toGroupListReplyProto(groupListReply))
.build();
case GROUPINFOREQUEST:
final GroupInfoRequestProto groupInfoRequest = proto.getGroupInfoRequest();
rpcRequest = groupInfoRequest.getRpcRequest();
final GroupInfoReply groupInfoReply = server.getGroupInfo(
ClientProtoUtils.toGroupInfoRequest(groupInfoRequest));
return RaftNettyServerReplyProto.newBuilder()
.setGroupInfoReply(ClientProtoUtils.toGroupInfoReplyProto(groupInfoReply))
.build();
case RAFTNETTYSERVERREQUEST_NOT_SET:
throw new IllegalArgumentException("Request case not set in proto: "
+ proto.getRaftNettyServerRequestCase());
default:
throw new UnsupportedOperationException("Request case not supported: "
+ proto.getRaftNettyServerRequestCase());
}
} catch (IOException ioe) {
return toRaftNettyServerReplyProto(
Objects.requireNonNull(rpcRequest, "rpcRequest = null"), ioe);
}
}