Skip to content

Commit

Permalink
KAFKA-10469: Resolve logger levels hierarchically (apache#9266)
Browse files Browse the repository at this point in the history
Previous to root logger level was used, ignoring intervening loggers with
different levels.

Reviewers: Stanislav Kozlovski <stanislav_kozlovski@outlook.com>, Lee Dongjin <dongjin@apache.org>, Ismael Juma <ismael@juma.me.uk>
tombentley authored Sep 30, 2020
1 parent 821c1ac commit fda6701
Showing 3 changed files with 45 additions and 7 deletions.
23 changes: 20 additions & 3 deletions core/src/main/scala/kafka/utils/Log4jController.scala
Original file line number Diff line number Diff line change
@@ -29,6 +29,24 @@ import scala.jdk.CollectionConverters._
object Log4jController {
val ROOT_LOGGER = "root"

private def resolveLevel(logger: Logger): String = {
var name = logger.getName
var level = logger.getLevel
while (level == null) {
val index = name.lastIndexOf(".")
if (index > 0) {
name = name.substring(0, index)
val ancestor = existingLogger(name)
if (ancestor != null) {
level = ancestor.getLevel
}
} else {
level = existingLogger(ROOT_LOGGER).getLevel
}
}
level.toString
}

/**
* Returns a map of the log4j loggers and their assigned log level.
* If a logger does not have a log level assigned, we return the root logger's log level
@@ -42,8 +60,7 @@ object Log4jController {
while (loggers.hasMoreElements) {
val logger = loggers.nextElement().asInstanceOf[Logger]
if (logger != null) {
val level = if (logger.getLevel != null) logger.getLevel.toString else rootLoggerLvl
logs.put(logger.getName, level)
logs.put(logger.getName, resolveLevel(logger))
}
}
logs
@@ -101,7 +118,7 @@ class Log4jController extends Log4jControllerMBean {
if (level != null)
log.getLevel.toString
else
Log4jController.existingLogger(Log4jController.ROOT_LOGGER).getLevel.toString
Log4jController.resolveLevel(log)
}
else "No such logger."
}
Original file line number Diff line number Diff line change
@@ -44,6 +44,7 @@ import org.apache.kafka.common.{ConsumerGroupState, ElectionType, TopicPartition
import org.junit.Assert._
import org.junit.{After, Before, Ignore, Test}
import org.scalatest.Assertions.intercept
import org.slf4j.LoggerFactory

import scala.annotation.nowarn
import scala.jdk.CollectionConverters._
@@ -2061,11 +2062,12 @@ class PlaintextAdminIntegrationTest extends BaseAdminIntegrationTest {
@Test
def testDescribeConfigsForLog4jLogLevels(): Unit = {
client = Admin.create(createConfig)

LoggerFactory.getLogger("kafka.cluster.Replica").trace("Message to create the logger")
val loggerConfig = describeBrokerLoggers()
val rootLogLevel = loggerConfig.get(Log4jController.ROOT_LOGGER).value()
val kafkaLogLevel = loggerConfig.get("kafka").value()
val logCleanerLogLevelConfig = loggerConfig.get("kafka.cluster.Replica")
assertEquals(rootLogLevel, logCleanerLogLevelConfig.value()) // we expect an undefined log level to be the same as the root logger
// we expect the log level to be inherited from the first ancestor with a level configured
assertEquals(kafkaLogLevel, logCleanerLogLevelConfig.value())
assertEquals("kafka.cluster.Replica", logCleanerLogLevelConfig.name())
assertEquals(ConfigEntry.ConfigSource.DYNAMIC_BROKER_LOGGER_CONFIG, logCleanerLogLevelConfig.source())
assertEquals(false, logCleanerLogLevelConfig.isReadOnly)
21 changes: 20 additions & 1 deletion core/src/test/scala/kafka/utils/LoggingTest.scala
Original file line number Diff line number Diff line change
@@ -18,10 +18,11 @@
package kafka.utils

import java.lang.management.ManagementFactory
import javax.management.ObjectName

import javax.management.ObjectName
import org.junit.Test
import org.junit.Assert.{assertEquals, assertTrue}
import org.slf4j.LoggerFactory


class LoggingTest extends Logging {
@@ -66,4 +67,22 @@ class LoggingTest extends Logging {

assertEquals(logging.getClass.getName, logging.log.underlying.getName)
}

@Test
def testLoggerLevelIsResolved(): Unit = {
val controller = new Log4jController()
val previousLevel = controller.getLogLevel("kafka")
try {
controller.setLogLevel("kafka", "TRACE")
// Do some logging so that the Logger is created within the hierarchy
// (until loggers are used only loggers in the config file exist)
LoggerFactory.getLogger("kafka.utils.Log4jControllerTest").trace("test")
assertEquals("TRACE", controller.getLogLevel("kafka"))
assertEquals("TRACE", controller.getLogLevel("kafka.utils.Log4jControllerTest"))
assertTrue(controller.getLoggers.contains("kafka=TRACE"))
assertTrue(controller.getLoggers.contains("kafka.utils.Log4jControllerTest=TRACE"))
} finally {
controller.setLogLevel("kafka", previousLevel)
}
}
}

0 comments on commit fda6701

Please sign in to comment.