Skip to content

Commit a4826e2

Browse files
authored
RATIS-2185. Improve gRPC log messages debugability. (#1186)
1 parent 3efec8d commit a4826e2

File tree

3 files changed

+26
-16
lines changed

3 files changed

+26
-16
lines changed

ratis-common/src/main/java/org/apache/ratis/util/LogUtils.java

Lines changed: 7 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -118,7 +118,13 @@ static void warn(Logger log, Supplier<String> message, Throwable t, Class<?>...
118118
if (log.isWarnEnabled()) {
119119
if (ReflectionUtils.isInstance(t, exceptionClasses)) {
120120
// do not print stack trace for known exceptions.
121-
log.warn(message.get() + ": " + t);
121+
final StringBuilder b = new StringBuilder()
122+
.append(message.get())
123+
.append(": ").append(t);
124+
for(Throwable cause = t.getCause(); cause != null; cause = cause.getCause()) {
125+
b.append("\n Caused by: ").append(cause);
126+
}
127+
log.warn(b.toString());
122128
} else {
123129
log.warn(message.get(), t);
124130
}

ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcUtil.java

Lines changed: 0 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -161,13 +161,6 @@ static IOException unwrapIOException(Throwable t) {
161161
return e;
162162
}
163163

164-
static <REPLY, REPLY_PROTO> void asyncCall(
165-
StreamObserver<REPLY_PROTO> responseObserver,
166-
CheckedSupplier<CompletableFuture<REPLY>, IOException> supplier,
167-
Function<REPLY, REPLY_PROTO> toProto) {
168-
asyncCall(responseObserver, supplier, toProto, throwable -> {});
169-
}
170-
171164
static <REPLY, REPLY_PROTO> void asyncCall(
172165
StreamObserver<REPLY_PROTO> responseObserver,
173166
CheckedSupplier<CompletableFuture<REPLY>, IOException> supplier,

ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcAdminProtocolService.java

Lines changed: 19 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -1,4 +1,4 @@
1-
/**
1+
/*
22
* Licensed to the Apache Software Foundation (ASF) under one
33
* or more contributor license agreements. See the NOTICE file
44
* distributed with this work for additional information
@@ -32,8 +32,12 @@
3232
import org.apache.ratis.proto.RaftProtos.RaftClientReplyProto;
3333
import org.apache.ratis.proto.RaftProtos.GroupManagementRequestProto;
3434
import org.apache.ratis.proto.grpc.AdminProtocolServiceGrpc.AdminProtocolServiceImplBase;
35+
import org.slf4j.Logger;
36+
import org.slf4j.LoggerFactory;
3537

3638
public class GrpcAdminProtocolService extends AdminProtocolServiceImplBase {
39+
static final Logger LOG = LoggerFactory.getLogger(GrpcAdminProtocolService.class);
40+
3741
private final AdminAsynchronousProtocol protocol;
3842

3943
public GrpcAdminProtocolService(AdminAsynchronousProtocol protocol) {
@@ -45,53 +49,60 @@ public void groupManagement(GroupManagementRequestProto proto,
4549
StreamObserver<RaftClientReplyProto> responseObserver) {
4650
final GroupManagementRequest request = ClientProtoUtils.toGroupManagementRequest(proto);
4751
GrpcUtil.asyncCall(responseObserver, () -> protocol.groupManagementAsync(request),
48-
ClientProtoUtils::toRaftClientReplyProto);
52+
ClientProtoUtils::toRaftClientReplyProto,
53+
t -> LOG.warn("Failed groupManagement: {}, {}", proto.getOpCase(), request, t));
4954
}
5055

5156
@Override
5257
public void groupList(GroupListRequestProto proto,
5358
StreamObserver<GroupListReplyProto> responseObserver) {
5459
final GroupListRequest request = ClientProtoUtils.toGroupListRequest(proto);
5560
GrpcUtil.asyncCall(responseObserver, () -> protocol.getGroupListAsync(request),
56-
ClientProtoUtils::toGroupListReplyProto);
61+
ClientProtoUtils::toGroupListReplyProto,
62+
t -> LOG.warn("Failed to groupList: {}", request, t));
5763
}
5864

5965
@Override
6066
public void groupInfo(GroupInfoRequestProto proto, StreamObserver<GroupInfoReplyProto> responseObserver) {
6167
final GroupInfoRequest request = ClientProtoUtils.toGroupInfoRequest(proto);
6268
GrpcUtil.asyncCall(responseObserver, () -> protocol.getGroupInfoAsync(request),
63-
ClientProtoUtils::toGroupInfoReplyProto);
69+
ClientProtoUtils::toGroupInfoReplyProto,
70+
t -> LOG.warn("Failed to groupInfo: {}", request, t));
6471
}
6572

6673
@Override
6774
public void setConfiguration(SetConfigurationRequestProto proto,
6875
StreamObserver<RaftClientReplyProto> responseObserver) {
6976
final SetConfigurationRequest request = ClientProtoUtils.toSetConfigurationRequest(proto);
7077
GrpcUtil.asyncCall(responseObserver, () -> protocol.setConfigurationAsync(request),
71-
ClientProtoUtils::toRaftClientReplyProto);
78+
ClientProtoUtils::toRaftClientReplyProto,
79+
t -> LOG.warn("Failed to setConfiguration: {}", request, t));
7280
}
7381

7482
@Override
7583
public void transferLeadership(TransferLeadershipRequestProto proto,
7684
StreamObserver<RaftClientReplyProto> responseObserver) {
7785
final TransferLeadershipRequest request = ClientProtoUtils.toTransferLeadershipRequest(proto);
7886
GrpcUtil.asyncCall(responseObserver, () -> protocol.transferLeadershipAsync(request),
79-
ClientProtoUtils::toRaftClientReplyProto);
87+
ClientProtoUtils::toRaftClientReplyProto,
88+
t -> LOG.warn("Failed to transferLeadership: {}", request, t));
8089
}
8190

8291
@Override
8392
public void snapshotManagement(SnapshotManagementRequestProto proto,
8493
StreamObserver<RaftClientReplyProto> responseObserver) {
8594
final SnapshotManagementRequest request = ClientProtoUtils.toSnapshotManagementRequest(proto);
8695
GrpcUtil.asyncCall(responseObserver, () -> protocol.snapshotManagementAsync(request),
87-
ClientProtoUtils::toRaftClientReplyProto);
96+
ClientProtoUtils::toRaftClientReplyProto,
97+
t -> LOG.warn("Failed snapshotManagement: {}, {}", proto.getOpCase(), request, t));
8898
}
8999

90100
@Override
91101
public void leaderElectionManagement(LeaderElectionManagementRequestProto proto,
92102
StreamObserver<RaftClientReplyProto> responseObserver) {
93103
final LeaderElectionManagementRequest request = ClientProtoUtils.toLeaderElectionManagementRequest(proto);
94104
GrpcUtil.asyncCall(responseObserver, () -> protocol.leaderElectionManagementAsync(request),
95-
ClientProtoUtils::toRaftClientReplyProto);
105+
ClientProtoUtils::toRaftClientReplyProto,
106+
t -> LOG.warn("Failed leaderElectionManagement: {}, {}", proto.getOpCase(), request, t));
96107
}
97108
}

0 commit comments

Comments
 (0)