This is an automated email from the ASF dual-hosted git repository.
adoroszlai pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ratis.git
The following commit(s) were added to refs/heads/master by this push:
new 0514e0926 RATIS-2225. RaftClientRequest leak in RaftServerImpl. (#1198)
0514e0926 is described below
commit 0514e0926202b50054643f677510c7bfe16c01f1
Author: Tsz-Wo Nicholas Sze <[email protected]>
AuthorDate: Mon Dec 23 11:33:05 2024 -0800
RATIS-2225. RaftClientRequest leak in RaftServerImpl. (#1198)
---
.../apache/ratis/server/impl/RaftServerImpl.java | 23 ++++++++++++++--------
1 file changed, 15 insertions(+), 8 deletions(-)
diff --git
a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java
b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java
index 5a7e37233..0ec73d5f4 100644
---
a/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java
+++
b/ratis-server/src/main/java/org/apache/ratis/server/impl/RaftServerImpl.java
@@ -631,6 +631,15 @@ class RaftServerImpl implements RaftServer.Division,
@Override
public Collection<CommitInfoProto> getCommitInfos() {
+ try {
+ return getCommitInfosImpl();
+ } catch (Throwable t) {
+ LOG.warn("{} Failed to getCommitInfos", getMemberId(), t);
+ return Collections.emptyList();
+ }
+ }
+
+ private Collection<CommitInfoProto> getCommitInfosImpl() {
final List<CommitInfoProto> infos = new ArrayList<>();
// add the commit info of this server
final long commitIndex = updateCommitInfoCache();
@@ -922,17 +931,10 @@ class RaftServerImpl implements RaftServer.Division,
public CompletableFuture<RaftClientReply> submitClientRequestAsync(
ReferenceCountedObject<RaftClientRequest> requestRef) {
final RaftClientRequest request = requestRef.retain();
- LOG.debug("{}: receive client request({})", getMemberId(), request);
-
try {
+ LOG.debug("{}: receive client request({})", getMemberId(), request);
assertLifeCycleState(LifeCycle.States.RUNNING);
- } catch (ServerNotReadyException e) {
- final RaftClientReply reply = newExceptionReply(request, e);
- requestRef.release();
- return CompletableFuture.completedFuture(reply);
- }
- try {
RaftClientRequest.Type type = request.getType();
final Timekeeper timer = raftServerMetrics.getClientRequestTimer(type);
final Optional<Timekeeper.Context> timerContext =
Optional.ofNullable(timer).map(Timekeeper::time);
@@ -942,6 +944,11 @@ class RaftServerImpl implements RaftServer.Division,
raftServerMetrics.incFailedRequestCount(type);
}
});
+ } catch (RaftException e) {
+ return CompletableFuture.completedFuture(newExceptionReply(request, e));
+ } catch (Throwable t) {
+ LOG.error("{} Failed to submitClientRequestAsync for {}", getMemberId(),
request, t);
+ return CompletableFuture.completedFuture(newExceptionReply(request, new
RaftException(t)));
} finally {
requestRef.release();
}