1717 */
1818package org .apache .ratis .grpc .client ;
1919
20+ import org .apache .ratis .client .RaftClientConfigKeys ;
2021import org .apache .ratis .client .impl .ClientProtoUtils ;
2122import org .apache .ratis .client .impl .RaftClientRpcWithProxy ;
2223import org .apache .ratis .conf .RaftProperties ;
2324import org .apache .ratis .grpc .GrpcConfigKeys ;
2425import org .apache .ratis .grpc .GrpcUtil ;
2526import org .apache .ratis .protocol .*;
2627import org .apache .ratis .protocol .exceptions .AlreadyClosedException ;
28+ import org .apache .ratis .protocol .exceptions .TimeoutIOException ;
29+ import org .apache .ratis .thirdparty .io .grpc .Status ;
2730import org .apache .ratis .thirdparty .io .grpc .StatusRuntimeException ;
2831import org .apache .ratis .thirdparty .io .grpc .stub .StreamObserver ;
2932import org .apache .ratis .proto .RaftProtos .GroupInfoRequestProto ;
3942import org .apache .ratis .util .IOUtils ;
4043import org .apache .ratis .util .JavaUtils ;
4144import org .apache .ratis .util .PeerProxyMap ;
45+ import org .apache .ratis .util .TimeDuration ;
46+ import org .apache .ratis .util .TimeoutExecutor ;
4247import org .slf4j .Logger ;
4348import org .slf4j .LoggerFactory ;
4449
4550import java .io .IOException ;
4651import java .io .InterruptedIOException ;
4752import java .util .concurrent .CompletableFuture ;
4853import java .util .concurrent .ExecutionException ;
54+ import java .util .concurrent .TimeUnit ;
4955
5056public class GrpcClientRpc extends RaftClientRpcWithProxy <GrpcClientProtocolClient > {
5157 public static final Logger LOG = LoggerFactory .getLogger (GrpcClientRpc .class );
5258
5359 private final ClientId clientId ;
5460 private final int maxMessageSize ;
61+ private final TimeDuration requestTimeoutDuration ;
62+ private final TimeDuration watchRequestTimeoutDuration ;
63+ private final TimeoutExecutor scheduler = TimeoutExecutor .getInstance ();
5564
5665 public GrpcClientRpc (ClientId clientId , RaftProperties properties ,
5766 SslContext adminSslContext , SslContext clientSslContext ) {
5867 super (new PeerProxyMap <>(clientId .toString (),
5968 p -> new GrpcClientProtocolClient (clientId , p , properties , adminSslContext , clientSslContext )));
6069 this .clientId = clientId ;
6170 this .maxMessageSize = GrpcConfigKeys .messageSizeMax (properties , LOG ::debug ).getSizeInt ();
71+ this .requestTimeoutDuration = RaftClientConfigKeys .Rpc .requestTimeout (properties );
72+ this .watchRequestTimeoutDuration = RaftClientConfigKeys .Rpc .watchRequestTimeout (properties );
6273 }
6374
6475 @ Override
@@ -122,11 +133,11 @@ public RaftClientReply sendRequest(RaftClientRequest request)
122133 return ClientProtoUtils .toRaftClientReply (proxy .leaderElectionManagement (proto ));
123134 } else {
124135 final CompletableFuture <RaftClientReply > f = sendRequest (request , proxy );
125- // TODO: timeout support
126136 try {
127137 return f .get ();
128138 } catch (InterruptedException e ) {
129139 Thread .currentThread ().interrupt ();
140+ f .cancel (true );
130141 throw new InterruptedIOException (
131142 "Interrupted while waiting for response of request " + request );
132143 } catch (ExecutionException e ) {
@@ -143,6 +154,7 @@ private CompletableFuture<RaftClientReply> sendRequest(
143154 final RaftClientRequestProto requestProto =
144155 toRaftClientRequestProto (request );
145156 final CompletableFuture <RaftClientReplyProto > replyFuture = new CompletableFuture <>();
157+ final TimeDuration timeout = getTimeoutDuration (request );
146158 // create a new grpc stream for each non-async call.
147159 final StreamObserver <RaftClientRequestProto > requestObserver =
148160 proxy .unorderedWithTimeout (new StreamObserver <RaftClientReplyProto >() {
@@ -164,12 +176,35 @@ public void onCompleted() {
164176 }
165177 }
166178 });
179+ replyFuture .whenComplete ((reply , exception ) -> {
180+ if (replyFuture .isCancelled ()) {
181+ requestObserver .onError (Status .CANCELLED
182+ .withDescription (clientId + ": request #" + request .getCallId () + " cancelled" )
183+ .asRuntimeException ());
184+ }
185+ });
186+ scheduler .onTimeout (timeout , () -> {
187+ if (replyFuture .completeExceptionally (new TimeoutIOException (
188+ clientId + ": request #" + request .getCallId () + " timeout " + timeout ))) {
189+ requestObserver .onError (Status .DEADLINE_EXCEEDED
190+ .withDescription (clientId + ": request #" + request .getCallId () + " timeout " + timeout )
191+ .asRuntimeException ());
192+ }
193+ }, LOG , () -> "Timeout check failed for client request #" + request .getCallId ());
167194 requestObserver .onNext (requestProto );
168195 requestObserver .onCompleted ();
169196
170197 return replyFuture .thenApply (ClientProtoUtils ::toRaftClientReply );
171198 }
172199
200+ private TimeDuration getTimeoutDuration (RaftClientRequest request ) {
201+ final long timeoutMs = request .getTimeoutMs ();
202+ if (timeoutMs > 0 ) {
203+ return TimeDuration .valueOf (timeoutMs , TimeUnit .MILLISECONDS );
204+ }
205+ return request .is (RaftClientRequestProto .TypeCase .WATCH ) ? watchRequestTimeoutDuration : requestTimeoutDuration ;
206+ }
207+
173208 private RaftClientRequestProto toRaftClientRequestProto (RaftClientRequest request ) throws IOException {
174209 final RaftClientRequestProto proto = ClientProtoUtils .toRaftClientRequestProto (request );
175210 if (proto .getSerializedSize () > maxMessageSize ) {
0 commit comments