chia7712 commented on code in PR #17373: URL: https://github.com/apache/kafka/pull/17373#discussion_r1846550136
########## connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Loggers.java: ########## @@ -174,43 +179,50 @@ private synchronized List<org.apache.log4j.Logger> loggers(String namespace) { } // visible for testing - org.apache.log4j.Logger lookupLogger(String logger) { + org.apache.logging.log4j.Logger lookupLogger(String logger) { return LogManager.getLogger(logger); } - @SuppressWarnings("unchecked") - // visible for testing - Enumeration<org.apache.log4j.Logger> currentLoggers() { - return LogManager.getCurrentLoggers(); + List<org.apache.logging.log4j.Logger> currentLoggers() { + LoggerContext context = (LoggerContext) LogManager.getContext(false); + Collection<LoggerConfig> loggerConfigs = context.getConfiguration().getLoggers().values(); + Set<String> loggerNames = loggerConfigs.stream() Review Comment: we don't need to recreate collection - ```java return loggerConfigs.stream() .map(LoggerConfig::getName) .distinct() .map(LogManager::getLogger) .collect(Collectors.toUnmodifiableList()); ``` ########## core/src/main/scala/kafka/utils/Log4jController.scala: ########## @@ -17,83 +17,80 @@ package kafka.utils +import org.apache.logging.log4j.core.LoggerContext +import org.apache.logging.log4j.core.config.Configurator +import org.apache.logging.log4j.{Level, LogManager} + import java.util import java.util.Locale - -import org.apache.kafka.common.utils.Utils -import org.apache.log4j.{Level, LogManager, Logger} - -import scala.collection.mutable 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 - } + + val ROOT_LOGGER = "" /** - * 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 - */ - def loggers: mutable.Map[String, String] = { - val logs = new mutable.HashMap[String, String]() - val rootLoggerLvl = existingLogger(ROOT_LOGGER).getLevel.toString - logs.put(ROOT_LOGGER, rootLoggerLvl) - - val loggers = LogManager.getCurrentLoggers - while (loggers.hasMoreElements) { - val logger = loggers.nextElement().asInstanceOf[Logger] - if (logger != null) { - logs.put(logger.getName, resolveLevel(logger)) - } - } - logs + * 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 log level of the first ancestor with a level configured. + */ + def loggers: Map[String, String] = { + val logContext = LogManager.getContext(false).asInstanceOf[LoggerContext] + val rootLoggerLevel = logContext.getRootLogger.getLevel.toString + + // Loggers defined in the configuration + val configured = logContext.getConfiguration.getLoggers.asScala + .values + .filter(_.getName != LogManager.ROOT_LOGGER_NAME) + .map { logger => + logger.getName -> logger.getLevel.toString + }.toMap + + // Loggers actually running + val actual = logContext.getLoggers.asScala + .filter(_.getName != LogManager.ROOT_LOGGER_NAME) + .map { logger => + logger.getName -> logger.getLevel.toString + }.toMap + + (configured ++ actual) + (ROOT_LOGGER -> rootLoggerLevel) } /** - * Sets the log level of a particular logger - */ + * Sets the log level of a particular logger. If the given logLevel is not an available log4j level + * (i.e., one of OFF, FATAL, ERROR, WARN, INFO, DEBUG, TRACE, ALL) it falls back to DEBUG. + * + * @see [[Level.toLevel]] + */ def logLevel(loggerName: String, logLevel: String): Boolean = { - val log = existingLogger(loggerName) - if (!Utils.isBlank(loggerName) && !Utils.isBlank(logLevel) && log != null) { - log.setLevel(Level.toLevel(logLevel.toUpperCase(Locale.ROOT))) + val level = Level.toLevel(logLevel.toUpperCase(Locale.ROOT)) Review Comment: we need to keep the null handle to avoid NPE ########## core/src/main/scala/kafka/utils/Log4jController.scala: ########## @@ -17,83 +17,80 @@ package kafka.utils +import org.apache.logging.log4j.core.LoggerContext +import org.apache.logging.log4j.core.config.Configurator +import org.apache.logging.log4j.{Level, LogManager} + import java.util import java.util.Locale - -import org.apache.kafka.common.utils.Utils -import org.apache.log4j.{Level, LogManager, Logger} - -import scala.collection.mutable 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 - } + + val ROOT_LOGGER = "" Review Comment: why don't we keep the `root` compatibility here? After this PR, users can't set 'root=warn' to change the root level. -- 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