-
Notifications
You must be signed in to change notification settings - Fork 14.1k
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
KAFKA-16814 KRaft broker cannot startup when partition.metadata
is missing
#16165
Changes from 1 commit
e64fdb2
fbaa648
8a540d5
220325c
daf19de
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,107 @@ | ||
/* | ||
* 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.serialization.StringDeserializer; | ||
import org.apache.kafka.common.serialization.StringSerializer; | ||
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.UUID; | ||
import java.util.concurrent.ExecutionException; | ||
|
||
import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; | ||
import static org.junit.jupiter.api.Assertions.assertEquals; | ||
|
||
@ExtendWith(value = ClusterTestExtensions.class) | ||
@Tag("integration") | ||
public class LogManagerIntegrationTest { | ||
private final ClusterInstance cluster; | ||
|
||
public LogManagerIntegrationTest(ClusterInstance cluster) { | ||
this.cluster = cluster; | ||
} | ||
|
||
@ClusterTest(types = {Type.KRAFT}) | ||
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) 1))); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Maybe wait for the topic to be created and partitionMetadataFile to exist at least in broker0 after this before shutting the broker down to avoid a race There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Thanks for the review 😃 ! I've address all comments in the latest commit, could you please take another look when you are available ? |
||
} | ||
|
||
raftInstance.getUnderlying().brokers().get(0).shutdown(); | ||
// delete partition.metadata file here to simulate the scenario that partition.metadata not flush to disk yet | ||
raftInstance.getUnderlying().brokers().get(0) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Maybe ensure ISR for the topic partition has shrunk before removing the file? |
||
.logManager().getLog(new TopicPartition("foo", 0), false).get().partitionMetadataFile().get().delete(); | ||
raftInstance.getUnderlying().brokers().get(0).startup(); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. We can verify |
||
assertDoesNotThrow(() -> raftInstance.getUnderlying().fatalFaultHandler().maybeRethrowFirstException()); | ||
|
||
// make sure topic still work fine | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Let's also wait until ISR is expanded again before producing/consuming. |
||
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)); | ||
} | ||
} | ||
} |
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.
nit:
// flushed to disk. So if the topic ID is missing, it mostly means no data was appended, and we can treat this as
// a stray log.
directory.info(s"The topicId does not exist in $log, treat it as [a] stray log
dir")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.
Thanks for the comments 😃 , already fixed all of them in the latest commit.