This is an automated email from the ASF dual-hosted git repository.
tanxinyu pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ratis.git
The following commit(s) were added to refs/heads/master by this push:
new a4826e239 RATIS-2185. Improve gRPC log messages debugability. (#1186)
a4826e239 is described below
commit a4826e2393a529cbe60d4e76084f2d5bf28f594b
Author: Tsz-Wo Nicholas Sze <[email protected]>
AuthorDate: Sun Dec 1 18:01:21 2024 -0800
RATIS-2185. Improve gRPC log messages debugability. (#1186)
---
.../main/java/org/apache/ratis/util/LogUtils.java | 8 ++++++-
.../main/java/org/apache/ratis/grpc/GrpcUtil.java | 7 ------
.../grpc/server/GrpcAdminProtocolService.java | 27 +++++++++++++++-------
3 files changed, 26 insertions(+), 16 deletions(-)
diff --git a/ratis-common/src/main/java/org/apache/ratis/util/LogUtils.java
b/ratis-common/src/main/java/org/apache/ratis/util/LogUtils.java
index d29f1e56e..63bb08acc 100644
--- a/ratis-common/src/main/java/org/apache/ratis/util/LogUtils.java
+++ b/ratis-common/src/main/java/org/apache/ratis/util/LogUtils.java
@@ -118,7 +118,13 @@ public interface LogUtils {
if (log.isWarnEnabled()) {
if (ReflectionUtils.isInstance(t, exceptionClasses)) {
// do not print stack trace for known exceptions.
- log.warn(message.get() + ": " + t);
+ final StringBuilder b = new StringBuilder()
+ .append(message.get())
+ .append(": ").append(t);
+ for(Throwable cause = t.getCause(); cause != null; cause =
cause.getCause()) {
+ b.append("\n Caused by: ").append(cause);
+ }
+ log.warn(b.toString());
} else {
log.warn(message.get(), t);
}
diff --git a/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcUtil.java
b/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcUtil.java
index 5a7f00318..e17d17bff 100644
--- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcUtil.java
+++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcUtil.java
@@ -161,13 +161,6 @@ public interface GrpcUtil {
return e;
}
- static <REPLY, REPLY_PROTO> void asyncCall(
- StreamObserver<REPLY_PROTO> responseObserver,
- CheckedSupplier<CompletableFuture<REPLY>, IOException> supplier,
- Function<REPLY, REPLY_PROTO> toProto) {
- asyncCall(responseObserver, supplier, toProto, throwable -> {});
- }
-
static <REPLY, REPLY_PROTO> void asyncCall(
StreamObserver<REPLY_PROTO> responseObserver,
CheckedSupplier<CompletableFuture<REPLY>, IOException> supplier,
diff --git
a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcAdminProtocolService.java
b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcAdminProtocolService.java
index feb780302..f6a6cfba6 100644
---
a/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcAdminProtocolService.java
+++
b/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcAdminProtocolService.java
@@ -1,4 +1,4 @@
-/**
+/*
* 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
@@ -32,8 +32,12 @@ import
org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver;
import org.apache.ratis.proto.RaftProtos.RaftClientReplyProto;
import org.apache.ratis.proto.RaftProtos.GroupManagementRequestProto;
import
org.apache.ratis.proto.grpc.AdminProtocolServiceGrpc.AdminProtocolServiceImplBase;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
public class GrpcAdminProtocolService extends AdminProtocolServiceImplBase {
+ static final Logger LOG =
LoggerFactory.getLogger(GrpcAdminProtocolService.class);
+
private final AdminAsynchronousProtocol protocol;
public GrpcAdminProtocolService(AdminAsynchronousProtocol protocol) {
@@ -45,7 +49,8 @@ public class GrpcAdminProtocolService extends
AdminProtocolServiceImplBase {
StreamObserver<RaftClientReplyProto> responseObserver) {
final GroupManagementRequest request =
ClientProtoUtils.toGroupManagementRequest(proto);
GrpcUtil.asyncCall(responseObserver, () ->
protocol.groupManagementAsync(request),
- ClientProtoUtils::toRaftClientReplyProto);
+ ClientProtoUtils::toRaftClientReplyProto,
+ t -> LOG.warn("Failed groupManagement: {}, {}", proto.getOpCase(),
request, t));
}
@Override
@@ -53,14 +58,16 @@ public class GrpcAdminProtocolService extends
AdminProtocolServiceImplBase {
StreamObserver<GroupListReplyProto> responseObserver) {
final GroupListRequest request =
ClientProtoUtils.toGroupListRequest(proto);
GrpcUtil.asyncCall(responseObserver, () ->
protocol.getGroupListAsync(request),
- ClientProtoUtils::toGroupListReplyProto);
+ ClientProtoUtils::toGroupListReplyProto,
+ t -> LOG.warn("Failed to groupList: {}", request, t));
}
@Override
public void groupInfo(GroupInfoRequestProto proto,
StreamObserver<GroupInfoReplyProto> responseObserver) {
final GroupInfoRequest request =
ClientProtoUtils.toGroupInfoRequest(proto);
GrpcUtil.asyncCall(responseObserver, () ->
protocol.getGroupInfoAsync(request),
- ClientProtoUtils::toGroupInfoReplyProto);
+ ClientProtoUtils::toGroupInfoReplyProto,
+ t -> LOG.warn("Failed to groupInfo: {}", request, t));
}
@Override
@@ -68,7 +75,8 @@ public class GrpcAdminProtocolService extends
AdminProtocolServiceImplBase {
StreamObserver<RaftClientReplyProto> responseObserver) {
final SetConfigurationRequest request =
ClientProtoUtils.toSetConfigurationRequest(proto);
GrpcUtil.asyncCall(responseObserver, () ->
protocol.setConfigurationAsync(request),
- ClientProtoUtils::toRaftClientReplyProto);
+ ClientProtoUtils::toRaftClientReplyProto,
+ t -> LOG.warn("Failed to setConfiguration: {}", request, t));
}
@Override
@@ -76,7 +84,8 @@ public class GrpcAdminProtocolService extends
AdminProtocolServiceImplBase {
StreamObserver<RaftClientReplyProto> responseObserver) {
final TransferLeadershipRequest request =
ClientProtoUtils.toTransferLeadershipRequest(proto);
GrpcUtil.asyncCall(responseObserver, () ->
protocol.transferLeadershipAsync(request),
- ClientProtoUtils::toRaftClientReplyProto);
+ ClientProtoUtils::toRaftClientReplyProto,
+ t -> LOG.warn("Failed to transferLeadership: {}", request, t));
}
@Override
@@ -84,7 +93,8 @@ public class GrpcAdminProtocolService extends
AdminProtocolServiceImplBase {
StreamObserver<RaftClientReplyProto> responseObserver) {
final SnapshotManagementRequest request =
ClientProtoUtils.toSnapshotManagementRequest(proto);
GrpcUtil.asyncCall(responseObserver, () ->
protocol.snapshotManagementAsync(request),
- ClientProtoUtils::toRaftClientReplyProto);
+ ClientProtoUtils::toRaftClientReplyProto,
+ t -> LOG.warn("Failed snapshotManagement: {}, {}", proto.getOpCase(),
request, t));
}
@Override
@@ -92,6 +102,7 @@ public class GrpcAdminProtocolService extends
AdminProtocolServiceImplBase {
StreamObserver<RaftClientReplyProto> responseObserver) {
final LeaderElectionManagementRequest request =
ClientProtoUtils.toLeaderElectionManagementRequest(proto);
GrpcUtil.asyncCall(responseObserver, () ->
protocol.leaderElectionManagementAsync(request),
- ClientProtoUtils::toRaftClientReplyProto);
+ ClientProtoUtils::toRaftClientReplyProto,
+ t -> LOG.warn("Failed leaderElectionManagement: {}, {}",
proto.getOpCase(), request, t));
}
}