gaurav-narula commented on code in PR #15136:
URL: https://github.com/apache/kafka/pull/15136#discussion_r1569131419


##########
core/src/main/scala/kafka/log/LogManager.scala:
##########
@@ -1189,50 +1216,61 @@ class LogManager(logDirs: Seq[File],
       val sourceLog = currentLogs.get(topicPartition)
       val destLog = futureLogs.get(topicPartition)
 
-      info(s"Attempting to replace current log $sourceLog with $destLog for 
$topicPartition")
       if (sourceLog == null)
         throw new KafkaStorageException(s"The current replica for 
$topicPartition is offline")
       if (destLog == null)
         throw new KafkaStorageException(s"The future replica for 
$topicPartition is offline")
 
-      destLog.renameDir(UnifiedLog.logDirName(topicPartition), 
shouldReinitialize = true)
-      // the metrics tags still contain "future", so we have to remove it.
-      // we will add metrics back after sourceLog remove the metrics
-      destLog.removeLogMetrics()
-      destLog.updateHighWatermark(sourceLog.highWatermark)
+      replaceCurrentWithFutureLog(Option(sourceLog), destLog, 
updateHighWatermark = true)
+    }
+  }
+
+  def replaceCurrentWithFutureLog(sourceLog: Option[UnifiedLog], destLog: 
UnifiedLog, updateHighWatermark: Boolean = false): Unit = {
+    val topicPartition = destLog.topicPartition
+    info(s"Attempting to replace current log $sourceLog with $destLog for 
$topicPartition")
 
-      // Now that future replica has been successfully renamed to be the 
current replica
-      // Update the cached map and log cleaner as appropriate.
-      futureLogs.remove(topicPartition)
-      currentLogs.put(topicPartition, destLog)
-      if (cleaner != null) {
-        cleaner.alterCheckpointDir(topicPartition, sourceLog.parentDirFile, 
destLog.parentDirFile)
-        resumeCleaning(topicPartition)
-      }
+    destLog.renameDir(UnifiedLog.logDirName(topicPartition), 
shouldReinitialize = true)
+    // the metrics tags still contain "future", so we have to remove it.
+    // we will add metrics back after sourceLog remove the metrics
+    destLog.removeLogMetrics()
+    if (updateHighWatermark && sourceLog.isDefined) {
+      destLog.updateHighWatermark(sourceLog.get.highWatermark)
+    }
 
-      try {
-        sourceLog.renameDir(UnifiedLog.logDeleteDirName(topicPartition), 
shouldReinitialize = true)
+    // Now that future replica has been successfully renamed to be the current 
replica
+    // Update the cached map and log cleaner as appropriate.
+    futureLogs.remove(topicPartition)
+    currentLogs.put(topicPartition, destLog)
+    if (cleaner != null) {
+      cleaner.alterCheckpointDir(topicPartition, 
sourceLog.map(_.parentDirFile), destLog.parentDirFile)

Review Comment:
   Addressed in 
[062e932](https://github.com/apache/kafka/pull/15136/commits/062e932f260ce9e1df9571b2fc982c63cbaf0f7c)



-- 
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

Reply via email to