-
Notifications
You must be signed in to change notification settings - Fork 14.9k
KAFKA-20036 Handle LogCleaner segment overflow caused by compression level changes #21379
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
Open
m1a2st
wants to merge
4
commits into
apache:trunk
Choose a base branch
from
m1a2st:KAFKA-20036
base: trunk
Could not load branches
Branch not found: {{ refName }}
Loading
Could not load tags
Nothing to show
Loading
Are you sure you want to change the base?
Some commits from the old base branch may be removed from the timeline,
and old review comments may become outdated.
Open
Changes from all commits
Commits
Show all changes
4 commits
Select commit
Hold shift + click to select a range
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
29 changes: 29 additions & 0 deletions
29
storage/src/main/java/org/apache/kafka/storage/internals/log/SegmentOverflowException.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,29 @@ | ||
| /* | ||
| * 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 SegmentOverflowException(LogSegment segment) { | ||
| super("Segment size would overflow during compaction for segment " + segment); | ||
| } | ||
| } |
93 changes: 93 additions & 0 deletions
93
storage/src/test/java/org/apache/kafka/storage/internals/log/CleanerIntegrationTest.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,93 @@ | ||
| /* | ||
| * 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.clients.admin.NewTopic; | ||
| import org.apache.kafka.clients.producer.ProducerConfig; | ||
| import org.apache.kafka.clients.producer.ProducerRecord; | ||
| import org.apache.kafka.common.Node; | ||
| import org.apache.kafka.common.Uuid; | ||
| import org.apache.kafka.common.test.ClusterInstance; | ||
| import org.apache.kafka.common.test.api.ClusterTest; | ||
| import org.apache.kafka.common.test.api.Type; | ||
| import org.apache.kafka.server.metrics.KafkaYammerMetrics; | ||
|
|
||
| import com.yammer.metrics.core.Gauge; | ||
|
|
||
| import java.util.List; | ||
| import java.util.Map; | ||
| import java.util.Random; | ||
| import java.util.concurrent.CompletableFuture; | ||
| import java.util.stream.IntStream; | ||
|
|
||
| import static org.junit.jupiter.api.Assertions.assertEquals; | ||
| import static org.junit.jupiter.api.Assertions.assertTrue; | ||
|
|
||
| public class CleanerIntegrationTest { | ||
|
|
||
| @SuppressWarnings("unchecked") | ||
| @ClusterTest(types = Type.CO_KRAFT) | ||
| public void testCleanerSegmentCompactionOverflow(ClusterInstance cluster) throws Exception { | ||
| String topic = "compaction-overflow-test"; | ||
| try (var admin = cluster.admin()) { | ||
| NewTopic newTopic = new NewTopic(topic, 1, (short) 1); | ||
| newTopic.configs(Map.of( | ||
| "cleanup.policy", "compact", | ||
| "compression.type", "lz4", | ||
| "segment.bytes", String.valueOf(Integer.MAX_VALUE - 1), | ||
| "min.cleanable.dirty.ratio", "0.01" | ||
| )); | ||
| admin.createTopics(List.of(newTopic)).all().get(); | ||
| cluster.waitTopicCreation(topic, 1); | ||
|
|
||
| var data = new byte[10240]; | ||
| var random = new Random(); | ||
| random.nextBytes(data); | ||
| var producers = IntStream.range(0, 5).mapToObj(__ -> CompletableFuture.runAsync(() -> { | ||
| try (var producer = cluster.producer(Map.of( | ||
| ProducerConfig.COMPRESSION_LZ4_LEVEL_CONFIG, "17", | ||
| ProducerConfig.COMPRESSION_TYPE_CONFIG, "lz4"))) { | ||
| for (int i = 0; i < 60_000; i++) { | ||
| byte[] key = Uuid.randomUuid().toString().getBytes(); | ||
| if (Math.random() < 0.1) | ||
| producer.send(new ProducerRecord<>(topic, key, null)); | ||
| else | ||
| producer.send(new ProducerRecord<>(topic, key, data)); | ||
| } | ||
| } | ||
| })).toList(); | ||
| producers.forEach(CompletableFuture::join); | ||
|
|
||
| var ids = admin.describeCluster().nodes().get().stream().map(Node::id).toList(); | ||
| var size = admin.describeLogDirs(ids).allDescriptions().get().entrySet() | ||
| .stream() | ||
| .flatMap(e -> e.getValue().values() | ||
| .stream() | ||
| .flatMap(v -> v.replicaInfos().entrySet().stream())) | ||
| .filter(v -> v.getKey().topic().equals(topic)) | ||
| .mapToLong(v -> v.getValue().size()).sum(); | ||
| assertTrue(Integer.MAX_VALUE < size, "log size should exceed Integer.MAX_VALUE to trigger overflow"); | ||
| } | ||
| var metrics = KafkaYammerMetrics.defaultRegistry().allMetrics(); | ||
| metrics.forEach((name, metric) -> { | ||
| if (name.getName().contains("uncleanable-partitions-count")) { | ||
| Gauge<Integer> value = (Gauge<Integer>) metric; | ||
| assertEquals(0, value.value(), "there should be no uncleanable partitions due to segment overflow"); | ||
| } | ||
| }); | ||
| } | ||
| } |
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
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.
Could you wrap only
dest.appendin the try-catch block to avoid catching unrelated error?