You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by GitBox <gi...@apache.org> on 2021/05/20 21:55:22 UTC

[GitHub] [kafka] kowshik opened a new pull request #10742: MINOR: Add log identifier/prefix printing in Log layer static functions

kowshik opened a new pull request #10742:
URL: https://github.com/apache/kafka/pull/10742


   When https://github.com/apache/kafka/pull/10478 was merged, we accidentally lost the identifier/prefix that we used to previously log from some of the functions in the `Log` class using the `Logging` APIs. In this PR, I have reinstated the identifier/prefix logging in these functions, so that the debuggability is restored.
   
   **Tests:**
   Ran existing unit tests and checked the output. Noticed that the log identifier/prefix shows up from the lines wherever it is additionally logged from now.


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kowshik commented on a change in pull request #10742: MINOR: Add log identifier/prefix printing in Log layer static functions

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10742:
URL: https://github.com/apache/kafka/pull/10742#discussion_r636623193



##########
File path: core/src/main/scala/kafka/log/LogLoader.scala
##########
@@ -172,7 +173,7 @@ object LogLoader extends Logging {
   private def removeTempFilesAndCollectSwapFiles(params: LoadLogParams): Set[File] = {
 
     def deleteIndicesIfExist(baseFile: File, suffix: String = ""): Unit = {
-      info(s"${params.logIdentifier} Deleting index files with suffix $suffix for baseFile $baseFile")
+      info(s"${params.logIdentifier}Deleting index files with suffix $suffix for baseFile $baseFile")

Review comment:
       Yes, because the existing identifier in the `Log` class is suffixed with a whitespace, see: https://github.com/apache/kafka/blob/5d4f9f917c8dc356a2d922980ba8101e0f2e7093/core/src/main/scala/kafka/log/Log.scala#L280
   I'm adding a whitespace here to be consistent with what we already have in the code.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] junrao commented on a change in pull request #10742: MINOR: Add log identifier/prefix printing in Log layer static functions

Posted by GitBox <gi...@apache.org>.
junrao commented on a change in pull request #10742:
URL: https://github.com/apache/kafka/pull/10742#discussion_r637236996



##########
File path: core/src/main/scala/kafka/log/Log.scala
##########
@@ -2005,7 +2005,7 @@ object Log extends Logging {
     Files.createDirectories(dir.toPath)
     val topicPartition = Log.parseTopicPartitionName(dir)
     val segments = new LogSegments(topicPartition)
-    val leaderEpochCache = Log.maybeCreateLeaderEpochCache(dir, topicPartition, logDirFailureChannel, config.messageFormatVersion.recordVersion)
+    val leaderEpochCache = Log.maybeCreateLeaderEpochCache(dir, topicPartition, logDirFailureChannel, config.messageFormatVersion.recordVersion, this.logIdent)

Review comment:
       Hmm, it seems that `this.logIdent` is always empty?

##########
File path: core/src/main/scala/kafka/log/Log.scala
##########
@@ -2332,7 +2336,8 @@ object Log extends Logging {
           config,
           scheduler,
           logDirFailureChannel,
-          producerStateManager)
+          producerStateManager,
+          logPrefix)

Review comment:
       Could we add the new param to javadoc?




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kowshik commented on a change in pull request #10742: MINOR: Add log identifier/prefix printing in Log layer static functions

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10742:
URL: https://github.com/apache/kafka/pull/10742#discussion_r636622730



##########
File path: core/src/main/scala/kafka/log/Log.scala
##########
@@ -2225,13 +2225,15 @@ object Log extends Logging {
    * @param dir The directory in which the log will reside
    * @param topicPartition The topic partition
    * @param logDirFailureChannel The LogDirFailureChannel to asynchronously handle log dir failure
-   * @param recordVersion The record version
+   * @param recordVersion The record

Review comment:
       No. It was changed accidentally. I have fixed it now in 35e389b5c05fddf5682f344ddf58222603133a19.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] showuon commented on a change in pull request #10742: MINOR: Add log identifier/prefix printing in Log layer static functions

Posted by GitBox <gi...@apache.org>.
showuon commented on a change in pull request #10742:
URL: https://github.com/apache/kafka/pull/10742#discussion_r637340043



##########
File path: core/src/main/scala/kafka/log/Log.scala
##########
@@ -2226,12 +2231,14 @@ object Log extends Logging {
    * @param topicPartition The topic partition
    * @param logDirFailureChannel The LogDirFailureChannel to asynchronously handle log dir failure
    * @param recordVersion The record version
+   * @param logPrefix The logging prefix
    * @return The new LeaderEpochFileCache instance (if created), none otherwise
    */
   def maybeCreateLeaderEpochCache(dir: File,
                                   topicPartition: TopicPartition,
                                   logDirFailureChannel: LogDirFailureChannel,
-                                  recordVersion: RecordVersion): Option[LeaderEpochFileCache] = {
+                                  recordVersion: RecordVersion,
+                                  logPrefix: String): Option[LeaderEpochFileCache] = {

Review comment:
       Could we add the default `logPrefix` parameter value to empty string? So all the tests don't need to update.

##########
File path: core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala
##########
@@ -283,7 +283,7 @@ class PartitionLockTest extends Logging {
         val log = super.createLog(isNew, isFutureReplica, offsetCheckpoints, None)
         val logDirFailureChannel = new LogDirFailureChannel(1)
         val segments = new LogSegments(log.topicPartition)
-        val leaderEpochCache = Log.maybeCreateLeaderEpochCache(log.dir, log.topicPartition, logDirFailureChannel, log.config.messageFormatVersion.recordVersion)
+        val leaderEpochCache = Log.maybeCreateLeaderEpochCache(log.dir, log.topicPartition, logDirFailureChannel, log.config.messageFormatVersion.recordVersion, "")

Review comment:
       As above mentioned, if the default logPrefix is empty string, then the following change in tests are not needed.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kowshik commented on a change in pull request #10742: MINOR: Add log identifier/prefix printing in Log layer static functions

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10742:
URL: https://github.com/apache/kafka/pull/10742#discussion_r637252631



##########
File path: core/src/main/scala/kafka/log/Log.scala
##########
@@ -2005,7 +2005,7 @@ object Log extends Logging {
     Files.createDirectories(dir.toPath)
     val topicPartition = Log.parseTopicPartitionName(dir)
     val segments = new LogSegments(topicPartition)
-    val leaderEpochCache = Log.maybeCreateLeaderEpochCache(dir, topicPartition, logDirFailureChannel, config.messageFormatVersion.recordVersion)
+    val leaderEpochCache = Log.maybeCreateLeaderEpochCache(dir, topicPartition, logDirFailureChannel, config.messageFormatVersion.recordVersion, this.logIdent)

Review comment:
       Done. Nice catch. Addressed in cc8353fd9e2475e56488ade8d6f01a772830edc4.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kowshik commented on a change in pull request #10742: MINOR: Add log identifier/prefix printing in Log layer static functions

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10742:
URL: https://github.com/apache/kafka/pull/10742#discussion_r637344863



##########
File path: core/src/main/scala/kafka/log/Log.scala
##########
@@ -2226,12 +2231,14 @@ object Log extends Logging {
    * @param topicPartition The topic partition
    * @param logDirFailureChannel The LogDirFailureChannel to asynchronously handle log dir failure
    * @param recordVersion The record version
+   * @param logPrefix The logging prefix
    * @return The new LeaderEpochFileCache instance (if created), none otherwise
    */
   def maybeCreateLeaderEpochCache(dir: File,
                                   topicPartition: TopicPartition,
                                   logDirFailureChannel: LogDirFailureChannel,
-                                  recordVersion: RecordVersion): Option[LeaderEpochFileCache] = {
+                                  recordVersion: RecordVersion,
+                                  logPrefix: String): Option[LeaderEpochFileCache] = {

Review comment:
       Introducing a default value can lead to programming error, because we could forget to pass it when it is really needed to be passed.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kowshik commented on a change in pull request #10742: MINOR: Add log identifier/prefix printing in Log layer static functions

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10742:
URL: https://github.com/apache/kafka/pull/10742#discussion_r637252571



##########
File path: core/src/main/scala/kafka/log/Log.scala
##########
@@ -2332,7 +2336,8 @@ object Log extends Logging {
           config,
           scheduler,
           logDirFailureChannel,
-          producerStateManager)
+          producerStateManager,
+          logPrefix)

Review comment:
       Done. Addressed in cc8353fd9e2475e56488ade8d6f01a772830edc4.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kowshik commented on a change in pull request #10742: MINOR: Add log identifier/prefix printing in Log layer static functions

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10742:
URL: https://github.com/apache/kafka/pull/10742#discussion_r636623604



##########
File path: core/src/main/scala/kafka/log/LogLoader.scala
##########
@@ -67,7 +67,7 @@ case class LoadLogParams(dir: File,
                          maxProducerIdExpirationMs: Int,
                          leaderEpochCache: Option[LeaderEpochFileCache],
                          producerStateManager: ProducerStateManager) {
-  val logIdentifier: String = s"[LogLoader partition=$topicPartition, dir=${dir.getParent}]"
+  val logIdentifier: String = s"[LogLoader partition=$topicPartition, dir=${dir.getParent}] "

Review comment:
       This is required because the existing identifier in the `Log` class is suffixed with a whitespace, see: https://github.com/apache/kafka/blob/5d4f9f917c8dc356a2d922980ba8101e0f2e7093/core/src/main/scala/kafka/log/Log.scala#L280.
   I'm adding a whitespace here to be consistent with what we already have in the code.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kowshik commented on pull request #10742: MINOR: Add log identifier/prefix printing in Log layer static functions

Posted by GitBox <gi...@apache.org>.
kowshik commented on pull request #10742:
URL: https://github.com/apache/kafka/pull/10742#issuecomment-845503390


   cc @junrao for review


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kowshik commented on a change in pull request #10742: MINOR: Add log identifier/prefix printing in Log layer static functions

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10742:
URL: https://github.com/apache/kafka/pull/10742#discussion_r637344880



##########
File path: core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala
##########
@@ -283,7 +283,7 @@ class PartitionLockTest extends Logging {
         val log = super.createLog(isNew, isFutureReplica, offsetCheckpoints, None)
         val logDirFailureChannel = new LogDirFailureChannel(1)
         val segments = new LogSegments(log.topicPartition)
-        val leaderEpochCache = Log.maybeCreateLeaderEpochCache(log.dir, log.topicPartition, logDirFailureChannel, log.config.messageFormatVersion.recordVersion)
+        val leaderEpochCache = Log.maybeCreateLeaderEpochCache(log.dir, log.topicPartition, logDirFailureChannel, log.config.messageFormatVersion.recordVersion, "")

Review comment:
       Explained here: https://github.com/apache/kafka/pull/10742#discussion_r637344863

##########
File path: core/src/test/scala/unit/kafka/cluster/PartitionLockTest.scala
##########
@@ -283,7 +283,7 @@ class PartitionLockTest extends Logging {
         val log = super.createLog(isNew, isFutureReplica, offsetCheckpoints, None)
         val logDirFailureChannel = new LogDirFailureChannel(1)
         val segments = new LogSegments(log.topicPartition)
-        val leaderEpochCache = Log.maybeCreateLeaderEpochCache(log.dir, log.topicPartition, logDirFailureChannel, log.config.messageFormatVersion.recordVersion)
+        val leaderEpochCache = Log.maybeCreateLeaderEpochCache(log.dir, log.topicPartition, logDirFailureChannel, log.config.messageFormatVersion.recordVersion, "")

Review comment:
       I've explained here: https://github.com/apache/kafka/pull/10742#discussion_r637344863




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kowshik commented on a change in pull request #10742: MINOR: Add log identifier/prefix printing in Log layer static functions

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10742:
URL: https://github.com/apache/kafka/pull/10742#discussion_r636623193



##########
File path: core/src/main/scala/kafka/log/LogLoader.scala
##########
@@ -172,7 +173,7 @@ object LogLoader extends Logging {
   private def removeTempFilesAndCollectSwapFiles(params: LoadLogParams): Set[File] = {
 
     def deleteIndicesIfExist(baseFile: File, suffix: String = ""): Unit = {
-      info(s"${params.logIdentifier} Deleting index files with suffix $suffix for baseFile $baseFile")
+      info(s"${params.logIdentifier}Deleting index files with suffix $suffix for baseFile $baseFile")

Review comment:
       Yes, because the existing identifier in the `Log` class is suffixed with a whitespace, see: https://github.com/apache/kafka/blob/5d4f9f917c8dc356a2d922980ba8101e0f2e7093/core/src/main/scala/kafka/log/Log.scala#L280.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kowshik commented on a change in pull request #10742: MINOR: Add log identifier/prefix printing in Log layer static functions

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10742:
URL: https://github.com/apache/kafka/pull/10742#discussion_r636623604



##########
File path: core/src/main/scala/kafka/log/LogLoader.scala
##########
@@ -67,7 +67,7 @@ case class LoadLogParams(dir: File,
                          maxProducerIdExpirationMs: Int,
                          leaderEpochCache: Option[LeaderEpochFileCache],
                          producerStateManager: ProducerStateManager) {
-  val logIdentifier: String = s"[LogLoader partition=$topicPartition, dir=${dir.getParent}]"
+  val logIdentifier: String = s"[LogLoader partition=$topicPartition, dir=${dir.getParent}] "

Review comment:
       This is required because the existing identifier in the `Log` class is suffixed with a whitespace, see: https://github.com/apache/kafka/blob/5d4f9f917c8dc356a2d922980ba8101e0f2e7093/core/src/main/scala/kafka/log/Log.scala#L280
   I'm adding a whitespace here to be consistent with what we already have in the code.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] ccding commented on a change in pull request #10742: MINOR: Add log identifier/prefix printing in Log layer static functions

Posted by GitBox <gi...@apache.org>.
ccding commented on a change in pull request #10742:
URL: https://github.com/apache/kafka/pull/10742#discussion_r636574778



##########
File path: core/src/main/scala/kafka/log/LogLoader.scala
##########
@@ -172,7 +173,7 @@ object LogLoader extends Logging {
   private def removeTempFilesAndCollectSwapFiles(params: LoadLogParams): Set[File] = {
 
     def deleteIndicesIfExist(baseFile: File, suffix: String = ""): Unit = {
-      info(s"${params.logIdentifier} Deleting index files with suffix $suffix for baseFile $baseFile")
+      info(s"${params.logIdentifier}Deleting index files with suffix $suffix for baseFile $baseFile")

Review comment:
       is the change (remove blank space here) along with https://github.com/apache/kafka/pull/10742/files#diff-54b3df71b1e0697a211d23a9018a91aef773fca0b9cbd1abafbdca6c79664930L70-R70 (add blank space there) needed? [I am not sure the best practice in Kafka codebase]




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kowshik commented on pull request #10742: MINOR: Add log identifier/prefix printing in Log layer static functions

Posted by GitBox <gi...@apache.org>.
kowshik commented on pull request #10742:
URL: https://github.com/apache/kafka/pull/10742#issuecomment-846281450


   @junrao Thanks for the review! I've addressed your comments in cc8353fd9e2475e56488ade8d6f01a772830edc4.


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] showuon commented on a change in pull request #10742: MINOR: Add log identifier/prefix printing in Log layer static functions

Posted by GitBox <gi...@apache.org>.
showuon commented on a change in pull request #10742:
URL: https://github.com/apache/kafka/pull/10742#discussion_r636598563



##########
File path: core/src/main/scala/kafka/log/LogLoader.scala
##########
@@ -67,7 +67,7 @@ case class LoadLogParams(dir: File,
                          maxProducerIdExpirationMs: Int,
                          leaderEpochCache: Option[LeaderEpochFileCache],
                          producerStateManager: ProducerStateManager) {
-  val logIdentifier: String = s"[LogLoader partition=$topicPartition, dir=${dir.getParent}]"
+  val logIdentifier: String = s"[LogLoader partition=$topicPartition, dir=${dir.getParent}] "

Review comment:
       Why do you add a space here, and remove the space in the following log content?

##########
File path: core/src/main/scala/kafka/log/Log.scala
##########
@@ -2225,13 +2225,15 @@ object Log extends Logging {
    * @param dir The directory in which the log will reside
    * @param topicPartition The topic partition
    * @param logDirFailureChannel The LogDirFailureChannel to asynchronously handle log dir failure
-   * @param recordVersion The record version
+   * @param recordVersion The record

Review comment:
       Is it intended to change the `The record version` to `The record`?




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kowshik commented on pull request #10742: MINOR: Add log identifier/prefix printing in Log layer static functions

Posted by GitBox <gi...@apache.org>.
kowshik commented on pull request #10742:
URL: https://github.com/apache/kafka/pull/10742#issuecomment-845641737


   @ccding and @showuon: Thanks for the review! I've addressed your comment(s) and responded to your questions.


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] showuon commented on a change in pull request #10742: MINOR: Add log identifier/prefix printing in Log layer static functions

Posted by GitBox <gi...@apache.org>.
showuon commented on a change in pull request #10742:
URL: https://github.com/apache/kafka/pull/10742#discussion_r637353302



##########
File path: core/src/main/scala/kafka/log/Log.scala
##########
@@ -2226,12 +2231,14 @@ object Log extends Logging {
    * @param topicPartition The topic partition
    * @param logDirFailureChannel The LogDirFailureChannel to asynchronously handle log dir failure
    * @param recordVersion The record version
+   * @param logPrefix The logging prefix
    * @return The new LeaderEpochFileCache instance (if created), none otherwise
    */
   def maybeCreateLeaderEpochCache(dir: File,
                                   topicPartition: TopicPartition,
                                   logDirFailureChannel: LogDirFailureChannel,
-                                  recordVersion: RecordVersion): Option[LeaderEpochFileCache] = {
+                                  recordVersion: RecordVersion,
+                                  logPrefix: String): Option[LeaderEpochFileCache] = {

Review comment:
       Ok, good to me.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] junrao commented on a change in pull request #10742: MINOR: Add log identifier/prefix printing in Log layer static functions

Posted by GitBox <gi...@apache.org>.
junrao commented on a change in pull request #10742:
URL: https://github.com/apache/kafka/pull/10742#discussion_r637278778



##########
File path: core/src/main/scala/kafka/log/Log.scala
##########
@@ -2005,7 +2005,12 @@ object Log extends Logging {
     Files.createDirectories(dir.toPath)
     val topicPartition = Log.parseTopicPartitionName(dir)
     val segments = new LogSegments(topicPartition)
-    val leaderEpochCache = Log.maybeCreateLeaderEpochCache(dir, topicPartition, logDirFailureChannel, config.messageFormatVersion.recordVersion)
+    val leaderEpochCache = Log.maybeCreateLeaderEpochCache(
+      dir,
+      topicPartition,
+      logDirFailureChannel,
+      config.messageFormatVersion.recordVersion,
+      s"[Log partition=$topicPartition, dir=${dir.getParent}] )")

Review comment:
       Should we remove ) before the ending quotation mark?




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] junrao merged pull request #10742: MINOR: Add log identifier/prefix printing in Log layer static functions

Posted by GitBox <gi...@apache.org>.
junrao merged pull request #10742:
URL: https://github.com/apache/kafka/pull/10742


   


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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] kowshik commented on a change in pull request #10742: MINOR: Add log identifier/prefix printing in Log layer static functions

Posted by GitBox <gi...@apache.org>.
kowshik commented on a change in pull request #10742:
URL: https://github.com/apache/kafka/pull/10742#discussion_r637283022



##########
File path: core/src/main/scala/kafka/log/Log.scala
##########
@@ -2005,7 +2005,12 @@ object Log extends Logging {
     Files.createDirectories(dir.toPath)
     val topicPartition = Log.parseTopicPartitionName(dir)
     val segments = new LogSegments(topicPartition)
-    val leaderEpochCache = Log.maybeCreateLeaderEpochCache(dir, topicPartition, logDirFailureChannel, config.messageFormatVersion.recordVersion)
+    val leaderEpochCache = Log.maybeCreateLeaderEpochCache(
+      dir,
+      topicPartition,
+      logDirFailureChannel,
+      config.messageFormatVersion.recordVersion,
+      s"[Log partition=$topicPartition, dir=${dir.getParent}] )")

Review comment:
       Done, sorry for the typo. Fixed in 0b429dcb5a21580d1b8f5cca7183101c98a6faee.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org