|
17 | 17 | */ |
18 | 18 | package org.apache.ratis.grpc.client; |
19 | 19 |
|
| 20 | +import org.apache.ratis.client.RaftClientConfigKeys; |
20 | 21 | import org.apache.ratis.client.impl.ClientProtoUtils; |
21 | 22 | import org.apache.ratis.client.impl.RaftClientRpcWithProxy; |
22 | 23 | import org.apache.ratis.conf.RaftProperties; |
23 | 24 | import org.apache.ratis.grpc.GrpcConfigKeys; |
24 | 25 | import org.apache.ratis.grpc.GrpcUtil; |
25 | 26 | import org.apache.ratis.protocol.*; |
26 | 27 | import org.apache.ratis.protocol.exceptions.AlreadyClosedException; |
| 28 | +import org.apache.ratis.protocol.exceptions.TimeoutIOException; |
| 29 | +import org.apache.ratis.thirdparty.io.grpc.Status; |
27 | 30 | import org.apache.ratis.thirdparty.io.grpc.StatusRuntimeException; |
28 | 31 | import org.apache.ratis.thirdparty.io.grpc.stub.StreamObserver; |
29 | 32 | import org.apache.ratis.proto.RaftProtos.GroupInfoRequestProto; |
|
39 | 42 | import org.apache.ratis.util.IOUtils; |
40 | 43 | import org.apache.ratis.util.JavaUtils; |
41 | 44 | import org.apache.ratis.util.PeerProxyMap; |
| 45 | +import org.apache.ratis.util.TimeDuration; |
42 | 46 | import org.slf4j.Logger; |
43 | 47 | import org.slf4j.LoggerFactory; |
44 | 48 |
|
45 | 49 | import java.io.IOException; |
46 | 50 | import java.io.InterruptedIOException; |
47 | 51 | import java.util.concurrent.CompletableFuture; |
48 | 52 | import java.util.concurrent.ExecutionException; |
| 53 | +import java.util.concurrent.TimeUnit; |
| 54 | +import java.util.concurrent.TimeoutException; |
49 | 55 |
|
50 | 56 | public class GrpcClientRpc extends RaftClientRpcWithProxy<GrpcClientProtocolClient> { |
51 | 57 | public static final Logger LOG = LoggerFactory.getLogger(GrpcClientRpc.class); |
52 | 58 |
|
53 | 59 | private final ClientId clientId; |
54 | 60 | private final int maxMessageSize; |
| 61 | + private final TimeDuration requestTimeoutDuration; |
| 62 | + private final TimeDuration watchRequestTimeoutDuration; |
55 | 63 |
|
56 | 64 | public GrpcClientRpc(ClientId clientId, RaftProperties properties, |
57 | 65 | SslContext adminSslContext, SslContext clientSslContext) { |
58 | 66 | super(new PeerProxyMap<>(clientId.toString(), |
59 | 67 | p -> new GrpcClientProtocolClient(clientId, p, properties, adminSslContext, clientSslContext))); |
60 | 68 | this.clientId = clientId; |
61 | 69 | this.maxMessageSize = GrpcConfigKeys.messageSizeMax(properties, LOG::debug).getSizeInt(); |
| 70 | + this.requestTimeoutDuration = RaftClientConfigKeys.Rpc.requestTimeout(properties); |
| 71 | + this.watchRequestTimeoutDuration = RaftClientConfigKeys.Rpc.watchRequestTimeout(properties); |
62 | 72 | } |
63 | 73 |
|
64 | 74 | @Override |
@@ -121,24 +131,11 @@ public RaftClientReply sendRequest(RaftClientRequest request) |
121 | 131 | ((LeaderElectionManagementRequest) request); |
122 | 132 | return ClientProtoUtils.toRaftClientReply(proxy.leaderElectionManagement(proto)); |
123 | 133 | } else { |
124 | | - final CompletableFuture<RaftClientReply> f = sendRequest(request, proxy); |
125 | | - // TODO: timeout support |
126 | | - try { |
127 | | - return f.get(); |
128 | | - } catch (InterruptedException e) { |
129 | | - Thread.currentThread().interrupt(); |
130 | | - throw new InterruptedIOException( |
131 | | - "Interrupted while waiting for response of request " + request); |
132 | | - } catch (ExecutionException e) { |
133 | | - if (LOG.isTraceEnabled()) { |
134 | | - LOG.trace(clientId + ": failed " + request, e); |
135 | | - } |
136 | | - throw IOUtils.toIOException(e); |
137 | | - } |
| 134 | + return sendRequest(request, proxy); |
138 | 135 | } |
139 | 136 | } |
140 | 137 |
|
141 | | - private CompletableFuture<RaftClientReply> sendRequest( |
| 138 | + private RaftClientReply sendRequest( |
142 | 139 | RaftClientRequest request, GrpcClientProtocolClient proxy) throws IOException { |
143 | 140 | final RaftClientRequestProto requestProto = |
144 | 141 | toRaftClientRequestProto(request); |
@@ -167,7 +164,44 @@ public void onCompleted() { |
167 | 164 | requestObserver.onNext(requestProto); |
168 | 165 | requestObserver.onCompleted(); |
169 | 166 |
|
170 | | - return replyFuture.thenApply(ClientProtoUtils::toRaftClientReply); |
| 167 | + final TimeDuration timeout = getTimeoutDuration(request); |
| 168 | + try { |
| 169 | + return replyFuture.thenApply(ClientProtoUtils::toRaftClientReply) |
| 170 | + .get(timeout.getDuration(), timeout.getUnit()); |
| 171 | + } catch (InterruptedException e) { |
| 172 | + Thread.currentThread().interrupt(); |
| 173 | + replyFuture.cancel(true); |
| 174 | + final InterruptedIOException ioe = new InterruptedIOException(clientId + ": Interrupted " + request); |
| 175 | + sendOnError(requestObserver, Status.CANCELLED, ioe.getMessage()); |
| 176 | + throw ioe; |
| 177 | + } catch (TimeoutException e) { |
| 178 | + replyFuture.cancel(true); |
| 179 | + final TimeoutIOException ioe = |
| 180 | + new TimeoutIOException(clientId + ": Timed out " + timeout + " for " + request, e); |
| 181 | + sendOnError(requestObserver, Status.DEADLINE_EXCEEDED, ioe.getMessage()); |
| 182 | + throw ioe; |
| 183 | + } catch (ExecutionException e) { |
| 184 | + if (LOG.isTraceEnabled()) { |
| 185 | + LOG.trace("{} : failed {}", clientId, request, e); |
| 186 | + } |
| 187 | + throw IOUtils.toIOException(e); |
| 188 | + } |
| 189 | + } |
| 190 | + |
| 191 | + private void sendOnError(StreamObserver<RaftClientRequestProto> requestObserver, Status status, String message) { |
| 192 | + try { |
| 193 | + requestObserver.onError(status.withDescription(message).asException()); |
| 194 | + } catch (Exception ignored) { |
| 195 | + // the stream already closed. |
| 196 | + } |
| 197 | + } |
| 198 | + |
| 199 | + private TimeDuration getTimeoutDuration(RaftClientRequest request) { |
| 200 | + final long timeoutMs = request.getTimeoutMs(); |
| 201 | + if (timeoutMs > 0) { |
| 202 | + return TimeDuration.valueOf(timeoutMs, TimeUnit.MILLISECONDS); |
| 203 | + } |
| 204 | + return request.is(RaftClientRequestProto.TypeCase.WATCH) ? watchRequestTimeoutDuration : requestTimeoutDuration; |
171 | 205 | } |
172 | 206 |
|
173 | 207 | private RaftClientRequestProto toRaftClientRequestProto(RaftClientRequest request) throws IOException { |
|
0 commit comments