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

[Bug] Direct memory may leak in shuffle server. #1152

Closed
3 tasks done
zhengchenyu opened this issue Aug 17, 2023 · 2 comments · Fixed by #1154
Closed
3 tasks done

[Bug] Direct memory may leak in shuffle server. #1152

zhengchenyu opened this issue Aug 17, 2023 · 2 comments · Fixed by #1154

Comments

@zhengchenyu
Copy link
Collaborator

zhengchenyu commented Aug 17, 2023

Code of Conduct

Search before asking

  • I have searched in the issues and found no similar issues.

Describe the bug

Direct memory may leak in shuffle server.

In client side, some task failed, Found error log:due to io.grpc.StatusRuntimeException: DEADLINE_EXCEEDED: deadline exceeded after 59.999950602s.

And then I found shuffle server have many oom log:

Exception in thread "Grpc-3250" java.lang.OutOfMemoryError: Direct buffer memory
	at java.base/java.nio.Bits.reserveMemory(Bits.java:175)
	at java.base/java.nio.DirectByteBuffer.<init>(DirectByteBuffer.java:118)
	at java.base/java.nio.ByteBuffer.allocateDirect(ByteBuffer.java:317)
	at io.grpc.netty.shaded.io.netty.buffer.PoolArena$DirectArena.allocateDirect(PoolArena.java:648)
	at io.grpc.netty.shaded.io.netty.buffer.PoolArena$DirectArena.newChunk(PoolArena.java:623)
	at io.grpc.netty.shaded.io.netty.buffer.PoolArena.allocateNormal(PoolArena.java:202)
	at io.grpc.netty.shaded.io.netty.buffer.PoolArena.tcacheAllocateNormal(PoolArena.java:186)

Monitor shows that the direct memory of shuffle server have reached 80G (MaxDirectMemorySize is just 80G).

Affects Version(s)

master

Uniffle Server Log Output

No response

Uniffle Engine Log Output

No response

Uniffle Server Configurations

some key configuration:

  1. We did not set netty port.
  2. client type is GRPC, but not GRPC_NETTY
  3. -Xmx200G -XX:MaxDirectMemorySize=80G

Uniffle Engine Configurations

No response

Additional context

No response

Are you willing to submit PR?

  • Yes I am willing to submit a PR!
@zhengchenyu
Copy link
Collaborator Author

zhengchenyu commented Aug 17, 2023

We add these startup options to analyse:

-Dio.netty.leakDetection.level=PARANOID -XX:NativeMemoryTracking=detail

Then we found these logs

[ERROR] 2023-08-16 12:56:11,453 Grpc-146 ResourceLeakDetector reportTracedLeak - LEAK: ByteBuf.release() was not called before it's garbage-collected. See https://netty.io/wiki/reference-counted-objects.html for more information.
Recent access records: 
#1:
        io.netty.buffer.AdvancedLeakAwareByteBuf.order(AdvancedLeakAwareByteBuf.java:71)
        io.netty.buffer.CompositeByteBuf.newComponent(CompositeByteBuf.java:346)
        io.netty.buffer.CompositeByteBuf.consolidate0(CompositeByteBuf.java:1758)
        io.netty.buffer.CompositeByteBuf.consolidateIfNeeded(CompositeByteBuf.java:571)
        io.netty.buffer.CompositeByteBuf.addComponent(CompositeByteBuf.java:266)
        io.netty.buffer.CompositeByteBuf.addComponent(CompositeByteBuf.java:222)
        org.apache.uniffle.server.buffer.ShuffleBuffer.updateShuffleData(ShuffleBuffer.java:272)
        org.apache.uniffle.server.buffer.ShuffleBuffer.getShuffleData(ShuffleBuffer.java:169)
        org.apache.uniffle.server.buffer.ShuffleBufferManager.getShuffleData(ShuffleBufferManager.java:231)
        org.apache.uniffle.server.ShuffleTaskManager.getInMemoryShuffleData(ShuffleTaskManager.java:516)
        org.apache.uniffle.server.ShuffleServerGrpcService.getMemoryShuffleData(ShuffleServerGrpcService.java:798)
        org.apache.uniffle.proto.ShuffleServerGrpc$MethodHandlers.invoke(ShuffleServerGrpc.java:1053)
        io.grpc.stub.ServerCalls$UnaryServerCallHandler$UnaryServerCallListener.onHalfClose(ServerCalls.java:182)
        io.grpc.PartialForwardingServerCallListener.onHalfClose(PartialForwardingServerCallListener.java:35)
        io.grpc.ForwardingServerCallListener.onHalfClose(ForwardingServerCallListener.java:23)
        io.grpc.internal.ServerCallImpl$ServerStreamListenerImpl.halfClosed(ServerCallImpl.java:352)
        io.grpc.internal.ServerImpl$JumpToApplicationThreadServerStreamListener$1HalfClosed.runInContext(ServerImpl.java:866)
        io.grpc.internal.ContextRunnable.run(ContextRunnable.java:37)
        io.grpc.internal.SerializingExecutor.run(SerializingExecutor.java:133)
        java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
        java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
        java.base/java.lang.Thread.run(Thread.java:829)
#2:
        io.netty.buffer.AdvancedLeakAwareByteBuf.writeBytes(AdvancedLeakAwareByteBuf.java:605)
        io.netty.buffer.CompositeByteBuf$Component.transferTo(CompositeByteBuf.java:1927)
        io.netty.buffer.CompositeByteBuf.consolidate0(CompositeByteBuf.java:1754)
        io.netty.buffer.CompositeByteBuf.consolidateIfNeeded(CompositeByteBuf.java:571)
        io.netty.buffer.CompositeByteBuf.addComponent(CompositeByteBuf.java:266)
        io.netty.buffer.CompositeByteBuf.addComponent(CompositeByteBuf.java:222)
        org.apache.uniffle.server.buffer.ShuffleBuffer.updateShuffleData(ShuffleBuffer.java:272)
        org.apache.uniffle.server.buffer.ShuffleBuffer.getShuffleData(ShuffleBuffer.java:169)
        org.apache.uniffle.server.buffer.ShuffleBufferManager.getShuffleData(ShuffleBufferManager.java:231)
        org.apache.uniffle.server.ShuffleTaskManager.getInMemoryShuffleData(ShuffleTaskManager.java:516)
        org.apache.uniffle.server.ShuffleServerGrpcService.getMemoryShuffleData(ShuffleServerGrpcService.java:798)
        org.apache.uniffle.proto.ShuffleServerGrpc$MethodHandlers.invoke(ShuffleServerGrpc.java:1053)
        io.grpc.stub.ServerCalls$UnaryServerCallHandler$UnaryServerCallListener.onHalfClose(ServerCalls.java:182)
        io.grpc.PartialForwardingServerCallListener.onHalfClose(PartialForwardingServerCallListener.java:35)
        io.grpc.ForwardingServerCallListener.onHalfClose(ForwardingServerCallListener.java:23)
        io.grpc.internal.ServerCallImpl$ServerStreamListenerImpl.halfClosed(ServerCallImpl.java:352)
        io.grpc.internal.ServerImpl$JumpToApplicationThreadServerStreamListener$1HalfClosed.runInContext(ServerImpl.java:866)
        io.grpc.internal.ContextRunnable.run(ContextRunnable.java:37)
        io.grpc.internal.SerializingExecutor.run(SerializingExecutor.java:133)
        java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
        java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
        java.base/java.lang.Thread.run(Thread.java:829)
Created at:
        io.netty.buffer.PooledByteBufAllocator.newDirectBuffer(PooledByteBufAllocator.java:402)
        io.netty.buffer.AbstractByteBufAllocator.directBuffer(AbstractByteBufAllocator.java:188)
        io.netty.buffer.AbstractByteBufAllocator.directBuffer(AbstractByteBufAllocator.java:179)
        io.netty.buffer.CompositeByteBuf.allocBuffer(CompositeByteBuf.java:1872)
        io.netty.buffer.CompositeByteBuf.consolidate0(CompositeByteBuf.java:1751)
        io.netty.buffer.CompositeByteBuf.consolidateIfNeeded(CompositeByteBuf.java:571)
        io.netty.buffer.CompositeByteBuf.addComponent(CompositeByteBuf.java:266)
        io.netty.buffer.CompositeByteBuf.addComponent(CompositeByteBuf.java:222)
        org.apache.uniffle.server.buffer.ShuffleBuffer.updateShuffleData(ShuffleBuffer.java:272)
        org.apache.uniffle.server.buffer.ShuffleBuffer.getShuffleData(ShuffleBuffer.java:169)
        org.apache.uniffle.server.buffer.ShuffleBufferManager.getShuffleData(ShuffleBufferManager.java:231)
        org.apache.uniffle.server.ShuffleTaskManager.getInMemoryShuffleData(ShuffleTaskManager.java:516)
        org.apache.uniffle.server.ShuffleServerGrpcService.getMemoryShuffleData(ShuffleServerGrpcService.java:798)
        org.apache.uniffle.proto.ShuffleServerGrpc$MethodHandlers.invoke(ShuffleServerGrpc.java:1053)
        io.grpc.stub.ServerCalls$UnaryServerCallHandler$UnaryServerCallListener.onHalfClose(ServerCalls.java:182)
        io.grpc.PartialForwardingServerCallListener.onHalfClose(PartialForwardingServerCallListener.java:35)
        io.grpc.ForwardingServerCallListener.onHalfClose(ForwardingServerCallListener.java:23)
        io.grpc.internal.ServerCallImpl$ServerStreamListenerImpl.halfClosed(ServerCallImpl.java:352)
        io.grpc.internal.ServerImpl$JumpToApplicationThreadServerStreamListener$1HalfClosed.runInContext(ServerImpl.java:866)
        io.grpc.internal.ContextRunnable.run(ContextRunnable.java:37)
        io.grpc.internal.SerializingExecutor.run(SerializingExecutor.java:133)
        java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
        java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
        java.base/java.lang.Thread.run(Thread.java:829)

We found NettyManagedBuffer in

return new ShuffleDataResult(byteBuf, bufferSegments);
is never released in GRPC mode.

I search the code. ManagedBuffer::release is only called in NETTY mode.

@jerqi
Copy link
Contributor

jerqi commented Aug 17, 2023

Great work!

zhengchenyu added a commit to zhengchenyu/incubator-uniffle that referenced this issue Aug 17, 2023
jerqi pushed a commit that referenced this issue Aug 17, 2023
### What changes were proposed in this pull request?

Release buffer when since the buffer have copied to data of reply.

### Why are the changes needed?

Fix: #1152

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

Test in cluster 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 a pull request may close this issue.

2 participants