mimaison commented on code in PR #18496:
URL: https://github.com/apache/kafka/pull/18496#discussion_r1930547606


##########
connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Loggers.java:
##########
@@ -52,184 +53,274 @@ public class Loggers {
     /**
      * 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.
-     *
+     * <p>
      * 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.
+     * </p>
      */
     private static final List<String> VALID_ROOT_LOGGER_NAMES = 
List.of(LogManager.ROOT_LOGGER_NAME, ROOT_LOGGER_NAME);
 
-    private final Time time;
+    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;
+    final Map<String, Long> lastModifiedTimes;
 
-    public Loggers(Time time) {
+    /**
+     * Creates a {@link Loggers} instance appropriate for the current 
environment.
+     *
+     * @param time A time source.
+     * @return A new {@link Loggers} instance, never {@link null}.
+     */
+    public static Loggers newInstance(Time time) {
+        Objects.requireNonNull(time);
+        try {
+            return new Log4jLoggers(time);
+        } catch (ClassCastException | LinkageError e) {
+            log.info("No supported logging implementation found. Logging 
configuration endpoint will be disabled.");
+            return new NoOpLoggers(time);
+        }
+    }
+
+    private Loggers(Time time) {
         this.time = time;
-        this.lastModifiedTimes = new HashMap<>();
+        this.lastModifiedTimes = new ConcurrentHashMap<>();
     }
 
     /**
      * Retrieve the current level for a single logger.
-     * @param logger the name of the logger to retrieve the level for; may not 
be null
+     *
+     * @param loggerName the name of the logger to retrieve the level for; may 
not be null
      * @return the current level (falling back on the effective level if 
necessary) of the logger,
      * or null if no logger with the specified name exists
      */
-    public synchronized LoggerLevel level(String logger) {
-        Objects.requireNonNull(logger, "Logger may not be null");
-
-        org.apache.logging.log4j.Logger foundLogger = null;
-        if (isValidRootLoggerName(logger)) {
-            foundLogger = rootLogger();
-        } else {
-            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).
-            for (org.apache.logging.log4j.Logger currentLogger : 
currentLoggers) {
-                if (logger.equals(currentLogger.getName())) {
-                    foundLogger = currentLogger;
-                    break;
-                }
-            }
-        }
-
-        if (foundLogger == null) {
-            log.warn("Unable to find level for logger {}", logger);
-            return null;
-        }
-
-        return loggerLevel(foundLogger);
-    }
+    public abstract LoggerLevel level(String loggerName);
 
     /**
      * Retrieve the current levels of all known loggers
+     *
      * @return the levels of all known loggers; may be empty, but never null
      */
-    public synchronized Map<String, LoggerLevel> allLevels() {
-        return currentLoggers()
-            .values()
-            .stream()
-            .filter(logger -> !logger.getLevel().equals(Level.OFF))
-            .collect(Collectors.toMap(
-                this::getLoggerName,
-                this::loggerLevel,
-                (existing, replacing) -> replacing,
-                TreeMap::new)
-            );
-    }
+    public abstract Map<String, LoggerLevel> allLevels();
 
     /**
      * Set the level for the specified logger and all of its children
+     *
      * @param namespace the name of the logger to adjust along with its 
children; may not be null
-     * @param level the level to set for the logger and its children; may not 
be null
+     * @param level     the level to set for the logger and its children; may 
not be null
      * @return all loggers that were affected by this action, sorted by their 
natural ordering;
      * may be empty, but never null
      */
-    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 {}", 
internalNameSpace, level);
-
-        var loggers = loggers(internalNameSpace);
-        var nameToLevel = allLevels();
-
-        List<String> result = new ArrayList<>();
-        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);
+    public abstract List<String> setLevel(String namespace, String level);
+
+    public abstract boolean isValidLevel(String level);
+
+    static class Log4jLoggers extends Loggers {
+
+        // package-private for testing
+        final LoggerContext loggerContext;
+
+        // Package-private for testing
+        Log4jLoggers(Time time) {
+            super(time);
+            loggerContext = (LoggerContext) LogManager.getContext(false);
+        }
+
+        @Override
+        public LoggerLevel level(String logger) {
+            Objects.requireNonNull(logger, "Logger may not be null");
+
+            org.apache.logging.log4j.Logger foundLogger = null;
+            if (isValidRootLoggerName(logger)) {
+                foundLogger = rootLogger();
+            } else {
+                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).
+                for (org.apache.logging.log4j.Logger currentLogger : 
currentLoggers) {
+                    if (logger.equals(currentLogger.getName())) {
+                        foundLogger = currentLogger;
+                        break;
+                    }
+                }
             }
+
+            if (foundLogger == null) {
+                log.warn("Unable to find level for logger {}", logger);
+                return null;
+            }
+
+            return loggerLevel(foundLogger);
         }
-        Collections.sort(result);
 
-        return result;
-    }
+        @Override
+        public Map<String, LoggerLevel> allLevels() {
+            return currentLoggers()
+                    .values()
+                    .stream()
+                    .filter(logger -> !logger.getLevel().equals(Level.OFF))
+                    .collect(Collectors.toMap(
+                            this::getLoggerName,
+                            this::loggerLevel,
+                            (existing, replacing) -> replacing,
+                            TreeMap::new)
+                    );
+        }
 
-    /**
-     * Retrieve all known loggers within a given namespace, creating an 
ancestor logger for that
-     * namespace if one does not already exist
-     * @param namespace the namespace that the loggers should fall under; may 
not be null
-     * @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 Collection<org.apache.logging.log4j.Logger> 
loggers(String namespace) {
-        Objects.requireNonNull(namespace, "Logging namespace may not be null");
+        @Override
+        public List<String> setLevel(String namespace, String 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 {}", 
internalNameSpace, level);
+
+            var loggers = loggers(internalNameSpace);

Review Comment:
   In Kafka we're currently sticking to explicit types. It seems you only used 
`var` here so can you update them.



-- 
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

Reply via email to