showuon commented on code in PR #15136: URL: https://github.com/apache/kafka/pull/15136#discussion_r1544295936
########## core/src/main/scala/kafka/log/LogManager.scala: ########## @@ -1173,6 +1173,35 @@ class LogManager(logDirs: Seq[File], } } + def recoverAbandonedFutureLogs(brokerId: Int, newTopicsImage: TopicsImage): Unit = { + val abandonedFutureLogs = findAbandonedFutureLogs(brokerId, newTopicsImage) + abandonedFutureLogs.foreach { log => + val tp = log.topicPartition + + log.renameDir(UnifiedLog.logDirName(tp), shouldReinitialize = true) + log.removeLogMetrics() + futureLogs.remove(tp) + + currentLogs.put(tp, log) + log.newMetrics() + + info(s"Successfully renamed abandoned future log for $tp") + } + } + + private def findAbandonedFutureLogs(brokerId: Int, newTopicsImage: TopicsImage): Iterable[UnifiedLog] = { + futureLogs.values.flatMap { log => + 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.") + } + val partitionId = log.topicPartition.partition() + Option(newTopicsImage.getPartition(topicId, partitionId)) + .filter(pr => directoryId(log.parentDir).contains(pr.directory(brokerId))) + .map(_ => log) Review Comment: For (1), I think we'll catch up until the high watermark before publish the image. So it should be safe. For (2), good suggestion. ########## core/src/main/scala/kafka/server/ReplicaManager.scala: ########## @@ -2745,10 +2745,10 @@ class ReplicaManager(val config: KafkaConfig, "local leaders.") replicaFetcherManager.removeFetcherForPartitions(localLeaders.keySet) localLeaders.forKeyValue { (tp, info) => + val partitionAssignedDirectoryId = directoryIds.find(_._1.topicPartition() == tp).map(_._2) getOrCreatePartition(tp, delta, info.topicId).foreach { case (partition, isNew) => try { val state = info.partition.toLeaderAndIsrPartitionState(tp, isNew) Review Comment: Same here, unnecessary change. -- 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