incubator-uniffle icon indicating copy to clipboard operation
incubator-uniffle copied to clipboard

[#2060]fix(server): Fix memory leak when reach memory limit

Open lwllvyb opened this issue 1 year ago • 11 comments

What changes were proposed in this pull request?

Fix shuffle server memory leak when reach memory limit.

Why are the changes needed?

Enable netty. One the shuffle server side, Netty will allocate memory for SEND_SHUFFLE_DATA_REQUEST request. However, when the memory limit is reached, an OutOfDirectMemoryError will be shown and the decode for this message will fail. This will cause the bytebuf allocated successfully in the previous batch in this message to not be released, resulting in memory leak.

Fix: https://github.com/apache/incubator-uniffle/issues/2060

Does this PR introduce any user-facing change?

No.

How was this patch tested?

(Please test your changes, and provide instructions on how to test it:

If you add a feature or fix a bug, add a test to cover your changes. If you fix a flaky test, repeat it for many times to prove it works.)

lwllvyb avatar Aug 19 '24 07:08 lwllvyb

The error stack:

[2024-08-08 20:49:31.754] [epollEventLoopGroup-3-2] [ERROR] ShuffleServerNettyHandler - exception caught /*:38381 io.netty.util.internal.OutOfDirectMemoryError: failed to allocate 65536 byte(s) of direct memory (used: 17179815063, max: 17179869184) at io.netty.util.internal.PlatformDependent.incrementMemoryCounter(PlatformDependent.java:843) at io.netty.util.internal.PlatformDependent.allocateDirectNoCleaner(PlatformDependent.java:772) at io.netty.buffer.UnpooledUnsafeNoCleanerDirectByteBuf.allocateDirect(UnpooledUnsafeNoCleanerDirectByteBuf.java:30) at io.netty.buffer.UnpooledByteBufAllocator$InstrumentedUnpooledUnsafeNoCleanerDirectByteBuf.allocateDirect(UnpooledByteBufAllocator.java:186) at io.netty.buffer.UnpooledDirectByteBuf.(UnpooledDirectByteBuf.java:64) at io.netty.buffer.UnpooledUnsafeDirectByteBuf.(UnpooledUnsafeDirectByteBuf.java:41) at io.netty.buffer.UnpooledUnsafeNoCleanerDirectByteBuf.(UnpooledUnsafeNoCleanerDirectByteBuf.java:25) at io.netty.buffer.UnpooledByteBufAllocator$InstrumentedUnpooledUnsafeNoCleanerDirectByteBuf.(UnpooledByteBufAllocator.java:181) at io.netty.buffer.UnpooledByteBufAllocator.newDirectBuffer(UnpooledByteBufAllocator.java:91) at io.netty.buffer.AbstractByteBufAllocator.directBuffer(AbstractByteBufAllocator.java:188) at io.netty.buffer.AbstractByteBufAllocator.directBuffer(AbstractByteBufAllocator.java:179) at io.netty.channel.unix.PreferredDirectByteBufAllocator.ioBuffer(PreferredDirectByteBufAllocator.java:53) at io.netty.channel.DefaultMaxMessagesRecvByteBufAllocator$MaxMessageHandle.allocate(DefaultMaxMessagesRecvByteBufAllocator.java:120) at io.netty.channel.epoll.EpollRecvByteAllocatorHandle.allocate(EpollRecvByteAllocatorHandle.java:75) at io.netty.channel.epoll.AbstractEpollStreamChannel$EpollStreamUnsafe.epollInReady(AbstractEpollStreamChannel.java:786) at io.netty.channel.epoll.EpollEventLoop.processReady(EpollEventLoop.java:501) at io.netty.channel.epoll.EpollEventLoop.run(EpollEventLoop.java:399) at io.netty.util.concurrent.SingleThreadEventExecutor$4.run(SingleThreadEventExecutor.java:997) at io.netty.util.internal.ThreadExecutorMap$2.run(ThreadExecutorMap.java:74) at io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30) at java.lang.Thread.run(Thread.java:750)

lwllvyb avatar Aug 19 '24 07:08 lwllvyb

Test Results

 2 926 files  +31   2 926 suites  +31   5h 57m 50s :stopwatch: + 10m 45s  1 043 tests ± 0   1 041 :white_check_mark: + 1   2 :zzz: ±0  0 :x:  - 1  13 033 runs  +60  13 003 :white_check_mark: +61  30 :zzz: ±0  0 :x:  - 1 

Results for commit 6903a211. ± Comparison against base commit f9b4c0e0.

:recycle: This comment has been updated with latest results.

github-actions[bot] avatar Aug 19 '24 08:08 github-actions[bot]

I think we need to first avoid the occurrence of OOM. The occurrence of OOM is either due to unreasonable memory configurations or bugs in the code. As for how the code should handle after OOM, I don't think it's very important, because the server has already malfunctioned at this point. Even if there is a memory leak, it's actually not important anymore.

rickyma avatar Aug 19 '24 12:08 rickyma

I think we need to first avoid the occurrence of OOM. The occurrence of OOM is either due to unreasonable memory configurations or bugs in the code. As for how the code should handle after OOM, I don't think it's very important, because the server has already malfunctioned at this point. Even if there is a memory leak, it's actually not important anymore.

So can this PR be accepted ? Or should I just cancel this PR? @rickyma

lwllvyb avatar Aug 28 '24 06:08 lwllvyb

I'm OK with this PR. But it's meaningless. When an OOM error occurs, this PR will not help much.

rickyma avatar Aug 28 '24 07:08 rickyma

I'm OK with this PR. But it's meaningless. When an OOM error occurs, this PR will not help much.

This PR is not to prevent the OOM exception, but to ensure that the pre-allocated ByteBuf can be released normally.

lwllvyb avatar Aug 28 '24 08:08 lwllvyb

I'm OK with this PR. But it's meaningless. When an OOM error occurs, this PR will not help much.

This PR is not to prevent the OOM exception, but to ensure that the pre-allocated ByteBuf can be released normally.

You shouldn't catch OOM exception. If it throws OOM, more errors may throw. You can't recover it by just catching it.

jerqi avatar Aug 28 '24 09:08 jerqi

I'm OK with this PR. But it's meaningless. When an OOM error occurs, this PR will not help much.

This PR is not to prevent the OOM exception, but to ensure that the pre-allocated ByteBuf can be released normally.

You shouldn't catch OOM exception. If it throws OOM, more errors may throw. You can't recover it by just catching it.

I don't understand what you mean. If the OOM exception happen, how we deal with the pre-allocated ByteBuf ? Reconfigure and restart the server , or other ways?

lwllvyb avatar Aug 28 '24 12:08 lwllvyb

I'm OK with this PR. But it's meaningless. When an OOM error occurs, this PR will not help much.

This PR is not to prevent the OOM exception, but to ensure that the pre-allocated ByteBuf can be released normally.

You shouldn't catch OOM exception. If it throws OOM, more errors may throw. You can't recover it by just catching it.

I don't understand what you mean. If the OOM exception happen, how we deal with the pre-allocated ByteBuf ? Reconfigure and restart the server , or other ways?

If it will OOM, the java process should exit.

jerqi avatar Aug 28 '24 12:08 jerqi

I'm OK with this PR. But it's meaningless. When an OOM error occurs, this PR will not help much.

This PR is not to prevent the OOM exception, but to ensure that the pre-allocated ByteBuf can be released normally.

You shouldn't catch OOM exception. If it throws OOM, more errors may throw. You can't recover it by just catching it.

I don't understand what you mean. If the OOM exception happen, how we deal with the pre-allocated ByteBuf ? Reconfigure and restart the server , or other ways?

If it will OOM, the java process should exit.

I need to clarify that it is not OOM but OutOfDirectMemoryError. From the stack trace, we can see that the server did not exit.

lwllvyb avatar Aug 30 '24 10:08 lwllvyb

Yeah, it is a Netty's internal OOM error. It's meaningless to catch this exception. On the other hand, this PR is harmless.

So I choose to remain neutral.

rickyma avatar Aug 30 '24 11:08 rickyma

I have also encountered such a problem and fixed it with a similar method. Although I also think that we can solve this problem by increasing MaxDirectMemorySize or reducing rss.server.buffer.capacity. But it is difficult for many people to find this reasonable value. If at a certain peak usage moment, direct memory happens to exceed MaxDirectMemorySize, leaks will occur. I think it is necessary to merge this pr.

zhengchenyu avatar Sep 24 '24 06:09 zhengchenyu

I'm OK with this PR. But it's meaningless. When an OOM error occurs, this PR will not help much.

This PR is not to prevent the OOM exception, but to ensure that the pre-allocated ByteBuf can be released normally.

You shouldn't catch OOM exception. If it throws OOM, more errors may throw. You can't recover it by just catching it.

I don't understand what you mean. If the OOM exception happen, how we deal with the pre-allocated ByteBuf ? Reconfigure and restart the server , or other ways?

If it will OOM, the java process should exit.

This OOM is not java OOM. The OOM is that the direct memory used exceeds the direct memory limit of Netty. If Netty's direct memory limit is too low, it will easily trigger io.netty.util.internal.OutOfDirectMemoryError without causing java process exit.

zhengchenyu avatar Sep 24 '24 06:09 zhengchenyu

I'm OK with this PR. But it's meaningless. When an OOM error occurs, this PR will not help much.

This PR is not to prevent the OOM exception, but to ensure that the pre-allocated ByteBuf can be released normally.

You shouldn't catch OOM exception. If it throws OOM, more errors may throw. You can't recover it by just catching it.

I don't understand what you mean. If the OOM exception happen, how we deal with the pre-allocated ByteBuf ? Reconfigure and restart the server , or other ways?

If it will OOM, the java process should exit.

This OOM is not java OOM. The OOM is that the direct memory used exceeds the direct memory limit of Netty. If Netty's direct memory limit is too low, it will easily trigger io.netty.util.internal.OutOfDirectMemoryError without causing java process exit.

Is io.netty.util.internal.OutOfDirectMemoryError a subclass of OutOfMemoryError? It's a little weird to catch an error. Because we can't recover an error usually.

jerqi avatar Sep 24 '24 06:09 jerqi

I'm OK with this PR. But it's meaningless. When an OOM error occurs, this PR will not help much.

This PR is not to prevent the OOM exception, but to ensure that the pre-allocated ByteBuf can be released normally.

You shouldn't catch OOM exception. If it throws OOM, more errors may throw. You can't recover it by just catching it.

I don't understand what you mean. If the OOM exception happen, how we deal with the pre-allocated ByteBuf ? Reconfigure and restart the server , or other ways?

If it will OOM, the java process should exit.

This OOM is not java OOM. The OOM is that the direct memory used exceeds the direct memory limit of Netty. If Netty's direct memory limit is too low, it will easily trigger io.netty.util.internal.OutOfDirectMemoryError without causing java process exit.

Is io.netty.util.internal.OutOfDirectMemoryError a subclass of OutOfMemoryError? It's a little weird to catch an error. Because we can't recover an error usually.

io.netty.util.internal.OutOfDirectMemoryError is not a subclass of OutOfMemoryError. Even though it's a little weird to catch an error, but this error can not make the process exit.

zhengchenyu avatar Sep 24 '24 06:09 zhengchenyu

o.netty.util.internal.OutOfDirectMemoryError

I read the code

package io.netty.util.internal;

import java.nio.ByteBuffer;

/**
 * {@link OutOfMemoryError} that is throws if {@link PlatformDependent#allocateDirectNoCleaner(int)} can not allocate
 * a new {@link ByteBuffer} due memory restrictions.
 */
public final class OutOfDirectMemoryError extends OutOfMemoryError {
    private static final long serialVersionUID = 4228264016184011555L;

    OutOfDirectMemoryError(String s) {
        super(s);
    }
}

jerqi avatar Sep 24 '24 06:09 jerqi

Maybe we should add some comments to explain this.

jerqi avatar Sep 24 '24 06:09 jerqi

Sorry, I write it wrongly. Below is the netty source code. If we set DIRECT_MEMORY_LIMIT to not a big number, will throw OutOfDirectMemoryError. But the jvm process will not exit, but the ByteBuf will be leaked.

    private static void incrementMemoryCounter(int capacity) {
        if (DIRECT_MEMORY_COUNTER != null) {
            long newUsedMemory = DIRECT_MEMORY_COUNTER.addAndGet(capacity);
            if (newUsedMemory > DIRECT_MEMORY_LIMIT) {
                DIRECT_MEMORY_COUNTER.addAndGet(-capacity);
                throw new OutOfDirectMemoryError("failed to allocate " + capacity
                        + " byte(s) of direct memory (used: " + (newUsedMemory - capacity)
                        + ", max: " + DIRECT_MEMORY_LIMIT + ')');
            }
        }
    }

zhengchenyu avatar Sep 24 '24 06:09 zhengchenyu

Sorry, I write it wrongly. Below is the netty source code. If we set DIRECT_MEMORY_LIMIT to not a big number, will throw OutOfDirectMemoryError. But the jvm process will not exit, but the ByteBuf will be leaked.

    private static void incrementMemoryCounter(int capacity) {
        if (DIRECT_MEMORY_COUNTER != null) {
            long newUsedMemory = DIRECT_MEMORY_COUNTER.addAndGet(capacity);
            if (newUsedMemory > DIRECT_MEMORY_LIMIT) {
                DIRECT_MEMORY_COUNTER.addAndGet(-capacity);
                throw new OutOfDirectMemoryError("failed to allocate " + capacity
                        + " byte(s) of direct memory (used: " + (newUsedMemory - capacity)
                        + ", max: " + DIRECT_MEMORY_LIMIT + ')');
            }
        }
    }

Maybe it's ok to add comments to merge this pull request.

jerqi avatar Sep 24 '24 06:09 jerqi

I have also encountered such a problem and fixed it with a similar method. Although I also think that we can solve this problem by increasing MaxDirectMemorySize or reducing rss.server.buffer.capacity. But it is difficult for many people to find this reasonable value. If at a certain peak usage moment, direct memory happens to exceed MaxDirectMemorySize, leaks will occur. I think it is necessary to merge this pr.

If the users follow this guidance, this error should not happen, or else there exists a bug.

rickyma avatar Sep 24 '24 07:09 rickyma

Ping @jerqi @rickyma @zhengchenyu Could you please take some time to review this PR? If it doesn't need to be merged, please let me know. This PR has been verified internally.

lwllvyb avatar Oct 15 '24 02:10 lwllvyb

Could you add some comments for this code?

jerqi avatar Oct 15 '24 09:10 jerqi

Could you add some comments for this code?

Thanks for your review. I added some comments to the code. Please take the time to review again when it is convenient for you. Thanks again. @jerqi

lwllvyb avatar Oct 16 '24 03:10 lwllvyb

Codecov Report

Attention: Patch coverage is 25.00000% with 6 lines in your changes missing coverage. Please review.

Project coverage is 52.86%. Comparing base (5ddcc28) to head (6903a21). Report is 125 commits behind head on master.

Files with missing lines Patch % Lines
.../common/netty/protocol/SendShuffleDataRequest.java 25.00% 6 Missing :warning:
Additional details and impacted files
@@             Coverage Diff              @@
##             master    #2058      +/-   ##
============================================
+ Coverage     52.77%   52.86%   +0.09%     
- Complexity     2498     3422     +924     
============================================
  Files           398      518     +120     
  Lines         18135    28359   +10224     
  Branches       1660     2650     +990     
============================================
+ Hits           9570    14991    +5421     
- Misses         7981    12400    +4419     
- Partials        584      968     +384     

:umbrella: View full report in Codecov by Sentry.
:loudspeaker: Have feedback on the report? Share it here.

codecov-commenter avatar Oct 19 '24 06:10 codecov-commenter