KAFKA-18243 Fix compatibility of Loggers class between log4j and log4j2 (#18185)

Reviewers: Chia-Ping Tsai <chia7712@gmail.com>
This commit is contained in:
TengYao Chi 2024-12-30 01:50:52 +08:00 committed by GitHub
parent 3654bc4513
commit 96527be90d
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
6 changed files with 462 additions and 257 deletions

View File

@ -120,7 +120,7 @@ public class LogCaptureAppender extends AbstractAppender implements AutoCloseabl
}
public List<String> getMessages() {
final LinkedList<String> result = new LinkedList<>();
final List<String> result = new LinkedList<>();
synchronized (events) {
for (final LogEvent event : events) {
result.add(event.getMessage().getFormattedMessage());

View File

@ -23,7 +23,6 @@ import org.apache.logging.log4j.Level;
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;
@ -48,16 +47,24 @@ 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;
/**
* Maps logger names to their last modification timestamps.
* Note: The logger name "root" refers to the actual root logger of log4j2.
*/
private final Map<String, Long> lastModifiedTimes;
public Loggers(Time time) {
@ -75,10 +82,10 @@ public class Loggers {
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).
@ -103,18 +110,16 @@ public class Loggers {
* @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()
return currentLoggers()
.values()
.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;
.collect(Collectors.toMap(
this::getLoggerName,
this::loggerLevel,
(existing, replacing) -> replacing,
TreeMap::new)
);
}
/**
@ -127,14 +132,25 @@ public class Loggers {
public synchronized List<String> setLevel(String namespace, Level level) {
Objects.requireNonNull(namespace, "Logging namespace may not be null");
Objects.requireNonNull(level, "Level may not be null");
String internalNameSpace = isValidRootLoggerName(namespace) ? LogManager.ROOT_LOGGER_NAME : namespace;
log.info("Setting level of namespace {} and children to {}", namespace, level);
List<org.apache.logging.log4j.Logger> childLoggers = loggers(namespace);
log.info("Setting level of namespace {} and children to {}", internalNameSpace, level);
var loggers = loggers(internalNameSpace);
var nameToLevel = allLevels();
List<String> result = new ArrayList<>();
for (org.apache.logging.log4j.Logger logger: childLoggers) {
setLevel(logger, level);
result.add(logger.getName());
Configurator.setAllLevels(internalNameSpace, level);
for (org.apache.logging.log4j.Logger logger : loggers) {
// We need to track level changes for each logger and record their update timestamps to ensure this method
// correctly returns only the loggers whose levels were actually modified.
String name = getLoggerName(logger);
String newLevel = logger.getLevel().name();
String oldLevel = nameToLevel.getOrDefault(name, new LoggerLevel("", time.milliseconds())).level();
if (!newLevel.equalsIgnoreCase(oldLevel)) {
lastModifiedTimes.put(name, time.milliseconds());
result.add(name);
}
}
Collections.sort(result);
@ -148,18 +164,18 @@ public class Loggers {
* @return all loggers that fall under the given namespace; never null, and will always contain
* at least one logger (the ancestor logger for the namespace)
*/
private synchronized List<org.apache.logging.log4j.Logger> loggers(String namespace) {
private synchronized Collection<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 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 nameToLogger = currentLoggers();
var ancestorLogger = lookupLogger(namespace);
var currentLoggers = nameToLogger.values();
boolean present = false;
for (org.apache.logging.log4j.Logger currentLogger : currentLoggers) {
if (currentLogger.getName().startsWith(namespace)) {
@ -179,17 +195,15 @@ public class Loggers {
// 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));
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
@ -197,27 +211,25 @@ public class Loggers {
return LogManager.getRootLogger();
}
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();
if (level.equals(currentLevel)) {
log.debug("Skipping update for logger {} since its level is already {}", loggerName, level);
return;
}
log.debug("Setting level of logger {} (excluding children) to {}", loggerName, level);
Configurator.setLevel(loggerName, level);
lastModifiedTimes.put(loggerName, 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);
Long lastModified = lastModifiedTimes.get(getLoggerName(logger));
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();
}
}

View File

@ -184,8 +184,7 @@ public class StandaloneWorkerIntegrationTest {
newLevels,
e -> hasNamespace(e, namespace)
&& (!level(e).equals(level)
|| !isModified(e)
|| lastModified(e) < requestTime
|| (isModified(e) && lastModified(e) < requestTime)
)
);
assertEquals(

View File

@ -20,248 +20,165 @@ import org.apache.kafka.common.utils.MockTime;
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.connect.runtime.rest.entities.LoggerLevel;
import org.apache.logging.log4j.Level;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import org.apache.logging.log4j.core.LoggerContext;
import org.apache.logging.log4j.core.config.Configuration;
import org.apache.logging.log4j.core.config.Configurator;
import org.apache.logging.log4j.core.config.LoggerConfig;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.extension.ExtendWith;
import org.mockito.junit.jupiter.MockitoExtension;
import org.mockito.junit.jupiter.MockitoSettings;
import org.mockito.quality.Strictness;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.function.Function;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import static org.apache.logging.log4j.Level.DEBUG;
import static org.apache.logging.log4j.Level.ERROR;
import static org.apache.logging.log4j.Level.INFO;
import static org.apache.logging.log4j.Level.WARN;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertNull;
import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.junit.jupiter.api.Assertions.assertTrue;
@ExtendWith(MockitoExtension.class)
@MockitoSettings(strictness = Strictness.STRICT_STUBS)
public class LoggersTest {
private static final long INITIAL_TIME = 1696951712135L;
private final LoggerContext context = (LoggerContext) LogManager.getContext(false);
private Loggers loggers;
private Time time;
@BeforeEach
public void setup() {
time = new MockTime(0, INITIAL_TIME, 0);
loggers = new Loggers(time);
}
@Test
public void testGetLoggersIgnoresNullLevels() {
LoggerContext loggerContext = (LoggerContext) LogManager.getContext(false);
Logger root = loggerContext.getRootLogger();
Configurator.setLevel(root, Level.OFF);
public void testLevelWithNullLoggerName() {
NullPointerException exception = assertThrows(NullPointerException.class, () -> loggers.level(null));
assertEquals("Logger may not be null", exception.getMessage());
}
Logger a = loggerContext.getLogger("a");
Configurator.setLevel(a, null);
Logger b = loggerContext.getLogger("b");
Configurator.setLevel(b, Level.INFO);
Loggers loggers = new TestLoggers(root, a, b);
Map<String, LoggerLevel> expectedLevels = Collections.singletonMap(
"b",
new LoggerLevel(Level.INFO.toString(), null)
@Test
public void testLevelWithValidRootLoggerNames() {
assertEquals(
loggers.level(""),
loggers.level("root"),
"Root logger level should be the same whether accessed via empty string or 'root' name"
);
Map<String, LoggerLevel> actualLevels = loggers.allLevels();
assertEquals(expectedLevels, actualLevels);
}
@Test
public void testGetLoggerFallsBackToEffectiveLogLevel() {
LoggerContext loggerContext = (LoggerContext) LogManager.getContext(false);
Logger root = loggerContext.getRootLogger();
Configurator.setLevel(root, Level.ERROR);
Logger a = loggerContext.getLogger("a");
Configurator.setLevel(a, null);
Logger b = loggerContext.getLogger("b");
Configurator.setLevel(b, Level.INFO);
Loggers loggers = new TestLoggers(root, a, b);
LoggerLevel expectedLevel = new LoggerLevel(Level.ERROR.toString(), null);
LoggerLevel actualLevel = loggers.level("a");
assertEquals(expectedLevel, actualLevel);
public void testLevelWithExistLoggerName() {
loggers.setLevel("foo", DEBUG);
assertEquals(new LoggerLevel(DEBUG.name(), INITIAL_TIME),
loggers.level("foo")
);
}
@Test
public void testGetUnknownLogger() {
LoggerContext loggerContext = (LoggerContext) LogManager.getContext(false);
Logger root = loggerContext.getRootLogger();
Configurator.setLevel(root, Level.ERROR);
public void testLevelWithNonExistLoggerName() {
assertNull(loggers.level("another-dummy"), "Unconfigured logger should return null");
}
Logger a = loggerContext.getLogger("a");
Configurator.setLevel(a, null);
@Test
public void testLevelWithNewlyCreatedLogger() {
loggers.setLevel("dummy", ERROR);
assertEquals(
new LoggerLevel(ERROR.name(), time.milliseconds()),
loggers.level("dummy"),
"Newly created logger should have the level we just set"
);
}
Logger b = loggerContext.getLogger("b");
Configurator.setLevel(b, Level.INFO);
@Test
public void testAllLevelsAfterCreatingNewLogger() {
loggers.setLevel("foo", WARN);
loggers.setLevel("bar", ERROR);
Map<String, LoggerLevel> loggerToLevel = loggers.allLevels();
Map<String, LoggerLevel> expectedLevels = Map.of(
"foo", new LoggerLevel(WARN.name(), INITIAL_TIME),
"bar", new LoggerLevel(ERROR.name(), INITIAL_TIME)
);
Loggers loggers = new TestLoggers(root, a, b);
assertTrue(loggerToLevel.entrySet().containsAll(expectedLevels.entrySet()));
}
LoggerLevel level = loggers.level("c");
assertNull(level);
@Test
public void testSetLevelWithNullNameSpaceOrNullLevel() {
NullPointerException exception = assertThrows(NullPointerException.class, () -> loggers.setLevel(null, null));
assertEquals("Logging namespace may not be null", exception.getMessage());
exception = assertThrows(NullPointerException.class, () -> loggers.setLevel("dummy", null));
assertEquals("Level may not be null", exception.getMessage());
}
@Test
public void testSetLevelWithValidRootLoggerNames() {
loggers.setLevel("", ERROR);
List<String> setLevelResultWithRoot = loggers.setLevel("root", ERROR);
assertTrue(setLevelResultWithRoot.isEmpty(),
"Setting level with empty string ('') and 'root' should affect the same set of loggers - " +
"when setting the same level twice, second call should return empty list indicating no loggers were affected");
}
@Test
public void testSetLevel() {
LoggerContext loggerContext = (LoggerContext) LogManager.getContext(false);
Logger root = loggerContext.getRootLogger();
Configurator.setLevel(root, Level.ERROR);
Logger x = loggerContext.getLogger("a.b.c.p.X");
Logger y = loggerContext.getLogger("a.b.c.p.Y");
Logger z = loggerContext.getLogger("a.b.c.p.Z");
Logger w = loggerContext.getLogger("a.b.c.s.W");
Configurator.setLevel(x, Level.INFO);
Configurator.setLevel(y, Level.INFO);
Configurator.setLevel(z, Level.INFO);
Configurator.setLevel(w, Level.INFO);
// We don't explicitly register a logger for a.b.c.p, so it won't appear in the list of current loggers;
// one should be created by the Loggers instance when we set the level
TestLoggers loggers = new TestLoggers(root, x, y, z, w);
List<String> modified = loggers.setLevel("a.b.c.p", Level.DEBUG);
assertEquals(Arrays.asList("a.b.c.p", "a.b.c.p.X", "a.b.c.p.Y", "a.b.c.p.Z"), modified);
assertEquals(Level.DEBUG.toString(), loggers.level("a.b.c.p").level());
assertEquals(Level.DEBUG, x.getLevel());
assertEquals(Level.DEBUG, y.getLevel());
assertEquals(Level.DEBUG, z.getLevel());
LoggerLevel expectedLevel = new LoggerLevel(Level.DEBUG.toString(), INITIAL_TIME);
LoggerLevel actualLevel = loggers.level("a.b.c.p");
assertEquals(expectedLevel, actualLevel);
// Sleep a little and adjust the level of a leaf logger
time.sleep(10);
loggers.setLevel("a.b.c.p.X", Level.ERROR);
expectedLevel = new LoggerLevel(Level.ERROR.toString(), INITIAL_TIME + 10);
actualLevel = loggers.level("a.b.c.p.X");
assertEquals(expectedLevel, actualLevel);
// Make sure that the direct parent logger and a sibling logger remain unaffected
expectedLevel = new LoggerLevel(Level.DEBUG.toString(), INITIAL_TIME);
actualLevel = loggers.level("a.b.c.p");
assertEquals(expectedLevel, actualLevel);
expectedLevel = new LoggerLevel(Level.DEBUG.toString(), INITIAL_TIME);
actualLevel = loggers.level("a.b.c.p.Y");
assertEquals(expectedLevel, actualLevel);
// Set the same level again, and verify that the last modified time hasn't been altered
time.sleep(10);
loggers.setLevel("a.b.c.p.X", Level.ERROR);
expectedLevel = new LoggerLevel(Level.ERROR.toString(), INITIAL_TIME + 10);
actualLevel = loggers.level("a.b.c.p.X");
assertEquals(expectedLevel, actualLevel);
loggers.setLevel("a.b.c", DEBUG);
loggers.setLevel("a.b", ERROR);
loggers.setLevel("a", WARN);
Map<String, LoggerLevel> expected = Map.of(
"a", new LoggerLevel(WARN.name(), INITIAL_TIME),
"a.b", new LoggerLevel(WARN.name(), INITIAL_TIME),
"a.b.c", new LoggerLevel(WARN.name(), INITIAL_TIME)
);
assertTrue(loggers.allLevels().entrySet().containsAll(expected.entrySet()));
}
@Test
public void testSetRootLevel() {
// In this test case, we focus on setting the level for the root logger.
// Ideally, we want to start with a "clean" configuration to conduct this test case.
// By programmatically creating a new configuration at the beginning, we can ensure
// that this test case is not affected by existing Log4j configurations.
LoggerContext loggerContext = (LoggerContext) LogManager.getContext(false);
Configuration config = loggerContext.getConfiguration();
String rootLoggerName = "root";
LoggerConfig rootConfig = new LoggerConfig(rootLoggerName, Level.ERROR, false);
config.addLogger(rootLoggerName, rootConfig);
loggerContext.updateLoggers();
Logger root = LogManager.getLogger(rootLoggerName);
Configurator.setLevel(root, Level.ERROR);
Logger p = loggerContext.getLogger("a.b.c.p");
Logger x = loggerContext.getLogger("a.b.c.p.X");
Logger y = loggerContext.getLogger("a.b.c.p.Y");
Logger z = loggerContext.getLogger("a.b.c.p.Z");
Logger w = loggerContext.getLogger("a.b.c.s.W");
Configurator.setLevel(p, Level.INFO);
Configurator.setLevel(x, Level.INFO);
Configurator.setLevel(y, Level.INFO);
Configurator.setLevel(z, Level.INFO);
Configurator.setLevel(w, Level.INFO);
Loggers loggers = new TestLoggers(root, x, y, z, w);
List<String> modified = loggers.setLevel(rootLoggerName, Level.DEBUG);
assertEquals(Arrays.asList("a.b.c.p.X", "a.b.c.p.Y", "a.b.c.p.Z", "a.b.c.s.W", rootLoggerName), modified);
assertEquals(p.getLevel(), Level.INFO);
assertEquals(root.getLevel(), Level.DEBUG);
assertEquals(w.getLevel(), Level.DEBUG);
assertEquals(x.getLevel(), Level.DEBUG);
assertEquals(y.getLevel(), Level.DEBUG);
assertEquals(z.getLevel(), Level.DEBUG);
Map<String, LoggerLevel> expectedLevels = new HashMap<>();
expectedLevels.put(rootLoggerName, new LoggerLevel(Level.DEBUG.toString(), INITIAL_TIME));
expectedLevels.put("a.b.c.p.X", new LoggerLevel(Level.DEBUG.toString(), INITIAL_TIME));
expectedLevels.put("a.b.c.p.Y", new LoggerLevel(Level.DEBUG.toString(), INITIAL_TIME));
expectedLevels.put("a.b.c.p.Z", new LoggerLevel(Level.DEBUG.toString(), INITIAL_TIME));
expectedLevels.put("a.b.c.s.W", new LoggerLevel(Level.DEBUG.toString(), INITIAL_TIME));
Map<String, LoggerLevel> actualLevels = loggers.allLevels();
assertEquals(expectedLevels, actualLevels);
public void testLookupLoggerAfterCreatingNewLogger() {
loggers.setLevel("dummy", INFO);
Logger logger = loggers.lookupLogger("dummy");
assertNotNull(logger);
assertEquals(INFO, logger.getLevel());
}
@Test
public void testSetLevelNullArguments() {
LoggerContext loggerContext = (LoggerContext) LogManager.getContext(false);
Logger root = loggerContext.getRootLogger();
Loggers loggers = new TestLoggers(root);
assertThrows(NullPointerException.class, () -> loggers.setLevel(null, Level.INFO));
assertThrows(NullPointerException.class, () -> loggers.setLevel("root", null));
public void testSetLevelWithSameLevel() {
String loggerName = "dummy";
loggers.setLevel(loggerName, DEBUG);
time.sleep(100);
loggers.setLevel(loggerName, DEBUG);
assertEquals(
new LoggerLevel(DEBUG.name(), INITIAL_TIME),
loggers.allLevels().get(loggerName),
"Setting same log level should not update the lastModified timestamp"
);
}
private class TestLoggers extends Loggers {
private final Logger rootLogger;
private final Map<String, Logger> currentLoggers;
public TestLoggers(Logger rootLogger, Logger... knownLoggers) {
super(time);
this.rootLogger = rootLogger;
this.currentLoggers = new HashMap<>(Stream.of(knownLoggers)
.collect(Collectors.toMap(
Logger::getName,
Function.identity()
)));
@Test
public void testSetLevelWithDifferentLevels() {
String loggerName = "dummy";
loggers.setLevel(loggerName, DEBUG);
time.sleep(100);
loggers.setLevel(loggerName, WARN);
assertEquals(
new LoggerLevel(WARN.name(), INITIAL_TIME + 100),
loggers.allLevels().get(loggerName),
"Setting different log level should update the lastModified timestamp"
);
}
@Override
Logger lookupLogger(String logger) {
return currentLoggers.computeIfAbsent(logger, LogManager::getLogger);
}
@Test
public void testLookupLoggerWithValidRootLoggerNames() {
assertEquals(
loggers.lookupLogger("root"),
loggers.lookupLogger(""),
"Both 'root' and empty string should retrieve the root logger"
);
@Override
List<Logger> currentLoggers() {
return new ArrayList<>(currentLoggers.values());
}
@Override
Logger rootLogger() {
return rootLogger;
}
assertEquals(
loggers.lookupLogger(""),
loggers.rootLogger(),
"Empty string lookup should match direct root logger access"
);
}
}

View File

@ -0,0 +1,268 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.connect.runtime;
import org.apache.kafka.common.utils.MockTime;
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.connect.runtime.rest.entities.LoggerLevel;
import org.apache.logging.log4j.Level;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import org.apache.logging.log4j.core.LoggerContext;
import org.apache.logging.log4j.core.config.Configuration;
import org.apache.logging.log4j.core.config.Configurator;
import org.apache.logging.log4j.core.config.LoggerConfig;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.extension.ExtendWith;
import org.mockito.junit.jupiter.MockitoExtension;
import org.mockito.junit.jupiter.MockitoSettings;
import org.mockito.quality.Strictness;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.function.Function;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertNull;
import static org.junit.jupiter.api.Assertions.assertThrows;
@ExtendWith(MockitoExtension.class)
@MockitoSettings(strictness = Strictness.STRICT_STUBS)
public class MockLoggersTest {
private static final long INITIAL_TIME = 1696951712135L;
private final LoggerContext context = (LoggerContext) LogManager.getContext(false);
private Time time;
@BeforeEach
public void setup() {
time = new MockTime(0, INITIAL_TIME, 0);
}
@Test
public void testGetLoggersIgnoresNullLevels() {
LoggerContext loggerContext = (LoggerContext) LogManager.getContext(false);
Logger root = loggerContext.getRootLogger();
Configurator.setLevel(root, Level.OFF);
Logger a = loggerContext.getLogger("a");
Configurator.setLevel(a, null);
Logger b = loggerContext.getLogger("b");
Configurator.setLevel(b, Level.INFO);
Loggers loggers = new TestLoggers(root, a, b);
Map<String, LoggerLevel> expectedLevels = Collections.singletonMap(
"b",
new LoggerLevel(Level.INFO.toString(), null)
);
Map<String, LoggerLevel> actualLevels = loggers.allLevels();
assertEquals(expectedLevels, actualLevels);
}
@Test
public void testGetLoggerFallsBackToEffectiveLogLevel() {
LoggerContext loggerContext = (LoggerContext) LogManager.getContext(false);
Logger root = loggerContext.getRootLogger();
Configurator.setLevel(root, Level.ERROR);
Logger a = loggerContext.getLogger("a");
Configurator.setLevel(a, null);
Logger b = loggerContext.getLogger("b");
Configurator.setLevel(b, Level.INFO);
Loggers loggers = new TestLoggers(root, a, b);
LoggerLevel expectedLevel = new LoggerLevel(Level.ERROR.toString(), null);
LoggerLevel actualLevel = loggers.level("a");
assertEquals(expectedLevel, actualLevel);
}
@Test
public void testGetUnknownLogger() {
LoggerContext loggerContext = (LoggerContext) LogManager.getContext(false);
Logger root = loggerContext.getRootLogger();
Configurator.setLevel(root, Level.ERROR);
Logger a = loggerContext.getLogger("a");
Configurator.setLevel(a, null);
Logger b = loggerContext.getLogger("b");
Configurator.setLevel(b, Level.INFO);
Loggers loggers = new TestLoggers(root, a, b);
LoggerLevel level = loggers.level("c");
assertNull(level);
}
@Test
public void testSetLevel() {
LoggerContext loggerContext = (LoggerContext) LogManager.getContext(false);
Logger root = loggerContext.getRootLogger();
Configurator.setLevel(root, Level.ERROR);
Logger x = loggerContext.getLogger("a.b.c.p.X");
Logger y = loggerContext.getLogger("a.b.c.p.Y");
Logger z = loggerContext.getLogger("a.b.c.p.Z");
Logger w = loggerContext.getLogger("a.b.c.s.W");
Configurator.setLevel(x, Level.INFO);
Configurator.setLevel(y, Level.INFO);
Configurator.setLevel(z, Level.INFO);
Configurator.setLevel(w, Level.INFO);
// We don't explicitly register a logger for a.b.c.p, so it won't appear in the list of current loggers;
// one should be created by the Loggers instance when we set the level
TestLoggers loggers = new TestLoggers(root, x, y, z, w);
List<String> modified = loggers.setLevel("a.b.c.p", Level.WARN);
assertEquals(Arrays.asList("a.b.c.p", "a.b.c.p.X", "a.b.c.p.Y", "a.b.c.p.Z"), modified);
assertEquals(Level.WARN.toString(), loggers.level("a.b.c.p").level());
assertEquals(Level.WARN, x.getLevel());
assertEquals(Level.WARN, y.getLevel());
assertEquals(Level.WARN, z.getLevel());
LoggerLevel expectedLevel = new LoggerLevel(Level.WARN.toString(), INITIAL_TIME);
LoggerLevel actualLevel = loggers.level("a.b.c.p");
assertEquals(expectedLevel, actualLevel);
// Sleep a little and adjust the level of a leaf logger
time.sleep(10);
loggers.setLevel("a.b.c.p.X", Level.ERROR);
expectedLevel = new LoggerLevel(Level.ERROR.toString(), INITIAL_TIME + 10);
actualLevel = loggers.level("a.b.c.p.X");
assertEquals(expectedLevel, actualLevel);
// Make sure that the direct parent logger and a sibling logger remain unaffected
expectedLevel = new LoggerLevel(Level.WARN.toString(), INITIAL_TIME);
actualLevel = loggers.level("a.b.c.p");
assertEquals(expectedLevel, actualLevel);
expectedLevel = new LoggerLevel(Level.WARN.toString(), INITIAL_TIME);
actualLevel = loggers.level("a.b.c.p.Y");
assertEquals(expectedLevel, actualLevel);
// Set the same level again, and verify that the last modified time hasn't been altered
time.sleep(10);
loggers.setLevel("a.b.c.p.X", Level.ERROR);
expectedLevel = new LoggerLevel(Level.ERROR.toString(), INITIAL_TIME + 10);
actualLevel = loggers.level("a.b.c.p.X");
assertEquals(expectedLevel, actualLevel);
}
@Test
public void testSetRootLevel() {
// In this test case, we focus on setting the level for the root logger.
// Ideally, we want to start with a "clean" configuration to conduct this test case.
// By programmatically creating a new configuration at the beginning, we can ensure
// that this test case is not affected by existing Log4j configurations.
LoggerContext loggerContext = (LoggerContext) LogManager.getContext(false);
Configuration config = loggerContext.getConfiguration();
String rootLoggerName = "root";
LoggerConfig rootConfig = new LoggerConfig(rootLoggerName, Level.ERROR, false);
config.addLogger(rootLoggerName, rootConfig);
loggerContext.updateLoggers();
Logger root = LogManager.getLogger(rootLoggerName);
Configurator.setLevel(root, Level.ERROR);
Logger p = loggerContext.getLogger("a.b.c.p");
Logger x = loggerContext.getLogger("a.b.c.p.X");
Logger y = loggerContext.getLogger("a.b.c.p.Y");
Logger z = loggerContext.getLogger("a.b.c.p.Z");
Logger w = loggerContext.getLogger("a.b.c.s.W");
Configurator.setLevel(p, Level.INFO);
Configurator.setLevel(x, Level.INFO);
Configurator.setLevel(y, Level.INFO);
Configurator.setLevel(z, Level.INFO);
Configurator.setLevel(w, Level.INFO);
Loggers loggers = new TestLoggers(root, x, y, z, w);
List<String> modified = loggers.setLevel(rootLoggerName, Level.DEBUG);
assertEquals(Arrays.asList("a.b.c.p.X", "a.b.c.p.Y", "a.b.c.p.Z", "a.b.c.s.W", rootLoggerName), modified);
assertEquals(Level.DEBUG, p.getLevel());
assertEquals(Level.DEBUG, root.getLevel());
assertEquals(Level.DEBUG, w.getLevel());
assertEquals(Level.DEBUG, x.getLevel());
assertEquals(Level.DEBUG, y.getLevel());
assertEquals(Level.DEBUG, z.getLevel());
Map<String, LoggerLevel> expectedLevels = Map.of(
"a.b.c.p.X", new LoggerLevel(Level.DEBUG.toString(), INITIAL_TIME),
"a.b.c.p.Y", new LoggerLevel(Level.DEBUG.toString(), INITIAL_TIME),
"a.b.c.p.Z", new LoggerLevel(Level.DEBUG.toString(), INITIAL_TIME),
"a.b.c.s.W", new LoggerLevel(Level.DEBUG.toString(), INITIAL_TIME),
rootLoggerName, new LoggerLevel(Level.DEBUG.toString(), INITIAL_TIME)
);
Map<String, LoggerLevel> actualLevels = loggers.allLevels();
assertEquals(expectedLevels, actualLevels);
}
@Test
public void testSetLevelNullArguments() {
LoggerContext loggerContext = (LoggerContext) LogManager.getContext(false);
Logger root = loggerContext.getRootLogger();
Loggers loggers = new TestLoggers(root);
assertThrows(NullPointerException.class, () -> loggers.setLevel(null, Level.INFO));
assertThrows(NullPointerException.class, () -> loggers.setLevel("root", null));
}
private class TestLoggers extends Loggers {
private final Logger rootLogger;
private final Map<String, Logger> currentLoggers;
public TestLoggers(Logger rootLogger, Logger... knownLoggers) {
super(time);
this.rootLogger = rootLogger;
this.currentLoggers = new HashMap<>(Stream.of(knownLoggers)
.collect(Collectors.toMap(
Logger::getName,
Function.identity()
)));
this.currentLoggers.put(rootLogger.getName(), rootLogger);
}
@Override
Logger lookupLogger(String logger) {
return currentLoggers.computeIfAbsent(logger, LogManager::getLogger);
}
@Override
Map<String, Logger> currentLoggers() {
return currentLoggers;
}
@Override
Logger rootLogger() {
return rootLogger;
}
}
}

View File

@ -563,7 +563,16 @@ class ConnectDistributedTest(Test):
# have been discarded
self._restart_worker(worker)
restarted_loggers = self.cc.get_all_loggers(worker)
assert initial_loggers == restarted_loggers
for loggerName in restarted_loggers:
logger = self.cc.get_logger(worker, loggerName)
level = logger['level']
# ConsumerConfig logger is pre-defined in log4j2 config with ERROR level,
# while other loggers should be set to DEBUG level
if loggerName == 'org.apache.kafka.clients.consumer.ConsumerConfig':
assert level == 'ERROR'
else:
assert level == 'DEBUG'
def _different_level(self, current_level):
return 'INFO' if current_level is None or current_level.upper() != 'INFO' else 'WARN'