chia7712 commented on code in PR #15516:
URL: https://github.com/apache/kafka/pull/15516#discussion_r2649600413
##########
clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java:
##########
@@ -293,14 +294,15 @@ private static MemoryRecordsBuilder
buildRetainedRecordsInto(RecordBatch origina
ByteBufferOutputStream bufferOutputStream,
final long
deleteHorizonMs) {
byte magic = originalBatch.magic();
+ Compression compression =
Compression.of(originalBatch.compressionType()).build();
Review Comment:
> If we use a different compression type, the compacted data could be
exceeding the max segment limit and failing the index append.
It appears that using a high compression level during ingestion can indeed
trigger overflow issue during compaction. This is because the cleaner's size
estimation becomes inaccurate when it rebuilds batches using the default
compression level instead of the origin one
```
[2025-12-28 08:25:42,915] WARN [kafka-log-cleaner-thread-0]: Unexpected
exception thrown when cleaning log Log(dir=/tmp/log-folder-0/chia-0,
topicId=5vKt56jQQ_S3QRuqrNmrTw, topic=chia, partition=0,
highWatermark=66028296, lastStableOffset=66028296, logStartOffset=0,
logEndOffset=66028296). Marking its partition (chia-0) as uncleanable
(org.apache.kafka.storage.internals.log.LogCleaner$CleanerThread)
org.apache.kafka.storage.internals.log.LogCleaningException: Append of size
258080 bytes is too large for segment with current file position at 2147262463
at
org.apache.kafka.storage.internals.log.LogCleaner$CleanerThread.cleanFilthiestLog(LogCleaner.java:570)
at
org.apache.kafka.storage.internals.log.LogCleaner$CleanerThread.tryCleanFilthiestLog(LogCleaner.java:544)
at
org.apache.kafka.storage.internals.log.LogCleaner$CleanerThread.doWork(LogCleaner.java:513)
at
org.apache.kafka.server.util.ShutdownableThread.run(ShutdownableThread.java:136)
Caused by: java.lang.IllegalArgumentException: Append of size 258080 bytes
is too large for segment with current file position at 2147262463
at
org.apache.kafka.common.record.FileRecords.append(FileRecords.java:196)
at
org.apache.kafka.storage.internals.log.LogSegment.append(LogSegment.java:260)
at
org.apache.kafka.storage.internals.log.Cleaner.cleanInto(Cleaner.java:405)
at
org.apache.kafka.storage.internals.log.Cleaner.cleanSegments(Cleaner.java:243)
at
org.apache.kafka.storage.internals.log.Cleaner.doClean(Cleaner.java:180)
at org.apache.kafka.storage.internals.log.Cleaner.clean(Cleaner.java:127)
at
org.apache.kafka.storage.internals.log.LogCleaner$CleanerThread.cleanLog(LogCleaner.java:596)
at
org.apache.kafka.storage.internals.log.LogCleaner$CleanerThread.cleanFilthiestLog(LogCleaner.java:565)
... 3 more
```
To mitigate this, I propose introducing a safety margin specifically for
partitions that fail to compact due to overflows.
@mimaison @junrao @Yunyung WDYT?
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]