forked from apache/kafka
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
KAFKA-16814 KRaft broker cannot startup when
partition.metadata
is …
…missing (apache#16165) When starting up kafka logManager, we'll check stray replicas to avoid some corner cases. But this check might cause broker unable to startup if partition.metadata is missing because when startup kafka, we load log from file, and the topicId of the log is coming from partition.metadata file. So, if partition.metadata is missing, the topicId will be None, and the LogManager#isStrayKraftReplica will fail with no topicID error. The partition.metadata missing could be some storage failure, or another possible path is unclean shutdown after topic is created in the replica, but before data is flushed into partition.metadata file. This is possible because we do the flush in async way here. When finding a log without topicID, we should treat it as a stray log and then delete it. Reviewers: Luke Chen <[email protected]>, Gaurav Narula <[email protected]>
- Loading branch information
1 parent
54a7d88
commit 0d25def
Showing
3 changed files
with
152 additions
and
3 deletions.
There are no files selected for viewing
This file contains 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
137 changes: 137 additions & 0 deletions
137
core/src/test/java/kafka/server/LogManagerIntegrationTest.java
This file contains 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,137 @@ | ||
/* | ||
* 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 kafka.server; | ||
|
||
import kafka.test.ClusterInstance; | ||
import kafka.test.annotation.ClusterTest; | ||
import kafka.test.annotation.Type; | ||
import kafka.test.junit.ClusterTestExtensions; | ||
import kafka.test.junit.RaftClusterInvocationContext; | ||
import org.apache.kafka.clients.admin.Admin; | ||
import org.apache.kafka.clients.admin.NewTopic; | ||
import org.apache.kafka.clients.consumer.Consumer; | ||
import org.apache.kafka.clients.consumer.ConsumerConfig; | ||
import org.apache.kafka.clients.consumer.ConsumerRecord; | ||
import org.apache.kafka.clients.consumer.ConsumerRecords; | ||
import org.apache.kafka.clients.consumer.KafkaConsumer; | ||
import org.apache.kafka.clients.producer.KafkaProducer; | ||
import org.apache.kafka.clients.producer.Producer; | ||
import org.apache.kafka.clients.producer.ProducerConfig; | ||
import org.apache.kafka.clients.producer.ProducerRecord; | ||
import org.apache.kafka.common.TopicPartition; | ||
import org.apache.kafka.common.TopicPartitionInfo; | ||
import org.apache.kafka.common.serialization.StringDeserializer; | ||
import org.apache.kafka.common.serialization.StringSerializer; | ||
import org.apache.kafka.storage.internals.checkpoint.PartitionMetadataFile; | ||
import org.apache.kafka.test.TestUtils; | ||
import org.junit.jupiter.api.Tag; | ||
import org.junit.jupiter.api.extension.ExtendWith; | ||
|
||
import java.io.IOException; | ||
import java.time.Duration; | ||
import java.util.ArrayList; | ||
import java.util.Collections; | ||
import java.util.HashMap; | ||
import java.util.List; | ||
import java.util.Map; | ||
import java.util.Optional; | ||
import java.util.UUID; | ||
import java.util.concurrent.ExecutionException; | ||
|
||
import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; | ||
import static org.junit.jupiter.api.Assertions.assertEquals; | ||
import static org.junit.jupiter.api.Assertions.assertFalse; | ||
import static org.junit.jupiter.api.Assertions.assertTrue; | ||
|
||
@ExtendWith(value = ClusterTestExtensions.class) | ||
@Tag("integration") | ||
public class LogManagerIntegrationTest { | ||
private final ClusterInstance cluster; | ||
|
||
public LogManagerIntegrationTest(ClusterInstance cluster) { | ||
this.cluster = cluster; | ||
} | ||
|
||
@ClusterTest(types = {Type.KRAFT, Type.CO_KRAFT}, brokers = 3) | ||
public void testRestartBrokerNoErrorIfMissingPartitionMetadata() throws IOException, ExecutionException, InterruptedException { | ||
RaftClusterInvocationContext.RaftClusterInstance raftInstance = | ||
(RaftClusterInvocationContext.RaftClusterInstance) cluster; | ||
|
||
try (Admin admin = cluster.createAdminClient()) { | ||
admin.createTopics(Collections.singletonList(new NewTopic("foo", 1, (short) 3))).all().get(); | ||
} | ||
cluster.waitForTopic("foo", 1); | ||
|
||
Optional<PartitionMetadataFile> partitionMetadataFile = Optional.ofNullable( | ||
raftInstance.getUnderlying().brokers().get(0).logManager() | ||
.getLog(new TopicPartition("foo", 0), false).get() | ||
.partitionMetadataFile().getOrElse(null)); | ||
assertTrue(partitionMetadataFile.isPresent()); | ||
|
||
raftInstance.getUnderlying().brokers().get(0).shutdown(); | ||
try (Admin admin = cluster.createAdminClient()) { | ||
TestUtils.waitForCondition(() -> { | ||
List<TopicPartitionInfo> partitionInfos = admin.describeTopics(Collections.singletonList("foo")) | ||
.topicNameValues().get("foo").get().partitions(); | ||
return partitionInfos.get(0).isr().size() == 2; | ||
}, "isr size is not shrink to 2"); | ||
} | ||
|
||
// delete partition.metadata file here to simulate the scenario that partition.metadata not flush to disk yet | ||
partitionMetadataFile.get().delete(); | ||
assertFalse(partitionMetadataFile.get().exists()); | ||
raftInstance.getUnderlying().brokers().get(0).startup(); | ||
// make sure there is no error during load logs | ||
assertDoesNotThrow(() -> raftInstance.getUnderlying().fatalFaultHandler().maybeRethrowFirstException()); | ||
try (Admin admin = cluster.createAdminClient()) { | ||
TestUtils.waitForCondition(() -> { | ||
List<TopicPartitionInfo> partitionInfos = admin.describeTopics(Collections.singletonList("foo")) | ||
.topicNameValues().get("foo").get().partitions(); | ||
return partitionInfos.get(0).isr().size() == 3; | ||
}, "isr size is not expand to 3"); | ||
} | ||
|
||
// make sure topic still work fine | ||
Map<String, Object> producerConfigs = new HashMap<>(); | ||
producerConfigs.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, cluster.bootstrapServers()); | ||
producerConfigs.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName()); | ||
producerConfigs.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName()); | ||
|
||
try (Producer<String, String> producer = new KafkaProducer<>(producerConfigs)) { | ||
producer.send(new ProducerRecord<>("foo", 0, null, "bar")).get(); | ||
producer.flush(); | ||
} | ||
|
||
Map<String, Object> consumerConfigs = new HashMap<>(); | ||
consumerConfigs.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, cluster.bootstrapServers()); | ||
consumerConfigs.put(ConsumerConfig.GROUP_ID_CONFIG, UUID.randomUUID().toString()); | ||
consumerConfigs.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName()); | ||
consumerConfigs.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName()); | ||
|
||
try (Consumer<String, String> consumer = new KafkaConsumer<>(consumerConfigs)) { | ||
consumer.assign(Collections.singletonList(new TopicPartition("foo", 0))); | ||
consumer.seekToBeginning(Collections.singletonList(new TopicPartition("foo", 0))); | ||
List<String> values = new ArrayList<>(); | ||
ConsumerRecords<String, String> records = consumer.poll(Duration.ofMinutes(1)); | ||
for (ConsumerRecord<String, String> record : records) { | ||
values.add(record.value()); | ||
} | ||
assertEquals(1, values.size()); | ||
assertEquals("bar", values.get(0)); | ||
} | ||
} | ||
} |
This file contains 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