You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ratis.apache.org by sz...@apache.org on 2019/12/05 22:05:18 UTC
[incubator-ratis] branch master updated: RATIS-765.
ClientProtoUtils#toRaftClientReplyProto should handle
AlreadyClosedException. Contributed by Hanisha Koneru
This is an automated email from the ASF dual-hosted git repository.
szetszwo pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-ratis.git
The following commit(s) were added to refs/heads/master by this push:
new 79d6004 RATIS-765. ClientProtoUtils#toRaftClientReplyProto should handle AlreadyClosedException. Contributed by Hanisha Koneru
79d6004 is described below
commit 79d6004dc19cdfff141751e373169f2af315265f
Author: Tsz Wo Nicholas Sze <sz...@apache.org>
AuthorDate: Thu Dec 5 14:04:30 2019 -0800
RATIS-765. ClientProtoUtils#toRaftClientReplyProto should handle AlreadyClosedException. Contributed by Hanisha Koneru
---
.../apache/ratis/client/impl/ClientProtoUtils.java | 39 ++++++++++++++++++++--
ratis-proto/src/main/proto/Raft.proto | 7 ++++
2 files changed, 44 insertions(+), 2 deletions(-)
diff --git a/ratis-client/src/main/java/org/apache/ratis/client/impl/ClientProtoUtils.java b/ratis-client/src/main/java/org/apache/ratis/client/impl/ClientProtoUtils.java
index e3d4dea..fc9f578 100644
--- a/ratis-client/src/main/java/org/apache/ratis/client/impl/ClientProtoUtils.java
+++ b/ratis-client/src/main/java/org/apache/ratis/client/impl/ClientProtoUtils.java
@@ -27,8 +27,8 @@ import org.apache.ratis.util.ReflectionUtils;
import java.util.List;
import java.util.stream.Collectors;
-import static org.apache.ratis.proto.RaftProtos.RaftClientReplyProto
- .ExceptionDetailsCase.LEADERNOTREADYEXCEPTION;
+import static org.apache.ratis.proto.RaftProtos.RaftClientReplyProto.ExceptionDetailsCase.ALREADYCLOSEDEXCEPTION;
+import static org.apache.ratis.proto.RaftProtos.RaftClientReplyProto.ExceptionDetailsCase.LEADERNOTREADYEXCEPTION;
import static org.apache.ratis.proto.RaftProtos.RaftClientReplyProto.ExceptionDetailsCase.NOTLEADEREXCEPTION;
import static org.apache.ratis.proto.RaftProtos.RaftClientReplyProto.ExceptionDetailsCase.NOTREPLICATEDEXCEPTION;
import static org.apache.ratis.proto.RaftProtos.RaftClientReplyProto.ExceptionDetailsCase.STATEMACHINEEXCEPTION;
@@ -196,6 +196,16 @@ public interface ClientProtoUtils {
b.setLeaderNotReadyException(lnreBuilder);
}
+ final AlreadyClosedException ace = reply.getAlreadyClosedException();
+ if (ace != null) {
+ final Throwable t = ace.getCause() != null ? ace.getCause() : ace;
+ AlreadyClosedExceptionProto.Builder aceBuilder = AlreadyClosedExceptionProto.newBuilder()
+ .setExceptionClassName(t.getClass().getName())
+ .setErrorMsg(ace.getMessage())
+ .setStacktrace(ProtoUtils.writeObject2ByteString(ace.getStackTrace()));
+ b.setAlreadyClosedException(aceBuilder);
+ }
+
final RaftClientReplyProto serialized = b.build();
final RaftException e = reply.getException();
if (e != null) {
@@ -264,6 +274,10 @@ public interface ClientProtoUtils {
} else if (replyProto.getExceptionDetailsCase().equals(LEADERNOTREADYEXCEPTION)) {
LeaderNotReadyExceptionProto lnreProto = replyProto.getLeaderNotReadyException();
e = new LeaderNotReadyException(ProtoUtils.toRaftGroupMemberId(lnreProto.getServerId()));
+ } else if (replyProto.getExceptionDetailsCase().equals(ALREADYCLOSEDEXCEPTION)) {
+ AlreadyClosedExceptionProto aceProto = replyProto.getAlreadyClosedException();
+ e = wrapAlreadyClosedException(aceProto.getExceptionClassName(),
+ aceProto.getErrorMsg(), aceProto.getStacktrace());
} else {
e = null;
}
@@ -319,6 +333,27 @@ public interface ClientProtoUtils {
return sme;
}
+ static AlreadyClosedException wrapAlreadyClosedException(
+ String className, String errorMsg, ByteString stackTraceBytes) {
+ AlreadyClosedException ace;
+ if (className == null) {
+ ace = new AlreadyClosedException(errorMsg);
+ } else {
+ try {
+ Class<?> clazz = Class.forName(className);
+ final Exception e = ReflectionUtils.instantiateException(
+ clazz.asSubclass(Exception.class), errorMsg, null);
+ ace = new AlreadyClosedException(errorMsg, e);
+ } catch (Exception e) {
+ ace = new AlreadyClosedException(className + ": " + errorMsg);
+ }
+ }
+ StackTraceElement[] stacktrace =
+ (StackTraceElement[]) ProtoUtils.toObject(stackTraceBytes);
+ ace.setStackTrace(stacktrace);
+ return ace;
+ }
+
static Message toMessage(final ClientMessageEntryProto p) {
return Message.valueOf(p.getContent());
}
diff --git a/ratis-proto/src/main/proto/Raft.proto b/ratis-proto/src/main/proto/Raft.proto
index 11c6bc5..e55f020 100644
--- a/ratis-proto/src/main/proto/Raft.proto
+++ b/ratis-proto/src/main/proto/Raft.proto
@@ -291,6 +291,12 @@ message StateMachineExceptionProto {
bytes stacktrace = 3;
}
+message AlreadyClosedExceptionProto {
+ string exceptionClassName = 1;
+ string errorMsg = 2;
+ bytes stacktrace = 3;
+}
+
message RaftClientReplyProto {
RaftRpcReplyProto rpcReply = 1;
ClientMessageEntryProto message = 2;
@@ -300,6 +306,7 @@ message RaftClientReplyProto {
NotReplicatedExceptionProto notReplicatedException = 4;
StateMachineExceptionProto stateMachineException = 5;
LeaderNotReadyExceptionProto leaderNotReadyException = 6;
+ AlreadyClosedExceptionProto alreadyClosedException = 7;
}
uint64 logIndex = 14; // When the request is a write request and the reply is success, the log index of the transaction