Skip to content

Commit

Permalink
RATIS-2185. Improve gRPC log messages debugability. (#1186)
Browse files Browse the repository at this point in the history
  • Loading branch information
szetszwo authored Dec 2, 2024
1 parent 3efec8d commit a4826e2
Show file tree
Hide file tree
Showing 3 changed files with 26 additions and 16 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -118,7 +118,13 @@ static void warn(Logger log, Supplier<String> 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);
}
Expand Down
7 changes: 0 additions & 7 deletions ratis-grpc/src/main/java/org/apache/ratis/grpc/GrpcUtil.java
Original file line number Diff line number Diff line change
Expand Up @@ -161,13 +161,6 @@ static IOException unwrapIOException(Throwable t) {
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,
Expand Down
Original file line number Diff line number Diff line change
@@ -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
Expand Down Expand Up @@ -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) {
Expand All @@ -45,53 +49,60 @@ public void groupManagement(GroupManagementRequestProto proto,
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
public void groupList(GroupListRequestProto proto,
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
public void setConfiguration(SetConfigurationRequestProto proto,
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
public void transferLeadership(TransferLeadershipRequestProto proto,
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
public void snapshotManagement(SnapshotManagementRequestProto proto,
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
public void leaderElectionManagement(LeaderElectionManagementRequestProto proto,
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));
}
}

0 comments on commit a4826e2

Please sign in to comment.