From a4826e2393a529cbe60d4e76084f2d5bf28f594b Mon Sep 17 00:00:00 2001 From: Tsz-Wo Nicholas Sze Date: Sun, 1 Dec 2024 18:01:21 -0800 Subject: [PATCH] RATIS-2185. Improve gRPC log messages debugability. (#1186) --- .../java/org/apache/ratis/util/LogUtils.java | 8 +++++- .../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 d29f1e56ec..63bb08accf 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 @@ static void warn(Logger log, Supplier message, Throwable t, Class... 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 5a7f003183..e17d17bff6 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 @@ static IOException unwrapIOException(Throwable t) { return e; } - static void asyncCall( - StreamObserver responseObserver, - CheckedSupplier, IOException> supplier, - Function toProto) { - asyncCall(responseObserver, supplier, toProto, throwable -> {}); - } - static void asyncCall( StreamObserver responseObserver, CheckedSupplier, 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 feb780302e..f6a6cfba6a 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.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 void groupManagement(GroupManagementRequestProto proto, StreamObserver 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 void groupList(GroupListRequestProto proto, StreamObserver 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 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 void setConfiguration(SetConfigurationRequestProto proto, StreamObserver 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 void transferLeadership(TransferLeadershipRequestProto proto, StreamObserver 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 void snapshotManagement(SnapshotManagementRequestProto proto, StreamObserver 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 void leaderElectionManagement(LeaderElectionManagementRequestProto proto, StreamObserver 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)); } }