chia7712 commented on code in PR #21379: URL: https://github.com/apache/kafka/pull/21379#discussion_r2749907193
########## storage/src/main/java/org/apache/kafka/storage/internals/log/SegmentOverflowException.java: ########## @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.storage.internals.log; + +import org.apache.kafka.common.KafkaException; + +/** + * Exception thrown when segment size would overflow during compaction + */ +public class SegmentOverflowException extends KafkaException { + public final LogSegment segment; + + public SegmentOverflowException(LogSegment segment) { + super("Segment size would overflow during compaction for segment " + segment); + this.segment = segment; Review Comment: why we need it? ########## storage/src/main/java/org/apache/kafka/storage/internals/log/Cleaner.java: ########## @@ -400,10 +436,15 @@ public boolean shouldRetainRecord(RecordBatch batch, Record record) { if (outputBuffer.position() > 0) { outputBuffer.flip(); MemoryRecords retained = MemoryRecords.readableRecords(outputBuffer); - // it's OK not to hold the Log's lock in this case, because this segment is only accessed by other threads - // after `Log.replaceSegments` (which acquires the lock) is called - dest.append(result.maxOffset(), retained); - throttler.maybeThrottle(outputBuffer.limit()); + try { + // it's OK not to hold the Log's lock in this case, because this segment is only accessed by other threads + // after `Log.replaceSegments` (which acquires the lock) is called + dest.append(result.maxOffset(), retained); Review Comment: Could you wrap only `dest.append` in the try-catch block to avoid catching unrelated error? ########## storage/src/main/java/org/apache/kafka/storage/internals/log/Cleaner.java: ########## @@ -180,6 +195,13 @@ public Map.Entry<Long, CleanerStats> doClean(LogToClean cleanable, long currentT cleanSegments(log, group, offsetMap, currentTime, stats, transactionMetadata, legacyDeleteHorizonMs, upperBoundOffset); } + if (segmentOverflowPartitions.containsKey(log.topicPartition())) { Review Comment: ```java if (segmentOverflowPartitions.remove(log.topicPartition()) != null) { logger.info("Successfully cleaned log {} with degraded size (ratio: {}%). " + "Cleared overflow marker. Next cleaning will use normal size.", log.name(), sizeRatio * 100); } ``` ########## storage/src/main/java/org/apache/kafka/storage/internals/log/Cleaner.java: ########## @@ -254,6 +276,20 @@ public void cleanSegments(UnifiedLog log, stats, currentTime ); + } catch (SegmentOverflowException e) { + if (segmentOverflowPartitions.containsKey(log.topicPartition())) { Review Comment: ```java var previousRatio = segmentOverflowPartitions.put(log.topicPartition(), segmentOverflowPartitions.getOrDefault(log.topicPartition(), 1.0) * 0.9); if (previousRatio == null) { logger.warn("Segment overflow detected for partition {}: {}. " + "Marked for degradation to 90% size in next cleaning round.", log.topicPartition(), e.getMessage()); } else { logger.warn("Repeated segment overflow for partition {}: {}. " + "Further degrading to {}% size in next cleaning round.", log.topicPartition(), e.getMessage(), previousRatio * 0.9 * 100); } ``` ########## storage/src/main/java/org/apache/kafka/storage/internals/log/Cleaner.java: ########## @@ -169,9 +175,18 @@ public Map.Entry<Long, CleanerStats> doClean(LogToClean cleanable, long currentT log.name(), new Date(cleanableHorizonMs), new Date(legacyDeleteHorizonMs)); CleanedTransactionMetadata transactionMetadata = new CleanedTransactionMetadata(); + double sizeRatio = 1.0; Review Comment: Would something like this work better? ```java double sizeRatio = segmentOverflowPartitions.getOrDefault(log.topicPartition(), 1.0); if (sizeRatio != 1.0) { logger.info("Partition {} has overflow history. " + "Reducing effective segment size to {}% for this round.", log.topicPartition(), sizeRatio * 100); } ``` -- 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]
