showuon commented on code in PR #16165: URL: https://github.com/apache/kafka/pull/16165#discussion_r1625840199
########## core/src/main/scala/kafka/log/LogManager.scala: ########## @@ -1604,11 +1604,16 @@ object LogManager { newTopicsImage: TopicsImage, log: UnifiedLog ): Boolean = { - val topicId = log.topicId.getOrElse { - throw new RuntimeException(s"The log dir $log does not have a topic ID, " + - "which is not allowed when running in KRaft mode.") + if (log.topicId.isEmpty) { + // Missing topic ID could result from storage failure or unclean shutdown after topic creation but before flushing + // data to the `partition.metadata` file. And before appending data to the log, the `partition.metadata` is always + // 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 stray log dir") Review Comment: 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 stray log ~~dir~~") ########## core/src/test/java/kafka/server/LogManagerIntegrationTest.java: ########## @@ -0,0 +1,135 @@ +/* + * 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.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(); + raftInstance.getUnderlying().brokers().get(0).startup(); Review Comment: We can verify `partitionMetadataFile.get().exists()` is false after deletion. -- 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: jira-unsubscr...@kafka.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org