[
https://issues.apache.org/jira/browse/RATIS-2152?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
Chung En Lee updated RATIS-2152:
--------------------------------
Description:
In `GrpcLogAppender`, it waits for signal at the end of `notifyInstallSnapshot`
as following.
[https://github.com/apache/ratis/blob/master/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java#L825-L831]
However, checking whether the `InstallSnapshotResponseHandler` is done and the
call `AwaitForSignal.await()` are not atomic. This creates a potential race
condition where InstallSnapshotResponseHandler.close() could finish after the
check but before the wait, causing that `GrpcLogAppender` is still waiting even
though `InstallSnapshotResponseHandler` has already completed, leading to
timeout.
was:
In `GrpcLogAppender`, it waits for signal at the end of `notifyInstallSnapshot`
as following.
https://github.com/apache/ratis/blob/master/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java#L825-L831
However, checking whether the `InstallSnapshotResponseHandler` is done and the
call `AwaitForSignal.await()` are not atomic. This creates a potential race
condition where InstallSnapshotResponseHandler.close() could finish after the
check but before the wait, causing that `GrpcLogAppender` is still waiting even
though `InstallSnapshotResponseHandler` has already completed, leading to a
timeout.
> The `done` should also lock in `AwaitForSignal`
> -----------------------------------------------
>
> Key: RATIS-2152
> URL: https://issues.apache.org/jira/browse/RATIS-2152
> Project: Ratis
> Issue Type: Bug
> Reporter: Chung En Lee
> Assignee: Chung En Lee
> Priority: Major
>
> In `GrpcLogAppender`, it waits for signal at the end of
> `notifyInstallSnapshot` as following.
> [https://github.com/apache/ratis/blob/master/ratis-grpc/src/main/java/org/apache/ratis/grpc/server/GrpcLogAppender.java#L825-L831]
> However, checking whether the `InstallSnapshotResponseHandler` is done and
> the call `AwaitForSignal.await()` are not atomic. This creates a potential
> race condition where InstallSnapshotResponseHandler.close() could finish
> after the check but before the wait, causing that `GrpcLogAppender` is still
> waiting even though `InstallSnapshotResponseHandler` has already completed,
> leading to timeout.
--
This message was sent by Atlassian Jira
(v8.20.10#820010)