This is an automated email from the ASF dual-hosted git repository.

szetszwo 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 3d06ad6  RATIS-1484. Add take snapshot grpc related request and proto 
(#575)
3d06ad6 is described below

commit 3d06ad6bbae2f05dd259c2f86b15add3c441db59
Author: Yaolong Liu <[email protected]>
AuthorDate: Tue Jan 11 15:05:54 2022 +0800

    RATIS-1484. Add take snapshot grpc related request and proto (#575)
---
 .../org/apache/ratis/grpc/client/GrpcClientProtocolClient.java   | 8 ++++++++
 .../main/java/org/apache/ratis/grpc/client/GrpcClientRpc.java    | 5 +++++
 .../org/apache/ratis/grpc/server/GrpcAdminProtocolService.java   | 9 +++++++++
 ratis-proto/src/main/proto/Grpc.proto                            | 3 +++
 4 files changed, 25 insertions(+)

diff --git 
a/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientProtocolClient.java
 
b/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientProtocolClient.java
index 3b4aed6..ab68032 100644
--- 
a/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientProtocolClient.java
+++ 
b/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientProtocolClient.java
@@ -33,6 +33,7 @@ import org.apache.ratis.proto.RaftProtos.RaftClientReplyProto;
 import org.apache.ratis.proto.RaftProtos.RaftClientRequestProto;
 import org.apache.ratis.proto.RaftProtos.SetConfigurationRequestProto;
 import org.apache.ratis.proto.RaftProtos.TransferLeadershipRequestProto;
+import org.apache.ratis.proto.RaftProtos.SnapshotManagementRequestProto;
 import org.apache.ratis.protocol.exceptions.LeaderNotReadyException;
 import org.apache.ratis.protocol.exceptions.TimeoutIOException;
 import org.apache.ratis.thirdparty.io.grpc.netty.GrpcSslContexts;
@@ -209,6 +210,13 @@ public class GrpcClientProtocolClient implements Closeable 
{
         .transferLeadership(request));
   }
 
+  RaftClientReplyProto snapshotManagement(
+      SnapshotManagementRequestProto request) throws IOException {
+    return blockingCall(() -> adminBlockingStub
+        .withDeadlineAfter(requestTimeoutDuration.getDuration(), 
requestTimeoutDuration.getUnit())
+        .snapshotManagement(request));
+  }
+
   private static RaftClientReplyProto blockingCall(
       CheckedSupplier<RaftClientReplyProto, StatusRuntimeException> supplier
       ) throws IOException {
diff --git 
a/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientRpc.java 
b/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientRpc.java
index 44c8e78..5686a2b 100644
--- a/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientRpc.java
+++ b/ratis-grpc/src/main/java/org/apache/ratis/grpc/client/GrpcClientRpc.java
@@ -34,6 +34,7 @@ import org.apache.ratis.proto.RaftProtos.RaftClientReplyProto;
 import org.apache.ratis.proto.RaftProtos.RaftClientRequestProto;
 import org.apache.ratis.proto.RaftProtos.SetConfigurationRequestProto;
 import org.apache.ratis.proto.RaftProtos.TransferLeadershipRequestProto;
+import org.apache.ratis.proto.RaftProtos.SnapshotManagementRequestProto;
 import org.apache.ratis.util.IOUtils;
 import org.apache.ratis.util.JavaUtils;
 import org.apache.ratis.util.PeerProxyMap;
@@ -110,6 +111,10 @@ public class GrpcClientRpc extends 
RaftClientRpcWithProxy<GrpcClientProtocolClie
       final TransferLeadershipRequestProto proto = 
ClientProtoUtils.toTransferLeadershipRequestProto(
           (TransferLeadershipRequest) request);
       return 
ClientProtoUtils.toRaftClientReply(proxy.transferLeadership(proto));
+    } else if (request instanceof SnapshotManagementRequest) {
+      final SnapshotManagementRequestProto proto = 
ClientProtoUtils.toSnapshotManagementRequestProto
+          ((SnapshotManagementRequest)request);
+      return 
ClientProtoUtils.toRaftClientReply(proxy.snapshotManagement(proto));
     } else {
       final CompletableFuture<RaftClientReply> f = sendRequest(request, proxy);
       // TODO: timeout support
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 1156aeb..6a4ff3f 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
@@ -25,6 +25,7 @@ import org.apache.ratis.protocol.GroupInfoRequest;
 import org.apache.ratis.protocol.GroupListRequest;
 import org.apache.ratis.protocol.GroupManagementRequest;
 import org.apache.ratis.protocol.SetConfigurationRequest;
+import org.apache.ratis.protocol.SnapshotManagementRequest;
 import org.apache.ratis.protocol.TransferLeadershipRequest;
 import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver;
 import org.apache.ratis.proto.RaftProtos.RaftClientReplyProto;
@@ -76,4 +77,12 @@ public class GrpcAdminProtocolService extends 
AdminProtocolServiceImplBase {
     GrpcUtil.asyncCall(responseObserver, () -> 
protocol.transferLeadershipAsync(request),
         ClientProtoUtils::toRaftClientReplyProto);
   }
+
+  @Override
+  public void snapshotManagement(SnapshotManagementRequestProto proto,
+      StreamObserver<RaftClientReplyProto> responseObserver) {
+    final SnapshotManagementRequest request = 
ClientProtoUtils.toSnapshotManagementRequest(proto);
+    GrpcUtil.asyncCall(responseObserver, () -> 
protocol.snapshotManagementAsync(request),
+        ClientProtoUtils::toRaftClientReplyProto);
+  }
 }
diff --git a/ratis-proto/src/main/proto/Grpc.proto 
b/ratis-proto/src/main/proto/Grpc.proto
index 61a8347..799b3ac 100644
--- a/ratis-proto/src/main/proto/Grpc.proto
+++ b/ratis-proto/src/main/proto/Grpc.proto
@@ -59,6 +59,9 @@ service AdminProtocolService {
   rpc groupManagement(ratis.common.GroupManagementRequestProto)
       returns(ratis.common.RaftClientReplyProto) {}
 
+  rpc snapshotManagement(ratis.common.SnapshotManagementRequestProto)
+      returns(ratis.common.RaftClientReplyProto) {}
+
   rpc groupList(ratis.common.GroupListRequestProto)
       returns(ratis.common.GroupListReplyProto) {}
 

Reply via email to