You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@dubbo.apache.org by gu...@apache.org on 2021/09/08 05:42:04 UTC

[dubbo] branch 3.0 updated: feature: remove TripleRpcException (#8712)

This is an automated email from the ASF dual-hosted git repository.

guohao pushed a commit to branch 3.0
in repository https://gitbox.apache.org/repos/asf/dubbo.git


The following commit(s) were added to refs/heads/3.0 by this push:
     new 3aa7168  feature: remove TripleRpcException (#8712)
3aa7168 is described below

commit 3aa71682a92a26e773a579435b73704046b37fe9
Author: Smile <14...@qq.com>
AuthorDate: Wed Sep 8 13:41:53 2021 +0800

    feature: remove TripleRpcException (#8712)
    
    * add feature: remove TripleRpcException
    
    * remove TripleRpcException
    
    * fix import format
    
    * optimize rpcExceptionCodeToGrpc
    
    * Fill exception msg
    
    * add msg
    
    Co-authored-by: guohao <gu...@gmail.com>
---
 .../java/org/apache/dubbo/rpc/RpcException.java    |  4 +-
 .../dubbo/rpc/protocol/tri/ClientStream.java       |  2 +-
 .../apache/dubbo/rpc/protocol/tri/GrpcStatus.java  | 57 +++++++++++-----
 .../tri/TripleHttp2FrameServerHandler.java         | 40 +++++------
 .../dubbo/rpc/protocol/tri/TripleRpcException.java | 78 ----------------------
 .../dubbo/rpc/protocol/tri/UnaryClientStream.java  | 14 ++--
 .../dubbo/rpc/protocol/tri/UnaryServerStream.java  | 31 ++++++---
 .../rpc/protocol/tri/service/TriHealthImpl.java    |  9 +--
 8 files changed, 97 insertions(+), 138 deletions(-)

diff --git a/dubbo-rpc/dubbo-rpc-api/src/main/java/org/apache/dubbo/rpc/RpcException.java b/dubbo-rpc/dubbo-rpc-api/src/main/java/org/apache/dubbo/rpc/RpcException.java
index be6ed68..dd43be3 100644
--- a/dubbo-rpc/dubbo-rpc-api/src/main/java/org/apache/dubbo/rpc/RpcException.java
+++ b/dubbo-rpc/dubbo-rpc-api/src/main/java/org/apache/dubbo/rpc/RpcException.java
@@ -26,7 +26,8 @@ import javax.naming.LimitExceededException;
  * @see org.apache.dubbo.rpc.Invoker#invoke(Invocation)
  * @since 1.0
  */
-public /**final**/ class RpcException extends RuntimeException {
+public /**final**/
+class RpcException extends RuntimeException {
 
     public static final int UNKNOWN_EXCEPTION = 0;
     public static final int NETWORK_EXCEPTION = 1;
@@ -39,6 +40,7 @@ public /**final**/ class RpcException extends RuntimeException {
     public static final int TIMEOUT_TERMINATE = 8;
     public static final int REGISTRY_EXCEPTION = 9;
     public static final int ROUTER_CACHE_NOT_BUILD = 10;
+    public static final int METHOD_NOT_FOUND = 11;
     private static final long serialVersionUID = 7815426752583648734L;
     /**
      * RpcException cannot be extended, use error code for exception type to keep compatibility
diff --git a/dubbo-rpc/dubbo-rpc-triple/src/main/java/org/apache/dubbo/rpc/protocol/tri/ClientStream.java b/dubbo-rpc/dubbo-rpc-triple/src/main/java/org/apache/dubbo/rpc/protocol/tri/ClientStream.java
index 4e31f6e..c4242f0 100644
--- a/dubbo-rpc/dubbo-rpc-triple/src/main/java/org/apache/dubbo/rpc/protocol/tri/ClientStream.java
+++ b/dubbo-rpc/dubbo-rpc-triple/src/main/java/org/apache/dubbo/rpc/protocol/tri/ClientStream.java
@@ -70,7 +70,7 @@ public class ClientStream extends AbstractClientStream implements Stream {
                     if (GrpcStatus.Code.isOk(status.code.code)) {
                         getStreamSubscriber().onCompleted();
                     } else {
-                        getStreamSubscriber().onError(status.asException(getTrailers()));
+                        getStreamSubscriber().onError(status.asException());
                     }
                 });
             }
diff --git a/dubbo-rpc/dubbo-rpc-triple/src/main/java/org/apache/dubbo/rpc/protocol/tri/GrpcStatus.java b/dubbo-rpc/dubbo-rpc-triple/src/main/java/org/apache/dubbo/rpc/protocol/tri/GrpcStatus.java
index 68f8769..fa11ee3 100644
--- a/dubbo-rpc/dubbo-rpc-triple/src/main/java/org/apache/dubbo/rpc/protocol/tri/GrpcStatus.java
+++ b/dubbo-rpc/dubbo-rpc-triple/src/main/java/org/apache/dubbo/rpc/protocol/tri/GrpcStatus.java
@@ -22,7 +22,16 @@ import org.apache.dubbo.remoting.exchange.Response;
 import io.netty.handler.codec.http.QueryStringDecoder;
 import io.netty.handler.codec.http.QueryStringEncoder;
 
-import static io.netty.util.internal.ObjectUtil.checkNotNull;
+import org.apache.dubbo.rpc.RpcException;
+
+import static org.apache.dubbo.rpc.RpcException.FORBIDDEN_EXCEPTION;
+import static org.apache.dubbo.rpc.RpcException.LIMIT_EXCEEDED_EXCEPTION;
+import static org.apache.dubbo.rpc.RpcException.METHOD_NOT_FOUND;
+import static org.apache.dubbo.rpc.RpcException.NETWORK_EXCEPTION;
+import static org.apache.dubbo.rpc.RpcException.SERIALIZATION_EXCEPTION;
+import static org.apache.dubbo.rpc.RpcException.TIMEOUT_EXCEPTION;
+import static org.apache.dubbo.rpc.RpcException.TIMEOUT_TERMINATE;
+import static org.apache.dubbo.rpc.protocol.tri.GrpcStatus.Code.UNAVAILABLE;
 
 /**
  * See https://github.com/grpc/grpc/blob/master/doc/statuscodes.md
@@ -82,6 +91,33 @@ public class GrpcStatus {
         return status;
     }
 
+    public static GrpcStatus rpcExceptionCodeToGrpc(int rpcExceptionCode) {
+        Code code;
+        switch (rpcExceptionCode) {
+            case TIMEOUT_EXCEPTION:
+            case TIMEOUT_TERMINATE:
+                code = Code.DEADLINE_EXCEEDED;
+                break;
+            case FORBIDDEN_EXCEPTION:
+                code = Code.PERMISSION_DENIED;
+                break;
+            case LIMIT_EXCEEDED_EXCEPTION:
+            case NETWORK_EXCEPTION:
+                code = UNAVAILABLE;
+                break;
+            case METHOD_NOT_FOUND:
+                code = Code.NOT_FOUND;
+                break;
+            case SERIALIZATION_EXCEPTION:
+                code = Code.INTERNAL;
+                break;
+            default:
+                code = Code.UNKNOWN;
+                break;
+        }
+        return fromCode(code);
+    }
+
     public static String limitSizeTo4KB(String desc) {
         if (desc.length() < 4096) {
             return desc;
@@ -97,17 +133,6 @@ public class GrpcStatus {
         return QueryStringDecoder.decodeComponent(raw);
     }
 
-    public static Metadata trailersFromThrowable(Throwable t) {
-        Throwable cause = checkNotNull(t, "t");
-        while (cause != null) {
-            if (cause instanceof TripleRpcException) {
-                return ((TripleRpcException) cause).getTrailers();
-            }
-            cause = cause.getCause();
-        }
-        return null;
-    }
-
     public GrpcStatus withCause(Throwable cause) {
         return new GrpcStatus(this.code, cause, this.description);
     }
@@ -116,12 +141,8 @@ public class GrpcStatus {
         return new GrpcStatus(this.code, this.cause, description);
     }
 
-    public TripleRpcException asException() {
-        return new TripleRpcException(this);
-    }
-
-    public TripleRpcException asException(Metadata trailers) {
-        return new TripleRpcException(this, trailers);
+    public RpcException asException() {
+        return new RpcException(this.code.code, this.description, this.cause);
     }
 
     public String toMessage() {
diff --git a/dubbo-rpc/dubbo-rpc-triple/src/main/java/org/apache/dubbo/rpc/protocol/tri/TripleHttp2FrameServerHandler.java b/dubbo-rpc/dubbo-rpc-triple/src/main/java/org/apache/dubbo/rpc/protocol/tri/TripleHttp2FrameServerHandler.java
index 992e350..21ccbe1 100644
--- a/dubbo-rpc/dubbo-rpc-triple/src/main/java/org/apache/dubbo/rpc/protocol/tri/TripleHttp2FrameServerHandler.java
+++ b/dubbo-rpc/dubbo-rpc-triple/src/main/java/org/apache/dubbo/rpc/protocol/tri/TripleHttp2FrameServerHandler.java
@@ -21,6 +21,7 @@ import org.apache.dubbo.common.constants.CommonConstants;
 import org.apache.dubbo.common.logger.Logger;
 import org.apache.dubbo.common.logger.LoggerFactory;
 import org.apache.dubbo.rpc.Invoker;
+import org.apache.dubbo.rpc.RpcException;
 import org.apache.dubbo.rpc.model.FrameworkModel;
 import org.apache.dubbo.rpc.model.FrameworkServiceRepository;
 import org.apache.dubbo.rpc.model.MethodDescriptor;
@@ -42,6 +43,7 @@ import io.netty.util.ReferenceCountUtil;
 
 import java.util.List;
 
+import static org.apache.dubbo.rpc.protocol.tri.GrpcStatus.rpcExceptionCodeToGrpc;
 import static org.apache.dubbo.rpc.protocol.tri.TripleUtil.responseErr;
 import static org.apache.dubbo.rpc.protocol.tri.TripleUtil.responsePlainTextError;
 
@@ -74,11 +76,11 @@ public class TripleHttp2FrameServerHandler extends ChannelDuplexHandler {
         if (LOGGER.isWarnEnabled()) {
             LOGGER.warn("Exception in processing triple message", cause);
         }
-        if (cause instanceof TripleRpcException) {
-            TripleUtil.responseErr(ctx, ((TripleRpcException) cause).getStatus());
+        if (cause instanceof RpcException) {
+            TripleUtil.responseErr(ctx, rpcExceptionCodeToGrpc(((RpcException) cause).getCode()));
         } else {
             TripleUtil.responseErr(ctx, GrpcStatus.fromCode(GrpcStatus.Code.INTERNAL)
-                    .withDescription("Provider's error:\n" + cause.getMessage()));
+                .withDescription("Provider's error:\n" + cause.getMessage()));
         }
     }
 
@@ -97,7 +99,7 @@ public class TripleHttp2FrameServerHandler extends ChannelDuplexHandler {
         final String version = headers.contains(TripleHeaderEnum.SERVICE_VERSION.getHeader()) ? headers.get(
             TripleHeaderEnum.SERVICE_VERSION.getHeader()).toString() : null;
         final String group = headers.contains(TripleHeaderEnum.SERVICE_GROUP.getHeader()) ? headers.get(TripleHeaderEnum.SERVICE_GROUP.getHeader())
-                .toString() : null;
+            .toString() : null;
         final String key = URL.buildKey(serviceName, group, version);
         Invoker<?> invoker = PATH_RESOLVER.resolve(key);
         if (invoker == null) {
@@ -111,38 +113,38 @@ public class TripleHttp2FrameServerHandler extends ChannelDuplexHandler {
 
         if (!HttpMethod.POST.asciiName().contentEquals(headers.method())) {
             responsePlainTextError(ctx, HttpResponseStatus.METHOD_NOT_ALLOWED.code(),
-                    GrpcStatus.fromCode(GrpcStatus.Code.INTERNAL)
-                            .withDescription(String.format("Method '%s' is not supported", headers.method())));
+                GrpcStatus.fromCode(GrpcStatus.Code.INTERNAL)
+                    .withDescription(String.format("Method '%s' is not supported", headers.method())));
             return;
         }
 
         if (headers.path() == null) {
             responsePlainTextError(ctx, HttpResponseStatus.NOT_FOUND.code(),
-                    GrpcStatus.fromCode(Code.UNIMPLEMENTED.code).withDescription("Expected path but is missing"));
+                GrpcStatus.fromCode(Code.UNIMPLEMENTED.code).withDescription("Expected path but is missing"));
             return;
         }
 
         final String path = headers.path().toString();
         if (path.charAt(0) != '/') {
             responsePlainTextError(ctx, HttpResponseStatus.NOT_FOUND.code(),
-                    GrpcStatus.fromCode(Code.UNIMPLEMENTED.code)
-                            .withDescription(String.format("Expected path to start with /: %s", path)));
+                GrpcStatus.fromCode(Code.UNIMPLEMENTED.code)
+                    .withDescription(String.format("Expected path to start with /: %s", path)));
             return;
         }
 
         final CharSequence contentType = HttpUtil.getMimeType(headers.get(HttpHeaderNames.CONTENT_TYPE));
         if (contentType == null) {
             responsePlainTextError(ctx, HttpResponseStatus.UNSUPPORTED_MEDIA_TYPE.code(),
-                    GrpcStatus.fromCode(GrpcStatus.Code.INTERNAL.code)
-                            .withDescription("Content-Type is missing from the request"));
+                GrpcStatus.fromCode(GrpcStatus.Code.INTERNAL.code)
+                    .withDescription("Content-Type is missing from the request"));
             return;
         }
 
         final String contentString = contentType.toString();
         if (!TripleUtil.supportContentType(contentString)) {
             responsePlainTextError(ctx, HttpResponseStatus.UNSUPPORTED_MEDIA_TYPE.code(),
-                    GrpcStatus.fromCode(Code.INTERNAL.code)
-                            .withDescription(String.format("Content-Type '%s' is not supported", contentString)));
+                GrpcStatus.fromCode(Code.INTERNAL.code)
+                    .withDescription(String.format("Content-Type '%s' is not supported", contentString)));
             return;
         }
 
@@ -158,14 +160,14 @@ public class TripleHttp2FrameServerHandler extends ChannelDuplexHandler {
         final Invoker<?> invoker = getInvoker(headers, serviceName);
         if (invoker == null) {
             responseErr(ctx,
-                    GrpcStatus.fromCode(Code.UNIMPLEMENTED).withDescription("Service not found:" + serviceName));
+                GrpcStatus.fromCode(Code.UNIMPLEMENTED).withDescription("Service not found:" + serviceName));
             return;
         }
         FrameworkServiceRepository repo = frameworkModel.getServiceRepository();
         ProviderModel providerModel = repo.lookupExportedService(invoker.getUrl().getServiceKey());
         if (providerModel == null || providerModel.getServiceModel() == null) {
             responseErr(ctx,
-                    GrpcStatus.fromCode(Code.UNIMPLEMENTED).withDescription("Service not found:" + serviceName));
+                GrpcStatus.fromCode(Code.UNIMPLEMENTED).withDescription("Service not found:" + serviceName));
             return;
         }
 
@@ -180,7 +182,7 @@ public class TripleHttp2FrameServerHandler extends ChannelDuplexHandler {
             methodDescriptors = providerModel.getServiceModel().getMethods(methodName);
             if (methodDescriptors == null || methodDescriptors.isEmpty()) {
                 responseErr(ctx, GrpcStatus.fromCode(Code.UNIMPLEMENTED)
-                        .withDescription("Method :" + methodName + " not found of service:" + serviceName));
+                    .withDescription("Method :" + methodName + " not found of service:" + serviceName));
                 return;
             }
             if (methodDescriptors.size() == 1) {
@@ -194,9 +196,9 @@ public class TripleHttp2FrameServerHandler extends ChannelDuplexHandler {
             stream = AbstractServerStream.unary(invoker.getUrl());
         }
         stream.service(providerModel.getServiceModel())
-                .invoker(invoker)
-                .methodName(methodName)
-                .subscribe(new ServerTransportObserver(ctx));
+            .invoker(invoker)
+            .methodName(methodName)
+            .subscribe(new ServerTransportObserver(ctx));
         if (methodDescriptor != null) {
             stream.method(methodDescriptor);
         } else {
diff --git a/dubbo-rpc/dubbo-rpc-triple/src/main/java/org/apache/dubbo/rpc/protocol/tri/TripleRpcException.java b/dubbo-rpc/dubbo-rpc-triple/src/main/java/org/apache/dubbo/rpc/protocol/tri/TripleRpcException.java
deleted file mode 100644
index 85a2c56..0000000
--- a/dubbo-rpc/dubbo-rpc-triple/src/main/java/org/apache/dubbo/rpc/protocol/tri/TripleRpcException.java
+++ /dev/null
@@ -1,78 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.dubbo.rpc.protocol.tri;
-
-import org.apache.dubbo.rpc.RpcException;
-
-public class TripleRpcException extends RpcException {
-    private final GrpcStatus status;
-    private final Metadata trailers;
-    private final boolean fillInStackTrace;
-    private int code;
-
-    public TripleRpcException(int code, String msg) {
-        this(code, msg, null);
-    }
-
-    public TripleRpcException(int code, String msg, Metadata trailers) {
-        super(msg);
-        this.code = code;
-        this.status = null;
-        this.trailers = trailers;
-        this.fillInStackTrace = false;
-    }
-
-    public TripleRpcException(GrpcStatus status) {
-        this(status, null);
-    }
-
-    public TripleRpcException(GrpcStatus status, Metadata trailers) {
-        this(status, trailers, true);
-    }
-
-    public TripleRpcException(GrpcStatus status, Metadata trailers, boolean fillInStackTrace) {
-        super(status.description, status.cause);
-        this.status = status;
-        this.trailers = trailers;
-        this.fillInStackTrace = fillInStackTrace;
-        this.code = status.code.code;
-        fillInStackTrace();
-    }
-
-    @Override
-    public synchronized Throwable fillInStackTrace() {
-        return fillInStackTrace ? super.fillInStackTrace() : this;
-    }
-
-    @Override
-    public int getCode() {
-        return code;
-    }
-
-    @Override
-    public void setCode(int code) {
-        this.code = code;
-    }
-
-    public GrpcStatus getStatus() {
-        return status;
-    }
-
-    public Metadata getTrailers() {
-        return trailers;
-    }
-}
diff --git a/dubbo-rpc/dubbo-rpc-triple/src/main/java/org/apache/dubbo/rpc/protocol/tri/UnaryClientStream.java b/dubbo-rpc/dubbo-rpc-triple/src/main/java/org/apache/dubbo/rpc/protocol/tri/UnaryClientStream.java
index b16f975..7f40727 100644
--- a/dubbo-rpc/dubbo-rpc-triple/src/main/java/org/apache/dubbo/rpc/protocol/tri/UnaryClientStream.java
+++ b/dubbo-rpc/dubbo-rpc-triple/src/main/java/org/apache/dubbo/rpc/protocol/tri/UnaryClientStream.java
@@ -27,6 +27,8 @@ import com.google.protobuf.Any;
 import com.google.rpc.DebugInfo;
 import com.google.rpc.Status;
 
+import org.apache.dubbo.rpc.RpcException;
+
 import java.util.List;
 import java.util.Map;
 
@@ -66,8 +68,8 @@ public class UnaryClientStream extends AbstractClientStream implements Stream {
                     DefaultFuture2.received(getConnection(), response);
                 } catch (Exception e) {
                     final GrpcStatus status = GrpcStatus.fromCode(GrpcStatus.Code.INTERNAL)
-                            .withCause(e)
-                            .withDescription("Failed to deserialize response");
+                        .withCause(e)
+                        .withDescription("Failed to deserialize response");
                     onError(status);
                 }
             });
@@ -76,7 +78,7 @@ public class UnaryClientStream extends AbstractClientStream implements Stream {
         @Override
         protected void onError(GrpcStatus status) {
             // run in callback executor will truncate exception stack and avoid blocking netty's event loop
-            execute(()-> {
+            execute(() -> {
                 Response response = new Response(getRequest().getId(), TripleConstant.TRI_VERSION);
                 response.setErrorMessage(status.description);
                 final AppResponse result = new AppResponse();
@@ -111,11 +113,11 @@ public class UnaryClientStream extends AbstractClientStream implements Stream {
                 // get common exception from DebugInfo
                 DebugInfo debugInfo = (DebugInfo) classObjectMap.get(DebugInfo.class);
                 if (debugInfo == null) {
-                    return new TripleRpcException(statusDetail.getCode(),
-                            statusDetail.getMessage(), metadata);
+                    return new RpcException(statusDetail.getCode(),
+                        statusDetail.getMessage());
                 }
                 String msg = ExceptionUtils.getStackFrameString(debugInfo.getStackEntriesList());
-                return new TripleRpcException(statusDetail.getCode(), msg, metadata);
+                return new RpcException(statusDetail.getCode(), msg);
             } finally {
                 ClassLoadUtil.switchContextLoader(tccl);
             }
diff --git a/dubbo-rpc/dubbo-rpc-triple/src/main/java/org/apache/dubbo/rpc/protocol/tri/UnaryServerStream.java b/dubbo-rpc/dubbo-rpc-triple/src/main/java/org/apache/dubbo/rpc/protocol/tri/UnaryServerStream.java
index ca03aa4..2140463 100644
--- a/dubbo-rpc/dubbo-rpc-triple/src/main/java/org/apache/dubbo/rpc/protocol/tri/UnaryServerStream.java
+++ b/dubbo-rpc/dubbo-rpc-triple/src/main/java/org/apache/dubbo/rpc/protocol/tri/UnaryServerStream.java
@@ -23,6 +23,7 @@ import org.apache.dubbo.remoting.TimeoutException;
 import org.apache.dubbo.rpc.AppResponse;
 import org.apache.dubbo.rpc.Result;
 import org.apache.dubbo.rpc.RpcContext;
+import org.apache.dubbo.rpc.RpcException;
 import org.apache.dubbo.rpc.RpcInvocation;
 
 import io.netty.handler.codec.http.HttpHeaderNames;
@@ -32,6 +33,8 @@ import java.util.concurrent.CompletionStage;
 import java.util.function.BiConsumer;
 import java.util.function.Function;
 
+import static org.apache.dubbo.rpc.protocol.tri.GrpcStatus.rpcExceptionCodeToGrpc;
+
 public class UnaryServerStream extends AbstractServerStream implements Stream {
 
     protected UnaryServerStream(URL url) {
@@ -60,7 +63,7 @@ public class UnaryServerStream extends AbstractServerStream implements Stream {
                 execute(this::invoke);
             } else {
                 onError(GrpcStatus.fromCode(GrpcStatus.Code.INTERNAL)
-                        .withDescription("Missing request data"));
+                    .withDescription("Missing request data"));
             }
         }
 
@@ -78,7 +81,7 @@ public class UnaryServerStream extends AbstractServerStream implements Stream {
             } catch (Throwable t) {
                 LOGGER.warn("Exception processing triple message", t);
                 transportError(GrpcStatus.fromCode(GrpcStatus.Code.INTERNAL)
-                        .withDescription("Decode request failed:" + t.getMessage()));
+                    .withDescription("Decode request failed:" + t.getMessage()));
                 return;
             }
 
@@ -98,11 +101,15 @@ public class UnaryServerStream extends AbstractServerStream implements Stream {
                 try {
                     if (response.hasException()) {
                         final Throwable exception = response.getException();
-                        if (exception instanceof TripleRpcException) {
-                            transportError(((TripleRpcException) exception).getStatus(), response.getObjectAttachments());
+                        if (exception instanceof RpcException) {
+                            transportError(rpcExceptionCodeToGrpc(((RpcException) exception).getCode())
+                                .withCause(exception), response.getObjectAttachments());
+                            final GrpcStatus status = rpcExceptionCodeToGrpc(((RpcException) exception).getCode())
+                                    .withCause(exception);
+                            transportError(status, response.getObjectAttachments());
                         } else {
                             transportError(GrpcStatus.fromCode(GrpcStatus.Code.UNKNOWN)
-                                    .withCause(exception), response.getObjectAttachments());
+                                .withCause(exception), response.getObjectAttachments());
                         }
                         return;
                     }
@@ -114,7 +121,7 @@ public class UnaryServerStream extends AbstractServerStream implements Stream {
                     final byte[] data;
                     try {
                         ClassLoadUtil.switchContextLoader(
-                                getProviderModel().getServiceInterfaceClass().getClassLoader());
+                            getProviderModel().getServiceInterfaceClass().getClassLoader());
                         data = encodeResponse(response.getValue());
                     } finally {
                         ClassLoadUtil.switchContextLoader(tccl);
@@ -122,7 +129,7 @@ public class UnaryServerStream extends AbstractServerStream implements Stream {
                     getTransportSubscriber().onData(data, false);
 
                     Metadata trailers = new DefaultMetadata()
-                            .put(TripleHeaderEnum.STATUS_KEY.getHeader(), Integer.toString(GrpcStatus.Code.OK.code));
+                        .put(TripleHeaderEnum.STATUS_KEY.getHeader(), Integer.toString(GrpcStatus.Code.OK.code));
                     final Map<String, Object> attachments = response.getObjectAttachments();
                     if (attachments != null) {
                         convertAttachment(trailers, attachments);
@@ -130,12 +137,14 @@ public class UnaryServerStream extends AbstractServerStream implements Stream {
                     getTransportSubscriber().onMetadata(trailers, true);
                 } catch (Throwable e) {
                     LOGGER.warn("Exception processing triple message", e);
-                    if (e instanceof TripleRpcException) {
-                        transportError(((TripleRpcException) e).getStatus(), response.getObjectAttachments());
+                    if (e instanceof RpcException) {
+                        final GrpcStatus status = rpcExceptionCodeToGrpc(((RpcException) e).getCode())
+                                .withCause(e);
+                        transportError(status, response.getObjectAttachments());
                     } else {
                         transportError(GrpcStatus.fromCode(GrpcStatus.Code.UNKNOWN)
-                                .withDescription("Exception occurred in provider's execution:" + e.getMessage())
-                                .withCause(e), response.getObjectAttachments());
+                            .withDescription("Exception occurred in provider's execution:" + e.getMessage())
+                            .withCause(e), response.getObjectAttachments());
                     }
                 }
             };
diff --git a/dubbo-rpc/dubbo-rpc-triple/src/main/java/org/apache/dubbo/rpc/protocol/tri/service/TriHealthImpl.java b/dubbo-rpc/dubbo-rpc-triple/src/main/java/org/apache/dubbo/rpc/protocol/tri/service/TriHealthImpl.java
index af313ff..3919b1a 100644
--- a/dubbo-rpc/dubbo-rpc-triple/src/main/java/org/apache/dubbo/rpc/protocol/tri/service/TriHealthImpl.java
+++ b/dubbo-rpc/dubbo-rpc-triple/src/main/java/org/apache/dubbo/rpc/protocol/tri/service/TriHealthImpl.java
@@ -20,9 +20,9 @@ package org.apache.dubbo.rpc.protocol.tri.service;
 import grpc.health.v1.Health;
 import grpc.health.v1.HealthCheckRequest;
 import grpc.health.v1.HealthCheckResponse;
+
 import org.apache.dubbo.common.stream.StreamObserver;
-import org.apache.dubbo.rpc.protocol.tri.GrpcStatus;
-import org.apache.dubbo.rpc.protocol.tri.TripleRpcException;
+import org.apache.dubbo.rpc.RpcException;
 
 import java.util.HashMap;
 import java.util.IdentityHashMap;
@@ -31,6 +31,8 @@ import java.util.concurrent.ConcurrentHashMap;
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
+import static org.apache.dubbo.rpc.RpcException.METHOD_NOT_FOUND;
+
 public class TriHealthImpl implements Health {
 
     private static final Logger logger = Logger.getLogger(TriHealthImpl.class.getName());
@@ -63,8 +65,7 @@ public class TriHealthImpl implements Health {
         if (status != null) {
             return HealthCheckResponse.newBuilder().setStatus(status).build();
         }
-        throw new TripleRpcException(GrpcStatus.fromCode(GrpcStatus.Code.NOT_FOUND)
-            .withDescription("unknown service " + request.getService()));
+        throw new RpcException(METHOD_NOT_FOUND, "unknown service " + request.getService());
     }
 
     @Override