4747
4848import static org .apache .ratis .grpc .GrpcUtil .addMethodWithCustomMarshaller ;
4949import static org .apache .ratis .proto .grpc .RaftServerProtocolServiceGrpc .getAppendEntriesMethod ;
50+ import static org .apache .ratis .proto .grpc .RaftServerProtocolServiceGrpc .getInstallSnapshotMethod ;
5051
5152class GrpcServerProtocolService extends RaftServerProtocolServiceImplBase {
5253 public static final Logger LOG = LoggerFactory .getLogger (GrpcServerProtocolService .class );
@@ -59,6 +60,7 @@ private enum BatchLogKey implements BatchLogger.Key {
5960 static class PendingServerRequest <REQUEST > {
6061 private final AtomicReference <ReferenceCountedObject <REQUEST >> requestRef ;
6162 private final CompletableFuture <Void > future = new CompletableFuture <>();
63+ private volatile String requestString ;
6264
6365 PendingServerRequest (ReferenceCountedObject <REQUEST > requestRef ) {
6466 requestRef .retain ();
@@ -71,6 +73,14 @@ REQUEST getRequest() {
7173 .orElse (null );
7274 }
7375
76+ void setRequestString (String requestString ) {
77+ this .requestString = requestString ;
78+ }
79+
80+ String getRequestString () {
81+ return requestString ;
82+ }
83+
7484 CompletableFuture <Void > getFuture () {
7585 return future ;
7686 }
@@ -104,8 +114,7 @@ String getName() {
104114
105115 private String getPreviousRequestString () {
106116 return Optional .ofNullable (previousOnNext .get ())
107- .map (PendingServerRequest ::getRequest )
108- .map (this ::requestToString )
117+ .map (PendingServerRequest ::getRequestString )
109118 .orElse (null );
110119 }
111120
@@ -178,12 +187,17 @@ public void onNext(REQUEST request) {
178187 }
179188
180189 final PendingServerRequest <REQUEST > current = new PendingServerRequest <>(requestRef );
181- final long callId = getCallId (current .getRequest ());
182- final boolean isHeartbeat = isHeartbeat (current .getRequest ());
183- final Optional <PendingServerRequest <REQUEST >> previous = Optional .ofNullable (previousOnNext .getAndSet (current ));
184- final CompletableFuture <Void > previousFuture = previous .map (PendingServerRequest ::getFuture )
185- .orElse (CompletableFuture .completedFuture (null ));
190+ current .getFuture ().whenComplete ((r , e ) -> current .release ());
191+ final REQUEST currentRequest = current .getRequest ();
192+ final long callId = getCallId (currentRequest );
193+ final boolean isHeartbeat = isHeartbeat (currentRequest );
194+ Optional <PendingServerRequest <REQUEST >> previous = Optional .empty ();
195+ CompletableFuture <Void > previousFuture = CompletableFuture .completedFuture (null );
186196 try {
197+ current .setRequestString (requestToString (currentRequest ));
198+ previous = Optional .ofNullable (previousOnNext .getAndSet (current ));
199+ previousFuture = previous .map (PendingServerRequest ::getFuture )
200+ .orElse (CompletableFuture .completedFuture (null ));
187201 final CompletableFuture <REPLY > f = process (requestRef ).exceptionally (e -> {
188202 // Handle cases, such as RaftServer is paused
189203 handleError (e , callId , isHeartbeat );
@@ -243,15 +257,23 @@ private void releaseLast() {
243257 private final RaftServer server ;
244258 private final boolean zeroCopyEnabled ;
245259 private final ZeroCopyMessageMarshaller <AppendEntriesRequestProto > zeroCopyRequestMarshaller ;
260+ private final ZeroCopyMessageMarshaller <InstallSnapshotRequestProto > zeroCopyInstallSnapshotMarshaller ;
246261
247262 GrpcServerProtocolService (Supplier <RaftPeerId > idSupplier , RaftServer server , boolean zeroCopyEnabled ,
248263 ZeroCopyMetrics zeroCopyMetrics ) {
249264 this .idSupplier = idSupplier ;
250265 this .server = server ;
251266 this .zeroCopyEnabled = zeroCopyEnabled ;
252267 this .zeroCopyRequestMarshaller = new ZeroCopyMessageMarshaller <>(AppendEntriesRequestProto .getDefaultInstance (),
253- zeroCopyMetrics ::onZeroCopyMessage , zeroCopyMetrics ::onNonZeroCopyMessage , zeroCopyMetrics ::onReleasedMessage );
268+ zeroCopyMetrics ::onZeroCopyAppendEntries , zeroCopyMetrics ::onNonZeroCopyMessage ,
269+ zeroCopyMetrics ::onReleasedMessage , zeroCopyMetrics .newMarshallerMetrics ());
270+ this .zeroCopyInstallSnapshotMarshaller = new ZeroCopyMessageMarshaller <>(
271+ InstallSnapshotRequestProto .getDefaultInstance (),
272+ zeroCopyMetrics ::onZeroCopyInstallSnapshot , zeroCopyMetrics ::onNonZeroCopyMessage ,
273+ zeroCopyMetrics ::onReleasedMessage , zeroCopyMetrics .newMarshallerMetrics ());
254274 zeroCopyMetrics .addUnreleased ("server_protocol" , zeroCopyRequestMarshaller ::getUnclosedCount );
275+ zeroCopyMetrics .addUnreleased ("server_protocol_install_snapshot" ,
276+ zeroCopyInstallSnapshotMarshaller ::getUnclosedCount );
255277 }
256278
257279 RaftPeerId getId () {
@@ -268,9 +290,16 @@ ServerServiceDefinition bindServiceWithZeroCopy() {
268290
269291 // Add appendEntries with zero copy marshaller.
270292 addMethodWithCustomMarshaller (orig , builder , getAppendEntriesMethod (), zeroCopyRequestMarshaller );
293+ // Add installSnapshot with zero copy marshaller for zero-copy counters/metrics.
294+ addMethodWithCustomMarshaller (orig , builder , getInstallSnapshotMethod (), zeroCopyInstallSnapshotMarshaller );
271295 // Add remaining methods as is.
296+ final String appendEntriesMethod = getAppendEntriesMethod ().getFullMethodName ();
297+ final String installSnapshotMethod = getInstallSnapshotMethod ().getFullMethodName ();
272298 orig .getMethods ().stream ().filter (
273- x -> !x .getMethodDescriptor ().getFullMethodName ().equals (getAppendEntriesMethod ().getFullMethodName ())
299+ x -> {
300+ final String methodName = x .getMethodDescriptor ().getFullMethodName ();
301+ return !methodName .equals (appendEntriesMethod ) && !methodName .equals (installSnapshotMethod );
302+ }
274303 ).forEach (
275304 builder ::addMethod
276305 );
@@ -365,6 +394,11 @@ CompletableFuture<InstallSnapshotReplyProto> process(InstallSnapshotRequestProto
365394 return CompletableFuture .completedFuture (server .installSnapshot (request ));
366395 }
367396
397+ @ Override
398+ void release (InstallSnapshotRequestProto request ) {
399+ zeroCopyInstallSnapshotMarshaller .release (request );
400+ }
401+
368402 @ Override
369403 long getCallId (InstallSnapshotRequestProto request ) {
370404 return request .getServerRequest ().getCallId ();
0 commit comments