Skip to content

Commit

Permalink
KAFKA-18243: Fix compatibility of Loggers class between log4j and log4j2
Browse files Browse the repository at this point in the history
  • Loading branch information
frankvicky committed Dec 25, 2024
1 parent 5a4590c commit 015e80e
Show file tree
Hide file tree
Showing 6 changed files with 449 additions and 251 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -109,16 +109,16 @@ public void setClassLogger(final Class<?> clazz, Level level) {
LoggerContext ctx = (LoggerContext) LogManager.getContext(false);
Configuration config = ctx.getConfiguration();
String loggerName = clazz.getName();
LoggerConfig loggerConfig = config.getLoggerConfig(loggerName);
org.apache.logging.log4j.core.Logger logger = ctx.getLoggerRegistry().getLogger(loggerName);

Level originalLevel = loggerConfig.getLevel();
Level originalLevel = logger.getLevel();
logLevelChanges.add(new LogLevelChange(originalLevel, clazz));

if (!loggerConfig.getName().equals(loggerName)) {
if (!logger.getName().equals(loggerName)) {
LoggerConfig newLoggerConfig = new LoggerConfig(loggerName, level, true);
config.addLogger(loggerName, newLoggerConfig);
} else {
loggerConfig.setLevel(level);
logger.setLevel(level);
}
ctx.updateLoggers();
}
Expand Down Expand Up @@ -182,7 +182,7 @@ public void close() {

for (final LogLevelChange logLevelChange : logLevelChanges) {
String loggerName = logLevelChange.clazz.getName();
LoggerConfig loggerConfig = config.getLoggerConfig(loggerName);
org.apache.logging.log4j.core.Logger loggerConfig = ctx.getLogger(loggerName);
if (!loggerConfig.getName().equals(loggerName)) {
LoggerConfig newLoggerConfig = new LoggerConfig(loggerName, logLevelChange.originalLevel, true);
config.addLogger(loggerName, newLoggerConfig);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,12 +23,10 @@
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.core.LoggerContext;
import org.apache.logging.log4j.core.config.Configurator;
import org.apache.logging.log4j.core.config.LoggerConfig;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
Expand All @@ -48,14 +46,17 @@ public class Loggers {

private static final Logger log = LoggerFactory.getLogger(Loggers.class);

private static final String ROOT_LOGGER_NAME = "root";

/**
* Log4j uses "root" (case-insensitive) as name of the root logger.
* Note: In log4j, the root logger's name was "root" and Kafka also followed that name for dynamic logging control feature.
*
* The root logger's name is changed in log4j2 to empty string (see: [[LogManager.ROOT_LOGGER_NAME]]) but for backward-
* compatibility. Kafka keeps its original root logger name. It is why here is a dedicated definition for the root logger name.
* While log4j2 changed the root logger's name to empty string (see: [[LogManager.ROOT_LOGGER_NAME]]),
* for backward-compatibility purposes, we accept both empty string and "root" as valid root logger names.
* This is why we have a dedicated definition that includes both values.
*/
private static final String ROOT_LOGGER_NAME = "root";
private static final List<String> VALID_ROOT_LOGGER_NAMES = List.of(LogManager.ROOT_LOGGER_NAME, ROOT_LOGGER_NAME);

private final Time time;
private final Map<String, Long> lastModifiedTimes;
Expand All @@ -75,10 +76,10 @@ public synchronized LoggerLevel level(String logger) {
Objects.requireNonNull(logger, "Logger may not be null");

org.apache.logging.log4j.Logger foundLogger = null;
if (ROOT_LOGGER_NAME.equalsIgnoreCase(logger)) {
if (isValidRootLoggerName(logger)) {
foundLogger = rootLogger();
} else {
List<org.apache.logging.log4j.Logger> currentLoggers = currentLoggers();
var currentLoggers = currentLoggers().values();
// search within existing loggers for the given name.
// using LogManger.getLogger() will create a logger if it doesn't exist
// (potential leak since these don't get cleaned up).
Expand All @@ -103,18 +104,16 @@ public synchronized LoggerLevel level(String logger) {
* @return the levels of all known loggers; may be empty, but never null
*/
public synchronized Map<String, LoggerLevel> allLevels() {
Map<String, LoggerLevel> result = new TreeMap<>();

currentLoggers().stream()
.filter(logger -> !logger.getLevel().equals(Level.OFF))
.forEach(logger -> result.put(logger.getName(), loggerLevel(logger)));

org.apache.logging.log4j.Logger root = rootLogger();
if (!root.getLevel().equals(Level.OFF)) {
result.put(ROOT_LOGGER_NAME, loggerLevel(root));
}

return result;
return currentLoggers()
.values()
.stream()
.filter(logger -> !logger.getLevel().equals(Level.OFF))
.collect(Collectors.toMap(
this::getLoggerName,
this::loggerLevel,
(existing, replacing) -> replacing,
TreeMap::new)
);
}

/**
Expand All @@ -129,12 +128,12 @@ public synchronized List<String> setLevel(String namespace, Level level) {
Objects.requireNonNull(level, "Level may not be null");

log.info("Setting level of namespace {} and children to {}", namespace, level);
List<org.apache.logging.log4j.Logger> childLoggers = loggers(namespace);
var loggers = loggers(namespace);

List<String> result = new ArrayList<>();
for (org.apache.logging.log4j.Logger logger: childLoggers) {
for (org.apache.logging.log4j.Logger logger: loggers) {
setLevel(logger, level);
result.add(logger.getName());
result.add(getLoggerName(logger));
}
Collections.sort(result);

Expand All @@ -151,15 +150,13 @@ public synchronized List<String> setLevel(String namespace, Level level) {
private synchronized List<org.apache.logging.log4j.Logger> loggers(String namespace) {
Objects.requireNonNull(namespace, "Logging namespace may not be null");

if (ROOT_LOGGER_NAME.equalsIgnoreCase(namespace)) {
List<org.apache.logging.log4j.Logger> result = currentLoggers();
result.add(rootLogger());
return result;
if (isValidRootLoggerName(namespace)) {
return new ArrayList<>(currentLoggers().values());
}

List<org.apache.logging.log4j.Logger> result = new ArrayList<>();
org.apache.logging.log4j.Logger ancestorLogger = lookupLogger(namespace);
List<org.apache.logging.log4j.Logger> currentLoggers = currentLoggers();
var result = new ArrayList<org.apache.logging.log4j.Logger>();
var ancestorLogger = lookupLogger(namespace);
var currentLoggers = currentLoggers().values();
boolean present = false;
for (org.apache.logging.log4j.Logger currentLogger : currentLoggers) {
if (currentLogger.getName().startsWith(namespace)) {
Expand All @@ -179,17 +176,16 @@ private synchronized List<org.apache.logging.log4j.Logger> loggers(String namesp

// visible for testing
org.apache.logging.log4j.Logger lookupLogger(String logger) {
return LogManager.getLogger(logger);
return LogManager.getLogger(isValidRootLoggerName(logger) ? LogManager.ROOT_LOGGER_NAME : logger);
}

List<org.apache.logging.log4j.Logger> currentLoggers() {
Map<String, org.apache.logging.log4j.Logger> currentLoggers() {
LoggerContext context = (LoggerContext) LogManager.getContext(false);
Collection<LoggerConfig> loggerConfigs = context.getConfiguration().getLoggers().values();
return loggerConfigs.stream()
.map(LoggerConfig::getName)
.distinct()
.map(LogManager::getLogger)
.collect(Collectors.toCollection(ArrayList::new));
// Make sure root logger has been initialized
var results = new HashMap<String, org.apache.logging.log4j.Logger>();
context.getConfiguration().getLoggers().forEach((name, logger) -> results.put(name, LogManager.getLogger(name)));
context.getLoggerRegistry().getLoggers().forEach(logger -> results.put(logger.getName(), logger));
return results;
}

// visible for testing
Expand All @@ -198,26 +194,37 @@ org.apache.logging.log4j.Logger rootLogger() {
}

private void setLevel(org.apache.logging.log4j.Logger logger, Level level) {
String loggerName = logger.getName();
LoggerContext context = (LoggerContext) LogManager.getContext(false);
LoggerConfig loggerConfig = context.getConfiguration().getLoggerConfig(loggerName);
Level currentLevel = loggerConfig.getLevel();
Level currentLevel = logger.getLevel();

if (level.equals(currentLevel)) {
log.debug("Skipping update for logger {} since its level is already {}", loggerName, level);
log.debug("Skipping update for logger {} since its level is already {}", logger.getName(), level);
return;
}

log.debug("Setting level of logger {} (excluding children) to {}", loggerName, level);
Configurator.setLevel(loggerName, level);
lastModifiedTimes.put(loggerName, time.milliseconds());
log.debug("Setting level of logger {} (excluding children) to {}", logger.getName(), level);
Configurator.setLevel(logger.getName(), level);
lastModifiedTimes.put(logger.getName(), time.milliseconds());
}

private LoggerLevel loggerLevel(org.apache.logging.log4j.Logger logger) {
LoggerContext context = (LoggerContext) LogManager.getContext(false);
LoggerConfig loggerConfig = context.getConfiguration().getLoggerConfig(logger.getName());
Level level = loggerConfig.getLevel();
Long lastModified = lastModifiedTimes.get(logger.getName());
return new LoggerLevel(Objects.toString(level), lastModified);
return new LoggerLevel(Objects.toString(logger.getLevel()), lastModified);
}

private boolean isValidRootLoggerName(String namespace) {
return VALID_ROOT_LOGGER_NAMES.stream()
.anyMatch(rootLoggerNames -> rootLoggerNames.equalsIgnoreCase(namespace));
}

/**
* Converts logger name to ensure backward compatibility between log4j and log4j2.
* If the logger name is empty (log4j2's root logger representation), converts it to "root" (log4j's style).
* Otherwise, returns the original logger name.
*
* @param logger The logger instance to get the name from
* @return The logger name - returns "root" for empty string, otherwise returns the original logger name
*/
private String getLoggerName(org.apache.logging.log4j.Logger logger) {
return logger.getName().equals(LogManager.ROOT_LOGGER_NAME) ? ROOT_LOGGER_NAME : logger.getName();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -184,8 +184,7 @@ private Map<String, LoggerLevel> testSetLoggingLevel(
newLevels,
e -> hasNamespace(e, namespace)
&& (!level(e).equals(level)
|| !isModified(e)
|| lastModified(e) < requestTime
|| (isModified(e) && lastModified(e) < requestTime)
)
);
assertEquals(
Expand Down
Loading

0 comments on commit 015e80e

Please sign in to comment.