-
Notifications
You must be signed in to change notification settings - Fork 967
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
Support WAL Compression #12476
base: master
Are you sure you want to change the base?
Support WAL Compression #12476
Conversation
834e61d
to
70d8400
Compare
89b0f35
to
94152b2
Compare
Codecov ReportAttention: Patch coverage is
Additional details and impacted files@@ Coverage Diff @@
## master #12476 +/- ##
============================================
- Coverage 43.06% 42.16% -0.91%
- Complexity 69 71 +2
============================================
Files 3202 3262 +60
Lines 200013 205915 +5902
Branches 23853 24556 +703
============================================
+ Hits 86139 86814 +675
- Misses 113874 119101 +5227 ☔ View full report in Codecov by Sentry. |
e7250f8
to
069795d
Compare
@@ -79,6 +132,10 @@ public long size() { | |||
return size; | |||
} | |||
|
|||
public long oiginalSize() { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
May be replaced with originalSize
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Fixed.
@@ -221,7 +222,7 @@ public class IoTDBConfig { | |||
private int walBufferQueueCapacity = 500; | |||
|
|||
/** Size threshold of each wal file. Unit: byte */ | |||
private volatile long walFileSizeThresholdInByte = 30 * 1024 * 1024L; | |||
private volatile long walFileSizeThresholdInByte = 30 * 1024L * 1024L; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Why change this here?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Recover.
@@ -763,6 +763,10 @@ data_replication_factor=1 | |||
# Datatype: long | |||
# iot_consensus_cache_window_time_in_ms=-1 | |||
|
|||
# WAL compression algorithm | |||
# options: UNCOMPRESSED, SNAPPY, LZ4, GZIP, ZSTD | |||
# wal_compress_algorithm=UNCOMPRESSED |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
May tune to decide the default algorithm.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Change the configuration item to enable or disable wal compression by true or false.
remove metrics in mem table flush task, cache hash code in partial path, use gzip to compress wal batch update metrics
eb167a8
to
93a6482
Compare
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
There are issues that should be solved. Also, please add new tests.
.../apache/iotdb/db/storageengine/dataregion/wal/allocation/AbstractNodeAllocationStrategy.java
Outdated
Show resolved
Hide resolved
...atanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/buffer/WALBuffer.java
Outdated
Show resolved
Hide resolved
...atanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/buffer/WALBuffer.java
Outdated
Show resolved
Hide resolved
|| (forceFlag && currentWALFileWriter.size() >= config.getWalFileSizeThresholdInByte())) { | ||
|| (forceFlag | ||
&& currentWALFileWriter.originalSize() >= config.getWalFileSizeThresholdInByte())) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Using the originalSize to control force may result in small actual IOs if the compression ratio is high.
I guess this is to control the risk of losing persistence.
It would be better to add a comment explaining why you should choose originalSize over actualSize here.
Ideally, I would expect some experiments to show using originalSize is better than actualSize, or the trade-off.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It's @OneSizeFitsQuorum 's suggestion.
The main reason to control the writer with originalSize is that when flushing a WALEntry into a buffer, an instance of WALFlushListener will be created. These objects are released when the file writer is closed. If the compression ratio is high, and we are using the actual size to control the writer, a lot of WALFlushListener instance will be kept in memory and cause an OOM error. This happened in the unit-test, so we fix it in this way.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
So, in the end, you want to control the number of WALFlushListeners (and ultimately their corresponding memory).
Then, why not just use the number of WALFlushListeners (or their corresponding memory) as one of the predicates?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The original WAL file open/close control used the size of the WAL file. This PR only made some simple modifications on top of that to accommodate compression. It is reasonable to control the opening and closing of WAL files using the number of WALFlushListeners, but this goes beyond the scope of this PR and would make the PR's integration cycle longer and the impact more unpredictable. Therefore, it is suggested that controlling the opening and closing of WAL files using WALFlushListeners be planned for future work.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Ok, then, please leave a TODO here.
...main/java/org/apache/iotdb/db/storageengine/dataregion/wal/checkpoint/CheckpointManager.java
Show resolved
Hide resolved
.../datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/io/WALMetaData.java
Outdated
Show resolved
Hide resolved
.../datanode/src/main/java/org/apache/iotdb/db/storageengine/dataregion/wal/io/WALMetaData.java
Show resolved
Hide resolved
try (WALInputStream is = openReadFileStream()) { | ||
is.skipToGivenPosition(position); | ||
ByteBuffer buffer = ByteBuffer.allocate(size); | ||
channel.position(position); | ||
channel.read(buffer); | ||
buffer.clear(); | ||
is.read(buffer); | ||
buffer.flip(); | ||
return buffer; | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I wonder about the efficiency of opening a stream for each WALEntry. Maybe the stream should be reused somehow.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
My response is the same as in the previous comments: this is how the original design was implemented. I just made slight modifications to make it suitable for compressed cases. Theoretically, the performance is similar to the original since the original design involved opening a Channel, and a WALInputStream essentially operates as a Channel. Sharing the Stream is a reasonable suggestion, but it is unrelated to the current task (Write Ahead Log compression) and may introduce other bugs and performance issues, so I will not be modifying this part of the code.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Just leave a TODO here, then.
if (isInSealedFile()) { | ||
walFile = walNode.getWALFile(walFileVersionId); | ||
return new WALInputStream(walFile); | ||
} else { | ||
try { | ||
walFile = walNode.getWALFile(walFileVersionId); | ||
return new WALInputStream(walFile); | ||
} catch (IOException e) { | ||
// unsealed file may be renamed after sealed, so we should try again | ||
if (isInSealedFile()) { | ||
walFile = walNode.getWALFile(walFileVersionId); | ||
return new WALInputStream(walFile); | ||
} else { | ||
throw e; | ||
} | ||
} | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It seems the "else" branch is just enough, and the branch above is not significantly different from it.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Same as above, this is the original design.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Just leave a TODO here, then.
...st/java/org/apache/iotdb/db/storageengine/dataregion/wal/compression/WALCompressionTest.java
Outdated
Show resolved
Hide resolved
posRemain -= currSegmentSize; | ||
channel.position( | ||
channel.position() | ||
+ currSegmentSize | ||
+ (type == CompressionType.UNCOMPRESSED ? 0 : Integer.BYTES)); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
You may skip with the channel by currSegmentSize + Integer.BYTES
if the WAL is compressed, but posRemain
is only reduced by currSegmentSize
.
Is this inconsistency problematic?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
No. posRemain is a logical size, not a physical size. It represents the offset of the WAL before compression, so you only need to subtract the originalSegmentSize.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This should be clarified by comments or better variable names; try it.
@@ -85,6 +86,9 @@ protected IWALNode createWALNode( | |||
} catch (FileNotFoundException e) { | |||
logger.error("Fail to create wal node", e); | |||
return WALFakeNode.getFailureInstance(e); | |||
} catch (IOException e) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Exception handling should be the same as above
@@ -1215,6 +1215,10 @@ data_replication_factor=1 | |||
# Datatype: long | |||
# iot_consensus_cache_window_time_in_ms=-1 | |||
|
|||
# Enable Write Ahead Log compression. | |||
# Option: true, false | |||
# enable_wal_compression=false |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
enable hot reload?
Quality Gate failedFailed conditions |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Good, please see to the remaining minor issues.
if (Objects.isNull(dataBuffer) | ||
|| dataBuffer.capacity() < segmentInfo.uncompressedSize | ||
|| dataBuffer.capacity() > segmentInfo.uncompressedSize * 2) { | ||
dataBuffer = ByteBuffer.allocateDirect(segmentInfo.uncompressedSize); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
If the previous buffer is not null, it should be manually released through MmapUtil.clean
—the same for the below.
// Do not compress it | ||
IoTDBDescriptor.getInstance().getConfig().setWALCompressionAlgorithm(CompressionType.LZ4); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Do not compress it?
The write-ahead logging consumes a significant amount of I/O resources during data writing, sometimes even 2-7 times the amount of TsFile I/O. Currently, the write-ahead logging in the system is uncompressed, this is a main reason for the large volume of WAL. Therefore, compressing the write-ahead logging can save I/O resources and enhance system performance. This PR primarily introduces support for compressing write-ahead logging, featuring the following characteristics:
The basic unit for compression and decompression is a WAL Buffer, which may contain several WAL Entries, with a default size of 16MB. This design achieves a good compression ratio while minimizing modifications to upper-layer applications.
To maintain compatibility with previous versions, we have replaced the original Magic String in the WAL file with a Version String to distinguish between different versions of WAL. If there are subsequent new versions of WAL, we can continue to add new Version Strings.
Multiple compression methods are supported. Currently, the choice of compression method is not open to users, but theoretically, our design supports various compression methods, and we may open this option in the future.
Changes in system resource consumption and system performance caused by this feature will be supplemented in subsequent comments.