Skip to content

RATIS-2421. Gracefully cancel stream after complete in GrpcLogAppender#1363

Open
symious wants to merge 5 commits intoapache:masterfrom
symious:RATIS-2421
Open

RATIS-2421. Gracefully cancel stream after complete in GrpcLogAppender#1363
symious wants to merge 5 commits intoapache:masterfrom
symious:RATIS-2421

Conversation

@symious
Copy link
Contributor

@symious symious commented Mar 6, 2026

What changes were proposed in this pull request?

When onCompleted() is called, the client waits for the server response to finish the RPC and release the underlying resources.

However, in some cases the server response may never arrive, which leaves the RPC stream open and prevents resources from being released.

This ticket introduces a client-side cancel() after a short grace period following onCompleted(), to ensure the RPC stream is forcefully closed and resources are cleaned up.

What is the link to the Apache JIRA

https://issues.apache.org/jira/browse/RATIS-2421

How was this patch tested?

Test locally.

.ifPresent(s -> completeStreamGracefully(s, "heartbeat"));
}
final long delayMs = Math.max(1L, completeGracePeriod.toLong(TimeUnit.MILLISECONDS));
closer.schedule(this::cancelIfStillNeeded, delayMs, TimeUnit.MILLISECONDS);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This schedules cancelIfStillNeeded() outside the completed.compareAndSet(...) block. Repeated onCompleted() calls can queue redundant cancel tasks.

Can this be avoided?
Also I think if it’s invoked again after shutdownCloser() it may throw exception.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The cancel and shutdown parts can be called multiple times.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

But if it is completed already aren't subsequent cancel calls queuing tasks which are no longer required?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If onComplete() is invoked, the scheudler should be called and shutdown later, later requests should be rejected then.

}

private void cancelStream(
ClientCallStreamObserver<AppendEntriesRequestProto> stream,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

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);
  }

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

As a client, this one should always be a ClientCallStreamObserver.

Copy link
Contributor

@spacemonkd spacemonkd left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for the patch @symious.
Just a few comments, I am not too sure in some cases so definitely another set of eyes would be good to have.
But apart from these it looks good to me

@symious
Copy link
Contributor Author

symious commented Mar 11, 2026

@devabhishekpal Thank you for the review. Updated and PTAL.

Copy link
Contributor

@spacemonkd spacemonkd left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for addressing the comments @symious. Looks good to me, +1

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

2 participants