Skip to content

Commit

Permalink
RATIS-2197. Clean remote stream to resolve direct memory leak (#1179)
Browse files Browse the repository at this point in the history
  • Loading branch information
symious authored Dec 25, 2024
1 parent accb612 commit 117d03a
Show file tree
Hide file tree
Showing 2 changed files with 26 additions and 12 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -153,6 +153,9 @@ private CompletableFuture<DataStreamReply> combineHeader(CompletableFuture<DataS

private CompletableFuture<DataStreamReply> writeAsyncImpl(Object data, long length, Iterable<WriteOption> options) {
if (isClosed()) {
if (data instanceof ByteBuf) {
((ByteBuf) data).release();
}
return JavaUtils.completeExceptionally(new AlreadyClosedException(
clientId + ": stream already closed, request=" + header));
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -211,6 +211,12 @@ private Set<RaftPeer> getSuccessors(RaftPeerId peerId) {

return Collections.emptySet();
}

void cleanUp(ClientInvocationId invocationId) {
getDivision().getDataStreamMap().remove(invocationId);
getLocal().cleanUp();
applyToRemotes(remote -> remote.out.closeAsync());
}
}

private final RaftServer server;
Expand Down Expand Up @@ -301,6 +307,9 @@ static long writeTo(ByteBuf buf, Iterable<WriteOption> options,
final DataChannel channel = stream.getDataChannel();
long byteWritten = 0;
for (ByteBuffer buffer : buf.nioBuffers()) {
if (buffer.remaining() == 0) {
continue;
}
final ReferenceCountedObject<ByteBuffer> wrapped = ReferenceCountedObject.wrap(
buffer, buf::retain, ignored -> buf.release());
try(UncheckedAutoCloseable ignore = wrapped.retainAndReleaseOnClose()) {
Expand Down Expand Up @@ -389,9 +398,7 @@ static void sendDataStreamException(Throwable throwable, DataStreamRequestByteBu

void cleanUp(Set<ClientInvocationId> ids) {
for (ClientInvocationId clientInvocationId : ids) {
Optional.ofNullable(streams.remove(clientInvocationId))
.map(StreamInfo::getLocal)
.ifPresent(LocalStream::cleanUp);
removeDataStream(clientInvocationId);
}
}

Expand All @@ -411,19 +418,16 @@ void read(DataStreamRequestByteBuf request, ChannelHandlerContext ctx,
readImpl(request, ctx, getStreams);
} catch (Throwable t) {
replyDataStreamException(t, request, ctx);
removeDataStream(ClientInvocationId.valueOf(request.getClientId(), request.getStreamId()), null);
removeDataStream(ClientInvocationId.valueOf(request.getClientId(), request.getStreamId()));
}
}

private void removeDataStream(ClientInvocationId invocationId, StreamInfo info) {
private StreamInfo removeDataStream(ClientInvocationId invocationId) {
final StreamInfo removed = streams.remove(invocationId);
if (info == null) {
info = removed;
}
if (info != null) {
info.getDivision().getDataStreamMap().remove(invocationId);
info.getLocal().cleanUp();
if (removed != null) {
removed.cleanUp(invocationId);
}
return removed;
}

private void readImpl(DataStreamRequestByteBuf request, ChannelHandlerContext ctx,
Expand Down Expand Up @@ -479,7 +483,14 @@ private void readImpl(DataStreamRequestByteBuf request, ChannelHandlerContext ct
try {
if (exception != null) {
replyDataStreamException(server, exception, info.getRequest(), request, ctx);
removeDataStream(key, info);
final StreamInfo removed = removeDataStream(key);
if (removed != null) {
Preconditions.assertSame(info, removed, "removed");
} else {
info.cleanUp(key);
}
} else if (close) {
info.applyToRemotes(remote -> remote.out.closeAsync());
}
} finally {
request.release();
Expand Down

0 comments on commit 117d03a

Please sign in to comment.