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
3232import org .apache .ratis .proto .RaftProtos .RaftClientReplyProto ;
3333import org .apache .ratis .proto .RaftProtos .GroupManagementRequestProto ;
3434import org .apache .ratis .proto .grpc .AdminProtocolServiceGrpc .AdminProtocolServiceImplBase ;
35+ import org .slf4j .Logger ;
36+ import org .slf4j .LoggerFactory ;
3537
3638public 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