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] Memory leakage detected when IndexStoreService is enabled and TieredMessageStore's FileSegment is PosixFileSegment #8017
Comments
We may introduce a DirectByteBuffer pool like TransientStorePool or use MappedByteBuffer |
|
Thx for reply, I tried
It turns out the cq data was not flush correctly by debugging, I wonder if the way I use
After:
|
It seems the modification is not equivalent. You might extend the file of |
Agree with you. But then I was wondering if the failed unittest necessary, it seems acting like:
Is it acceptable to remove step 1 in Furthermore, |
@bxfjb If you want to extend the size of writeFileChannel.write(ByteBuffer.wrap(new byte[]{0}), position - 1); |
Eventually I decide to use |
The default implementation in the current code is DefaultMetadataStore and PosixFileSegment. |
Adding a new file type for index looks good |
In general mmap has a greater impact on page cache and may cause resource contention with normal commit log. We need to evaluate the impact on overall performance carefully. |
If the target storage is based on the POSIX protocol, using directIO is a good choice. |
Got it, I am doing stress tests to reveal the impact as possible as I can. So far it looks good under about 60k producer TPS. |
I chose
I wonder what do you mean about |
对于 SSD 转 HDD 场景,当前设计中 PosixFileSegment 主要是为了测试场景,可以改进。如果用 MappedByteBuffer,把数据从 CommitLog 复制到 FileSegment,也会有 pagecache 浪费内存的问题。至于如何实现,可以搜索 Java directIO lib For copy data from SSD to HDD scenario, the PosixFileSegment in the current design is mainly for testing scenarios and can be improved. If you use MappedByteBuffer to copy data from CommitLog to FileSegment, there will also be a problem of pagecache wasting memory. We can use Java directIO lib to slove this problem |
Before Creating the Bug Report
I found a bug, not just asking a question, which should be created in GitHub Discussions.
I have searched the GitHub Issues and GitHub Discussions of this repository and believe that this is not a duplicate.
I have confirmed that this bug belongs to the current repository, not other repositories of RocketMQ.
Runtime platform environment
CentOS 7
RocketMQ version
branch: develop
JDK Version
OpenJDK 8
Describe the Bug
IndexStoreService.java
:Add a watch of direct memory when debugging the UT
doCompactionTest
atIndexStoreServiceTest.java
can prove thatdoCompactThenUploadFile
may cause direct memory leak.By diving in the code, the root cause actually is inappropriate use of FileChannel at
PosixFileSegment.java
:writeFileChannel.write(buffer);
would allocate a DirectByteBuffer whose size is same withbuffer
, which means in uploading of IndexFile, the DirectByteBuffer's size will be about 570MB. This DirectByteBuffer is an element of an array of ByteBuffer of a ThreadLocal variable insun.nio.ch.Util.class
:writeFileChannel.write(buffer)
is:buffer
, if there is not, allocate one;which means that the direct memory allocated would not be released until the thread is killed, which will never happen for teh thread runs
commit0
is in a thread pool of MessageStoreExecutor.Reproducing the bug with personal code is simple, writing a class like below, call
FileWrite.run()
and open jvisualvm, the increasing of direct buffer should be shown in Buffer Pools.Atention that the size of thread pool will limit the increase of buffer usage, for BufferCache of each thread is reusable, the increase should stop as each thread in thread pool runs
writeAsync()
at least once.For example, reduce the size of trhead pool to 4:
Steps to Reproduce
2 choices:
What Did You Expect to See?
The direct memory usage should not increase unlimitedly.
What Did You See Instead?
The direct memory usage keeps on increasing until OOM or all threads in commitExecutor run commit0 at least once.
Additional Context
There might be some solution of this bug:
-Djdk.nio.maxCachedBufferSize
;PosixFileSegment.commit0
's parameter list;The text was updated successfully, but these errors were encountered: