Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

RATIS-2197. Clean remote stream to resolve direct memory leak #1179

Open
wants to merge 5 commits into
base: master
Choose a base branch
from
Open
Changes from 2 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
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 @@ -390,8 +396,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);
.ifPresent(streamInfo -> streamInfo.cleanUp(clientInvocationId));
symious marked this conversation as resolved.
Show resolved Hide resolved
}
}

Expand All @@ -411,19 +416,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 @@ -459,7 +461,12 @@ private void readImpl(DataStreamRequestByteBuf request, ChannelHandlerContext ct
localWrite = CompletableFuture.completedFuture(0L);
remoteWrites = Collections.emptyList();
} else if (request.getType() == Type.STREAM_DATA) {
localWrite = info.getLocal().write(request.slice(), request.getWriteOptionList(), writeExecutor);
if (close && request.getDataLength() == 0) {
szetszwo marked this conversation as resolved.
Show resolved Hide resolved
info.getLocal().cleanUp();
localWrite = CompletableFuture.completedFuture(0L);
} else {
localWrite = info.getLocal().write(request.slice(), request.getWriteOptionList(), writeExecutor);
}
remoteWrites = info.applyToRemotes(out -> out.write(request, requestExecutor));
} else {
throw new IllegalStateException(this + ": Unexpected type " + request.getType() + ", request=" + request);
Expand All @@ -479,7 +486,10 @@ 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");
}
}
} finally {
request.release();
Expand Down
Loading