showuon commented on code in PR #12347:
URL: https://github.com/apache/kafka/pull/12347#discussion_r925591743


##########
core/src/main/scala/kafka/log/LogManager.scala:
##########
@@ -307,6 +309,27 @@ class LogManager(logDirs: Seq[File],
     log
   }
 
+  // factory class for naming the log recovery threads used in metrics
+  class LogRecoveryThreadFactory(val dirPath: String) extends ThreadFactory {
+    val threadNum = new AtomicInteger(0)
+
+    override def newThread(runnable: Runnable): Thread = {
+      KafkaThread.nonDaemon(logRecoveryThreadName(dirPath, 
threadNum.getAndIncrement()), runnable)
+    }
+  }
+
+  // create a unique log recovery thread name for each log dir as the format: 
prefix-dirPath-threadNum, ex: "log-recovery-/tmp/kafkaLogs-0"
+  private def logRecoveryThreadName(dirPath: String, threadNum: Int, prefix: 
String = "log-recovery"): String = s"$prefix-$dirPath-$threadNum"
+
+  /*
+   * decrement the number of remaining logs
+   * @return the number of remaining logs after decremented 1
+   */
+  private[log] def decNumRemainingLogs(numRemainingLogs: ConcurrentMap[String, 
AtomicInteger], path: String): Int = {
+    require(path != null, "path cannot be null to update remaining logs 
metric.")
+    numRemainingLogs.get(path).decrementAndGet()

Review Comment:
   Good point, Tom! Replaced AtomicInteger with Int, and use compute instead.
   ```
   numRemainingLogs.compute(path, (_, oldVal) => oldVal - 1)
   ```
   I confirmed from the javadoc[1]
   > The entire method invocation is performed atomically.
   
   So it is still thread safe. Thank you for the suggestion!
   
   [1] 
https://docs.oracle.com/javase/8/docs/api/java/util/concurrent/ConcurrentHashMap.html#compute-K-java.util.function.BiFunction-



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