Open bxfjb opened 6 months ago
We may introduce a DirectByteBuffer pool like TransientStorePool or use MappedByteBuffer
ByteStreams.toByteArray()
creates large array in memory and is unnecessary. It seems FileChannel.transferFrom()
with NIO channel would be more efficient for writing streams to a file.
ByteStreams.toByteArray()
creates large array in memory and is unnecessary. It seemsFileChannel.transferFrom()
with NIO channel would be more efficient for writing streams to a file.
Thx for reply, I tried FileChannel.transferFrom()
but find out some unittest failed:
@Test
public void consumeQueueTest() throws ClassNotFoundException, NoSuchMethodException {
...
ByteBuffer cqItem1 = fileSegment.read(initPosition, unitSize);
Assert.assertEquals(baseOffset, cqItem1.getLong()); // failed
...
}
java.lang.AssertionError: Expected :1000 Actual :0
It turns out the cq data was not flush correctly by debugging, I wonder if the way I use FileChannel.transferFrom()
is wrong:
Before:
byte[] byteArray = ByteStreams.toByteArray(inputStream);
writeFileChannel.position(position);
ByteBuffer buffer = ByteBuffer.wrap(byteArray);
while (buffer.hasRemaining()) {
writeFileChannel.write(buffer);
}
writeFileChannel.force(true);
After:
ReadableByteChannel readableByteChannel = Channels.newChannel(inputStream);
writeFileChannel.transferFrom(readableByteChannel, position, length);
writeFileChannel.force(true);
ByteStreams.toByteArray()
creates large array in memory and is unnecessary. It seemsFileChannel.transferFrom()
with NIO channel would be more efficient for writing streams to a file.Thx for reply, I tried
FileChannel.transferFrom()
but find out some unittest failed:@Test public void consumeQueueTest() throws ClassNotFoundException, NoSuchMethodException { ... ByteBuffer cqItem1 = fileSegment.read(initPosition, unitSize); Assert.assertEquals(baseOffset, cqItem1.getLong()); // failed ... }
java.lang.AssertionError: Expected :1000 Actual :0
It turns out the cq data was not flush correctly by debugging, I wonder if the way I use
FileChannel.transferFrom()
is wrong: Before:byte[] byteArray = ByteStreams.toByteArray(inputStream); writeFileChannel.position(position); ByteBuffer buffer = ByteBuffer.wrap(byteArray); while (buffer.hasRemaining()) { writeFileChannel.write(buffer); } writeFileChannel.force(true);
After:
ReadableByteChannel readableByteChannel = Channels.newChannel(inputStream); writeFileChannel.transferFrom(readableByteChannel, position, length); writeFileChannel.force(true);
It seems the modification is not equivalent. writeFileChannel.position(position)
has side effect, making writeFileChannel
written from position
. But writeFileChannel.transferFrom()
exits when position
is larger than the size of writeFileChannel
.
You might extend the file of writeFileChannel
to position
and also check if the transferred bytes is equal to length
.
ByteStreams.toByteArray()
creates large array in memory and is unnecessary. It seemsFileChannel.transferFrom()
with NIO channel would be more efficient for writing streams to a file.Thx for reply, I tried
FileChannel.transferFrom()
but find out some unittest failed:@Test public void consumeQueueTest() throws ClassNotFoundException, NoSuchMethodException { ... ByteBuffer cqItem1 = fileSegment.read(initPosition, unitSize); Assert.assertEquals(baseOffset, cqItem1.getLong()); // failed ... }
java.lang.AssertionError: Expected :1000 Actual :0
It turns out the cq data was not flush correctly by debugging, I wonder if the way I use
FileChannel.transferFrom()
is wrong: Before:byte[] byteArray = ByteStreams.toByteArray(inputStream); writeFileChannel.position(position); ByteBuffer buffer = ByteBuffer.wrap(byteArray); while (buffer.hasRemaining()) { writeFileChannel.write(buffer); } writeFileChannel.force(true);
After:
ReadableByteChannel readableByteChannel = Channels.newChannel(inputStream); writeFileChannel.transferFrom(readableByteChannel, position, length); writeFileChannel.force(true);
It seems the modification is not equivalent.
writeFileChannel.position(position)
has side effect, makingwriteFileChannel
written fromposition
. ButwriteFileChannel.transferFrom()
exits whenposition
is larger than the size ofwriteFileChannel
.You might extend the file of
writeFileChannel
toposition
and also check if the transferred bytes is equal tolength
.
Agree with you. But then I was wondering if the failed unittest necessary, it seems acting like:
commitPosition
is larger than the size of writeFileChannel
, commit failedIs it acceptable to remove step 1 in consumeQueueTest
? @lizhimins 'cause the scenario seems nonexistent in production.
Furthermore, FileChannel.read()
has the same problem with FileChannel.write()
, but to return a ByteBuffer
, calling it looks like inevitable. I didn't find out how to use FileChannel.transferTo()
to replace it completely. I'd like to use MappedByteBuffer
as a alternative.
@bxfjb If you want to extend the size of writeFileChannel
to make FileChannel.transferTo()
successful, you may try writing a byte at position - 1
:
writeFileChannel.write(ByteBuffer.wrap(new byte[]{0}), position - 1);
@bxfjb If you want to extend the size of
writeFileChannel
to makeFileChannel.transferTo()
successful, you may try writing a byte atposition - 1
:writeFileChannel.write(ByteBuffer.wrap(new byte[]{0}), position - 1);
Eventually I decide to use MappedByteBuffer
for FileChannel.transferFrom()
's worse IO preformance in tests, could you pls review the #8036?
The default implementation in the current code is DefaultMetadataStore and PosixFileSegment. In the original design, PosixFileSegment only as test cases and validation purposes. We used metadata management based on Rocksdb implementation and FileSegment based on object storage or HDFS implementation in out production. In order to achieve higher performance and lower storage costs, this part of the code will contribute in future pr.
Adding a new file type for index looks good
@bxfjb If you want to extend the size of
writeFileChannel
to makeFileChannel.transferTo()
successful, you may try writing a byte atposition - 1
:writeFileChannel.write(ByteBuffer.wrap(new byte[]{0}), position - 1);
Eventually I decide to use
MappedByteBuffer
forFileChannel.transferFrom()
's worse IO preformance in tests, could you pls review the #8036?
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.
@bxfjb If you want to extend the size of
writeFileChannel
to makeFileChannel.transferTo()
successful, you may try writing a byte atposition - 1
:writeFileChannel.write(ByteBuffer.wrap(new byte[]{0}), position - 1);
Eventually I decide to use
MappedByteBuffer
forFileChannel.transferFrom()
's worse IO preformance in tests, could you pls review the #8036?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.
@bxfjb If you want to extend the size of
writeFileChannel
to makeFileChannel.transferTo()
successful, you may try writing a byte atposition - 1
:writeFileChannel.write(ByteBuffer.wrap(new byte[]{0}), position - 1);
Eventually I decide to use
MappedByteBuffer
forFileChannel.transferFrom()
's worse IO preformance in tests, could you pls review the #8036?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.
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.
@bxfjb If you want to extend the size of
writeFileChannel
to makeFileChannel.transferTo()
successful, you may try writing a byte atposition - 1
:writeFileChannel.write(ByteBuffer.wrap(new byte[]{0}), position - 1);
Eventually I decide to use
MappedByteBuffer
forFileChannel.transferFrom()
's worse IO preformance in tests, could you pls review the #8036?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.
I chose MappedByteBuffer
cuz that:
FileChannel.write()/read()
is banned for memory leakFileChannel.transferFrom()
's performance is unacceptable low, more specifically, about 200 messages are written per secondI wonder what do you mean about direct IO
, does it mean DirectByteBuffer
or something?
对于 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
[X] I found a bug, not just asking a question, which should be created in GitHub Discussions.
[X] I have searched the GitHub Issues and GitHub Discussions of this repository and believe that this is not a duplicate.
[X] 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;