viktorsomogyi commented on code in PR #15697: URL: https://github.com/apache/kafka/pull/15697#discussion_r1570907841
########## core/src/main/scala/kafka/server/KafkaConfig.scala: ########## @@ -528,6 +529,10 @@ object KafkaConfig { "If log.message.timestamp.type=CreateTime, the message will be rejected if the difference in timestamps exceeds " + "this specified threshold. This configuration is ignored if log.message.timestamp.type=LogAppendTime." + val LogDirFailureTimeoutMsDoc = "If the broker is unable to successfully communicate to the controller that some log " + + "directory has failed for longer than this time, and there's at least one partition with leadership on that directory, " + Review Comment: I'll do another round with this, there might be a way in `BrokerServer` to extract this information using the combination of `MetadataCache`, `ReplicaManager` and `LogManager`. I'll update you tomorrow about my findings. ########## core/src/main/scala/kafka/server/BrokerLifecycleManager.scala: ########## @@ -327,16 +333,25 @@ class BrokerLifecycleManager( private class OfflineDirEvent(val dir: Uuid) extends EventQueue.Event { override def run(): Unit = { if (offlineDirs.isEmpty) { - offlineDirs = Set(dir) + offlineDirs = Map(dir -> false) } else { - offlineDirs = offlineDirs + dir + offlineDirs += (dir -> false) } if (registered) { scheduleNextCommunicationImmediately() } } } + private class OfflineDirBrokerFailureEvent(offlineDir: Uuid) extends EventQueue.Event { + override def run(): Unit = { + if (!offlineDirs.getOrElse(offlineDir, false)) { + error(s"Shutting down because couldn't communicate offline log dirs with controllers") Review Comment: I'll print the UUID here only and I'll modify other log statements to contain the UUID so one can pair these log statements when analyzing the logs. Printing the dir path here would be a little bit bigger stretch as we currently don't propagate it down to this level. Let me know if you think it'd be better to print the path here. -- 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