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

Support WAL Compression #12476

Open
wants to merge 29 commits into
base: master
Choose a base branch
from
Open

Support WAL Compression #12476

wants to merge 29 commits into from

Conversation

THUMarkLau
Copy link
Contributor

@THUMarkLau THUMarkLau commented May 7, 2024

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:

  1. 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.

  2. 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.

  3. 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.

@THUMarkLau THUMarkLau force-pushed the wal-compress-formal-branch branch 2 times, most recently from 89b0f35 to 94152b2 Compare May 24, 2024 06:04
Copy link

codecov bot commented May 25, 2024

Codecov Report

Attention: Patch coverage is 83.50515% with 48 lines in your changes missing coverage. Please review.

Project coverage is 42.16%. Comparing base (a4a7be0) to head (b99c977).
Report is 64 commits behind head on master.

Current head b99c977 differs from pull request most recent head 7201ed6

Please upload reports for the commit 7201ed6 to get more accurate results.

Files Patch % Lines
...torageengine/dataregion/wal/io/WALInputStream.java 81.25% 33 Missing ⚠️
...eengine/dataregion/wal/utils/WALEntryPosition.java 50.00% 7 Missing ⚠️
...java/org/apache/iotdb/db/conf/IoTDBDescriptor.java 33.33% 4 Missing ⚠️
...wal/allocation/AbstractNodeAllocationStrategy.java 0.00% 3 Missing ⚠️
...b/storageengine/dataregion/wal/io/WALMetaData.java 88.88% 1 Missing ⚠️
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.
📢 Have feedback on the report? Share it here.

@@ -79,6 +132,10 @@ public long size() {
return size;
}

public long oiginalSize() {
Copy link
Contributor

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

Copy link
Contributor Author

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;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why change this here?

Copy link
Contributor Author

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
Copy link
Contributor

@Caideyipi Caideyipi May 28, 2024

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.

Copy link
Contributor Author

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.

Copy link
Contributor

@jt2594838 jt2594838 left a 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.

Comment on lines -543 to +544
|| (forceFlag && currentWALFileWriter.size() >= config.getWalFileSizeThresholdInByte())) {
|| (forceFlag
&& currentWALFileWriter.originalSize() >= config.getWalFileSizeThresholdInByte())) {
Copy link
Contributor

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.

Copy link
Contributor Author

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.

Copy link
Contributor

@jt2594838 jt2594838 Jun 4, 2024

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?

Copy link
Contributor Author

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.

Copy link
Contributor

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.

Comment on lines +104 to 110
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;
}
Copy link
Contributor

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.

Copy link
Contributor Author

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.

Copy link
Contributor

@jt2594838 jt2594838 Jun 12, 2024

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.

Comment on lines +140 to +156
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;
}
}
}
Copy link
Contributor

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.

Copy link
Contributor Author

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.

Copy link
Contributor

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.

Comment on lines 256 to 260
posRemain -= currSegmentSize;
channel.position(
channel.position()
+ currSegmentSize
+ (type == CompressionType.UNCOMPRESSED ? 0 : Integer.BYTES));
Copy link
Contributor

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?

Copy link
Contributor Author

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.

Copy link
Contributor

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) {
Copy link
Contributor

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
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

enable hot reload?

Copy link

sonarcloud bot commented Jun 12, 2024

Quality Gate Failed Quality Gate failed

Failed conditions
0.0% Coverage on New Code (required ≥ 80%)

See analysis details on SonarCloud

Copy link
Contributor

@jt2594838 jt2594838 left a 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);
Copy link
Contributor

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.

Comment on lines +234 to +235
// Do not compress it
IoTDBDescriptor.getInstance().getConfig().setWALCompressionAlgorithm(CompressionType.LZ4);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do not compress it?

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 this pull request may close these issues.

None yet

4 participants