devabhishekpal commented on code in PR #1363:
URL: https://github.com/apache/ratis/pull/1363#discussion_r2914667294
##########
ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java:
##########
@@ -366,16 +382,73 @@ void onNext(AppendEntriesRequestProto proto)
while (!stream.isReady() && running) {
sleep(waitForReady, isHeartBeat);
}
- stream.onNext(proto);
+ try {
+ stream.onNext(proto);
+ } catch (Exception e) {
+ InterruptedIOException ioe =
+ new InterruptedIOException("Failed to send request via stream");
+ ioe.initCause(e);
+ throw ioe;
+ }
}
void stop() {
running = false;
}
void onCompleted() {
- appendLog.onCompleted();
- Optional.ofNullable(heartbeat).ifPresent(StreamObserver::onCompleted);
+ if (completed.compareAndSet(false, true)) {
+ completeStreamGracefully(appendLog, "appendLog");
+ Optional.ofNullable(heartbeat)
+ .ifPresent(s -> completeStreamGracefully(s, "heartbeat"));
+ }
+ final long delayMs = Math.max(1L,
completeGracePeriod.toLong(TimeUnit.MILLISECONDS));
+ closer.schedule(this::cancelIfStillNeeded, delayMs,
TimeUnit.MILLISECONDS);
+ }
+
+ void cancelNow(String reason, Throwable cause) {
+ if (cancelled.compareAndSet(false, true)) {
+ running = false;
+ cancelStream(appendLog, "appendLog", reason, cause);
+ Optional.ofNullable(heartbeat)
+ .ifPresent(s -> cancelStream(s, "heartbeat", reason, cause));
+ shutdownCloser();
+ }
+ }
+
+ private void cancelIfStillNeeded() {
+ if (cancelled.compareAndSet(false, true)) {
+ cancelStream(appendLog, "appendLog", "Stream completion timeout",
null);
+ Optional.ofNullable(heartbeat)
+ .ifPresent(s -> cancelStream(s, "heartbeat", "Stream completion
timeout", null));
+ }
+ shutdownCloser();
+ }
+
+ private void completeStreamGracefully(
+ ClientCallStreamObserver<AppendEntriesRequestProto> stream,
+ String name) {
+ try {
+ stream.onCompleted();
+ } catch (Exception e) {
+ LOG.warn("Failed to call onCompleted on {}", name, e);
+ }
+ }
+
+ private void cancelStream(
+ ClientCallStreamObserver<AppendEntriesRequestProto> stream,
Review Comment:
We are doing explicit cast to `ClientCallStreamObserver<>`, but I am
thinking can there be a case where `appendEntries()` can return only
`CallStreamObserver`?
That is the method signature in GrpcServerProtocolClient.
In such a case this can throw error.
Better to accept CallStreamObserver<> and then do an instanceof check?
```
try {
if (stream instanceof ClientCallStreamObserver) {
((ClientCallStreamObserver<AppendEntriesRequestProto>)
stream).cancel(reason, cause);
} else {
...handle cancel when it is not clientcallstreamobserver, something
like stream.onError(...)?
}
} catch (Exception e) {
LOG.warn("Failed to cancel {}", name, e);
}
```
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]