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

Conversation

symious
Copy link
Contributor

@symious symious commented Nov 21, 2024

What changes were proposed in this pull request?

During perssure test using ratis streaming, we found direct memory leak.

After days of debugging, we found it's caused by uncleaned remote stream.

What is the link to the Apache JIRA

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

How was this patch tested?

Local performance test.

Copy link
Contributor

@szetszwo szetszwo left a comment

Choose a reason for hiding this comment

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

@symious , Good catch on the bug! The change looks good. Just some minor suggestions inlined. See also https://issues.apache.org/jira/secure/attachment/13072995/1179_review.patch

Comment on lines 429 to 431
info.getDivision().getDataStreamMap().remove(invocationId);
info.getLocal().cleanUp();
info.applyToRemotes(out -> out.out.closeAsync());
Copy link
Contributor

Choose a reason for hiding this comment

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

Let's add a cleanUp() method to StreamInfo.

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

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Updated, PTAL

Comment on lines 393 to 399
.ifPresent(streamInfo -> {
streamInfo.getDivision()
.getDataStreamMap()
.remove(clientInvocationId);
streamInfo.getLocal().cleanUp();
streamInfo.applyToRemotes(out -> out.out.closeAsync());
});
Copy link
Contributor

Choose a reason for hiding this comment

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

Let's use the removeDataStream method here.

BTW, we should update it to not passing StreamInfo and then check if the removed steam is the same. If not, call cleanUp(..). See below.

-  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,
@@ -479,7 +480,10 @@ public class DataStreamManagement {
       try {
         if (exception != null) {
           replyDataStreamException(server, exception, info.getRequest(), request, ctx);
-          removeDataStream(key, info);
+          final StreamInfo removed = removeDataStream(key);
+          if (removed != info) {
+            info.cleanUp(key);
+          }
         }
       } finally {
         request.release();

Copy link
Contributor Author

Choose a reason for hiding this comment

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

@szetszwo Thanks for the review.

+          if (removed != info) {
+            info.cleanUp(key);
+          }

For the above code, I was wondering in what cases will "removed not equal to info". I think in the original code, info was checked because in catch exception in read() we don't know the info and we give an "null", but here info should be equal to removed?

Copy link
Contributor

Choose a reason for hiding this comment

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

@symious , You are right that it won't not be different. Let's add a precondition then.

          if (removed != null) {
            Preconditions.assertSame(info, removed, "removed");
          }

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Updated, PTAL.

@symious
Copy link
Contributor Author

symious commented Nov 23, 2024

Handle close for local stream specifically to resolve the following error.

2024-11-22 16:58:34,074 [a07093ab-6bbb-48c7-ae8d-2c67b5270e69-ChunkWriter-3-0] WARN org.apache.ratis.netty.server.DataStreamManagement: Failed to process DataStreamRequestByteBuf:clientId=client-C97A2E634E73,type=STREAM_DATA,id=568891,offset=78643200,length=0,timeoutMs=1732265937487
java.util.concurrent.CompletionException: Failed to close org.apache.hadoop.ozone.container.common.transport.server.ratis.LocalStream@78fa8311
        at org.apache.ratis.netty.server.DataStreamManagement.close(DataStreamManagement.java:342)
        at org.apache.ratis.netty.server.DataStreamManagement.writeTo(DataStreamManagement.java:333)
        at org.apache.ratis.netty.server.DataStreamManagement.lambda$writeToAsync$5(DataStreamManagement.java:307)
        at java.util.concurrent.CompletableFuture$AsyncSupply.run(CompletableFuture.java:1604)
        at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
        at java.lang.Thread.run(Thread.java:748)
Caused by: java.io.IOException: Failed to readPutBlockRequest from ReadOnlyByteBuf(ridx: 0, widx: 1048576, cap: 1048576, unwrapped: CompositeByteBuf(ridx: 0, widx: 1048576, cap: 1048576, components=17)): readerIndex=0, protoIndex=-1111115338, protoLength=1112163910, lengthIndex=1048572
        at org.apache.hadoop.ozone.container.keyvalue.impl.KeyValueStreamDataChannel.readPutBlockRequest(KeyValueStreamDataChannel.java:273)
        at org.apache.hadoop.ozone.container.keyvalue.impl.KeyValueStreamDataChannel.closeBuffers(KeyValueStreamDataChannel.java:236)
        at org.apache.hadoop.ozone.container.keyvalue.impl.KeyValueStreamDataChannel.close(KeyValueStreamDataChannel.java:215)
        at org.apache.ratis.netty.server.DataStreamManagement.close(DataStreamManagement.java:340)
        ... 6 more
Caused by: java.lang.IndexOutOfBoundsException: CompositeByteBuf(ridx: 0, widx: 1048576, cap: 1048576, components=17).slice(-1111115338, 1112163910)
        at org.apache.ratis.thirdparty.io.netty.buffer.AbstractUnpooledSlicedByteBuf.checkSliceOutOfBounds(AbstractUnpooledSlicedByteBuf.java:474)
        at org.apache.ratis.thirdparty.io.netty.buffer.AbstractUnpooledSlicedByteBuf.<init>(AbstractUnpooledSlicedByteBuf.java:38)
        at org.apache.ratis.thirdparty.io.netty.buffer.UnpooledSlicedByteBuf.<init>(UnpooledSlicedByteBuf.java:24)
        at org.apache.ratis.thirdparty.io.netty.buffer.AbstractByteBuf.slice(AbstractByteBuf.java:1221)
        at org.apache.ratis.thirdparty.io.netty.buffer.ReadOnlyByteBuf.slice(ReadOnlyByteBuf.java:298)
        at org.apache.hadoop.ozone.container.keyvalue.impl.KeyValueStreamDataChannel.readPutBlockRequest(KeyValueStreamDataChannel.java:270)
        ... 9 more

Copy link
Contributor

@szetszwo szetszwo left a comment

Choose a reason for hiding this comment

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

@symious , thanks for the update! Since some stream tests failed, I think about the code more. Please see the comments inlined and also https://issues.apache.org/jira/secure/attachment/13073035/1179_review2.patch

Comment on lines 464 to 468
if (close && request.getDataLength() == 0) {
localWrite = CompletableFuture.completedFuture(0L);
} else {
localWrite = info.getLocal().write(request.slice(), request.getWriteOptionList(), writeExecutor);
}
Copy link
Contributor

Choose a reason for hiding this comment

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

This part is an optimization for not calling getLocal().write(..). However, getLocal().write(..) also takes care the WriteOptions such as SYNC and FLUSH. So, we cannot skip it here. We could skip it later on in

//writeTo(..)
@@ -301,6 +307,9 @@ public class DataStreamManagement {
     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()) {

Copy link
Contributor Author

Choose a reason for hiding this comment

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

And the length check here, maybe we can remove it? since it will always get into here. https://github.com/apache/ratis/blob/master/ratis-netty/src/main/java/org/apache/ratis/netty/server/DataStreamManagement.java#L322

Copy link
Contributor

Choose a reason for hiding this comment

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

For length check, do you mean the new code?

if (buffer.remaining() == 0) {
   continue;
}

It is just a minor improvement. The value buffer.remaining() is likely non-zero for most cases. I am fine if we don't want to add it.

@@ -479,7 +485,12 @@ private void readImpl(DataStreamRequestByteBuf request, ChannelHandlerContext ct
try {
if (exception != null) {
replyDataStreamException(server, exception, info.getRequest(), request, ctx);
removeDataStream(key, info);
}
if (close || exception != null) {
Copy link
Contributor

Choose a reason for hiding this comment

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

For normal close case,

  • the local stream (to a local file) is a DataStream created by an application. It should be removed from the DataStreamMap and closed during link(..).
  • Remote streams (server to server) should be closed here. This is the leak.

To play safe, call info.cleanUp(key) when removed == null. The code becomes

        if (exception != null) {
          replyDataStreamException(server, exception, info.getRequest(), request, ctx);
          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());
        }

Copy link
Contributor Author

Choose a reason for hiding this comment

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

 else if (close) {
          info.applyToRemotes(remote -> remote.out.closeAsync());
        }

Do we need this? remotes should have done this in remote write.

Copy link
Contributor

Choose a reason for hiding this comment

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

Could you show the code? I cannot find the code closing the remote streams.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Copy link
Contributor

Choose a reason for hiding this comment

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

It will call DataStreamOutputImpl.writeAsync but not DataStreamOutputImpl.close().

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 it's a close message, writeAsync will send the close message to the next peer.

info.applyToRemotes(remote -> remote.out.closeAsync()); will send an empty close message again.

@symious
Copy link
Contributor Author

symious commented Nov 25, 2024

@szetszwo Updated the patch, PTAL.

Copy link
Contributor

@szetszwo szetszwo left a comment

Choose a reason for hiding this comment

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

@symious , thanks for the update! Just have a comment inlined.

Comment on lines +156 to +158
if (data instanceof ByteBuf) {
((ByteBuf) data).release();
}
Copy link
Contributor

Choose a reason for hiding this comment

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

For the ByteBuf, it is only used in DataStreamManagement for forwarding a request to remote. The request will be released at the end. So, we should not release here.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Copy link
Contributor

Choose a reason for hiding this comment

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

DataStreamRequestByteBuf has a release() method:

and the request is released in finally-blocks.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

But here, the request is retained twice, once in "decode", once in remote write. For release only once in finally-blocks, if so, resource will be leak.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

For this part, since ByteBuf is retained for remote write, the refCount will only decrease on a successful write operation (ctx.writeAndFlush() or ctx.write), but if stream is closed, the write operation is not done, so need to be release here.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Can I ask where is this ByteBuf released? (https://github.com/apache/ratis/blob/master/ratis-netty/src/main/java/org/apache/ratis/netty/server/DataStreamManagement.java#L134)

This one will be released on outbound write.

The final release I think is for the retain here. (https://github.com/apache/ratis/blob/master/ratis-netty/src/main/java/org/apache/ratis/netty/NettyDataStreamUtils.java#L165)

The final release is for the very first initialize of the ByteBuf. The retain in decode will be release after the decode itself.

Copy link
Contributor

Choose a reason for hiding this comment

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

This one will be released on outbound write.

The final release is for the very first initialize of the ByteBuf. The retain in decode will be release after the decode itself.

The bufs seem to be released correctly. Do you agree?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

This one will be released on outbound write.

The ByteBuf in the code should be here. So the retain needs to be released on outbound write. If stream closed, we need to release it manually.

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