This is an automated email from the ASF dual-hosted git repository.

chia7712 pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
     new 7c0a96d08d8 KAFKA-17185 Declare Loggers as static to prevent multiple 
logger instances (#16680)
7c0a96d08d8 is described below

commit 7c0a96d08d81c2f91c10c601f1e0fd6f4f5f6268
Author: Chung, Ming-Yen <[email protected]>
AuthorDate: Wed Jul 31 02:37:36 2024 +0800

    KAFKA-17185 Declare Loggers as static to prevent multiple logger instances 
(#16680)
    
    As discussed in #16657 (comment) , we should make logger as static to avoid 
creating multiple logger instances.
    I use the regex private.*Logger.*LoggerFactory to search and check all the 
results if certain logs need to be static.
    
    There are some exceptions that loggers don't need to be static:
    1) The logger in the inner class. Since java8 doesn't support static field 
in the inner class.
            
https://github.com/apache/kafka/blob/trunk/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetchRequestManagerTest.java#L3676
    
    2) Custom loggers for each instance (non-static + non-final). In this case, 
multiple logger instances is actually really needed.
            
https://github.com/apache/kafka/blob/trunk/storage/src/test/java/org/apache/kafka/server/log/remote/storage/LocalTieredStorage.java#L166
    
    3) The logger is initialized in constructor by LogContext. Many non-static 
but with final modifier loggers are in this category, that's why I use 
.*LoggerFactory to only check the loggers that are assigned initial value when 
declaration.
    
    4) protected final Logger log = Logger.getLogger(getClass())
        This is for subclass can do logging with subclass name instead of 
superclass name.
        But in this case, if the log access modifier is private, the purpose 
cannot be achieved since subclass cannot access the log defined in superclass. 
So if access modifier is private, we can replace getClass() with 
<className>.class
    
    Reviewers: Chia-Ping Tsai <[email protected]>
---
 .../src/main/java/org/apache/kafka/common/config/AbstractConfig.java    | 2 +-
 .../main/java/org/apache/kafka/common/metrics/stats/Percentiles.java    | 2 +-
 .../internals/unsecured/OAuthBearerUnsecuredLoginCallbackHandler.java   | 2 +-
 clients/src/main/java/org/apache/kafka/common/utils/KafkaThread.java    | 2 +-
 .../java/org/apache/kafka/connect/runtime/WorkerSinkTaskContext.java    | 2 +-
 metadata/src/test/java/org/apache/kafka/metalog/LocalLogManager.java    | 2 +-
 .../src/main/java/org/apache/kafka/server/util/TopicFilter.java         | 2 +-
 .../kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java  | 2 +-
 .../apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java  | 2 +-
 .../apache/kafka/streams/kstream/internals/graph/StreamSourceNode.java  | 2 +-
 .../org/apache/kafka/streams/processor/internals/ProcessorNode.java     | 2 +-
 .../org/apache/kafka/streams/processor/internals/ProcessorTopology.java | 2 +-
 .../org/apache/kafka/streams/state/internals/WindowStoreBuilder.java    | 2 +-
 .../kafka/streams/processor/internals/RepartitionOptimizingTest.java    | 2 +-
 .../streams/processor/internals/RepartitionWithMergeOptimizingTest.java | 2 +-
 .../kafka/streams/processor/internals/StreamsAssignmentScaleTest.java   | 2 +-
 .../test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java    | 2 +-
 .../org/apache/kafka/trogdor/workload/TimestampRecordProcessor.java     | 2 +-
 18 files changed, 18 insertions(+), 18 deletions(-)

diff --git 
a/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java 
b/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java
index 247bab18d55..72038b4564c 100644
--- a/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java
+++ b/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java
@@ -45,7 +45,7 @@ import java.util.stream.Collectors;
  */
 public class AbstractConfig {
 
-    private final Logger log = LoggerFactory.getLogger(getClass());
+    private static final Logger log = 
LoggerFactory.getLogger(AbstractConfig.class);
 
     /**
      * Configs for which values have been requested, used to detect unused 
configs.
diff --git 
a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Percentiles.java 
b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Percentiles.java
index c7c882496bf..5cb1da4f653 100644
--- 
a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Percentiles.java
+++ 
b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Percentiles.java
@@ -33,7 +33,7 @@ import java.util.List;
  */
 public class Percentiles extends SampledStat implements CompoundStat {
 
-    private final Logger log = LoggerFactory.getLogger(Percentiles.class);
+    private static final Logger log = 
LoggerFactory.getLogger(Percentiles.class);
 
     public enum BucketSizing {
         CONSTANT, LINEAR
diff --git 
a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/unsecured/OAuthBearerUnsecuredLoginCallbackHandler.java
 
b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/unsecured/OAuthBearerUnsecuredLoginCallbackHandler.java
index daae977d727..2241eb50fdc 100644
--- 
a/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/unsecured/OAuthBearerUnsecuredLoginCallbackHandler.java
+++ 
b/clients/src/main/java/org/apache/kafka/common/security/oauthbearer/internals/unsecured/OAuthBearerUnsecuredLoginCallbackHandler.java
@@ -99,7 +99,7 @@ import javax.security.sasl.SaslException;
  * broker configuration property.
  */
 public class OAuthBearerUnsecuredLoginCallbackHandler implements 
AuthenticateCallbackHandler {
-    private final Logger log = 
LoggerFactory.getLogger(OAuthBearerUnsecuredLoginCallbackHandler.class);
+    private static final Logger log = 
LoggerFactory.getLogger(OAuthBearerUnsecuredLoginCallbackHandler.class);
     private static final String OPTION_PREFIX = "unsecuredLogin";
     private static final String PRINCIPAL_CLAIM_NAME_OPTION = OPTION_PREFIX + 
"PrincipalClaimName";
     private static final String LIFETIME_SECONDS_OPTION = OPTION_PREFIX + 
"LifetimeSeconds";
diff --git 
a/clients/src/main/java/org/apache/kafka/common/utils/KafkaThread.java 
b/clients/src/main/java/org/apache/kafka/common/utils/KafkaThread.java
index 2f5e15f4634..a2ea3dc88b6 100644
--- a/clients/src/main/java/org/apache/kafka/common/utils/KafkaThread.java
+++ b/clients/src/main/java/org/apache/kafka/common/utils/KafkaThread.java
@@ -24,7 +24,7 @@ import org.slf4j.LoggerFactory;
  */
 public class KafkaThread extends Thread {
 
-    private final Logger log = LoggerFactory.getLogger(getClass());
+    private static final Logger log = 
LoggerFactory.getLogger(KafkaThread.class);
     
     public static KafkaThread daemon(final String name, Runnable runnable) {
         return new KafkaThread(name, runnable, true);
diff --git 
a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSinkTaskContext.java
 
b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSinkTaskContext.java
index 3baf4772b5b..e767c7640b4 100644
--- 
a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSinkTaskContext.java
+++ 
b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSinkTaskContext.java
@@ -35,7 +35,7 @@ import java.util.Set;
 
 public class WorkerSinkTaskContext implements SinkTaskContext {
 
-    private final Logger log = LoggerFactory.getLogger(getClass());
+    private static final Logger log = 
LoggerFactory.getLogger(WorkerSinkTaskContext.class);
     private final Map<TopicPartition, Long> offsets;
     private final Consumer<byte[], byte[]> consumer;
     private final WorkerSinkTask sinkTask;
diff --git 
a/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManager.java 
b/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManager.java
index ec733b3ca19..52a5792ced1 100644
--- a/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManager.java
+++ b/metadata/src/test/java/org/apache/kafka/metalog/LocalLogManager.java
@@ -163,7 +163,7 @@ public final class LocalLogManager implements 
RaftClient<ApiMessageAndVersion>,
     }
 
     public static class SharedLogData {
-        private final Logger log = 
LoggerFactory.getLogger(SharedLogData.class);
+        private static final Logger log = 
LoggerFactory.getLogger(SharedLogData.class);
 
         /**
          * Maps node IDs to the matching log managers.
diff --git 
a/server-common/src/main/java/org/apache/kafka/server/util/TopicFilter.java 
b/server-common/src/main/java/org/apache/kafka/server/util/TopicFilter.java
index 1dbebdfdecc..64c8f8441b9 100644
--- a/server-common/src/main/java/org/apache/kafka/server/util/TopicFilter.java
+++ b/server-common/src/main/java/org/apache/kafka/server/util/TopicFilter.java
@@ -54,7 +54,7 @@ public abstract class TopicFilter {
     }
 
     public static class IncludeList extends TopicFilter {
-        private final Logger log = LoggerFactory.getLogger(IncludeList.class);
+        private static final Logger log = 
LoggerFactory.getLogger(IncludeList.class);
         public IncludeList(String rawRegex) {
             super(rawRegex);
         }
diff --git 
a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
 
b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
index 6b5a1753c8b..4a288bb0e83 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamSlidingWindowAggregate.java
@@ -43,7 +43,7 @@ import static 
org.apache.kafka.streams.state.ValueAndTimestamp.getValueOrNull;
 
 public class KStreamSlidingWindowAggregate<KIn, VIn, VAgg> implements 
KStreamAggProcessorSupplier<KIn, VIn, Windowed<KIn>, VAgg> {
 
-    private final Logger log = LoggerFactory.getLogger(getClass());
+    private static final Logger log = 
LoggerFactory.getLogger(KStreamSlidingWindowAggregate.class);
 
     private final String storeName;
     private final SlidingWindows windows;
diff --git 
a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java
 
b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java
index 44169cf0eb2..340ce82d856 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java
@@ -41,7 +41,7 @@ import static 
org.apache.kafka.streams.state.ValueAndTimestamp.getValueOrNull;
 
 public class KStreamWindowAggregate<KIn, VIn, VAgg, W extends Window> 
implements KStreamAggProcessorSupplier<KIn, VIn, Windowed<KIn>, VAgg> {
 
-    private final Logger log = LoggerFactory.getLogger(getClass());
+    private static final Logger log = 
LoggerFactory.getLogger(KStreamWindowAggregate.class);
 
     private final String storeName;
     private final Windows<W> windows;
diff --git 
a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StreamSourceNode.java
 
b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StreamSourceNode.java
index 003298cc870..97b686eaff6 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StreamSourceNode.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/graph/StreamSourceNode.java
@@ -30,7 +30,7 @@ import java.util.regex.Pattern;
 
 public class StreamSourceNode<K, V> extends SourceGraphNode<K, V> {
 
-    private final Logger log = LoggerFactory.getLogger(StreamSourceNode.class);
+    private static final Logger log = 
LoggerFactory.getLogger(StreamSourceNode.class);
 
     public StreamSourceNode(final String nodeName,
                             final Collection<String> topicNames,
diff --git 
a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java
 
b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java
index 763edc9a045..ec94ef64298 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java
@@ -45,7 +45,7 @@ import static 
org.apache.kafka.streams.StreamsConfig.PROCESSING_EXCEPTION_HANDLE
 
 public class ProcessorNode<KIn, VIn, KOut, VOut> {
 
-    private final Logger log = LoggerFactory.getLogger(ProcessorNode.class);
+    private static final Logger log = 
LoggerFactory.getLogger(ProcessorNode.class);
     private final List<ProcessorNode<KOut, VOut, ?, ?>> children;
     private final Map<String, ProcessorNode<KOut, VOut, ?, ?>> childByName;
 
diff --git 
a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorTopology.java
 
b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorTopology.java
index b65bba4e16e..b0fc1f68511 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorTopology.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorTopology.java
@@ -31,7 +31,7 @@ import java.util.Optional;
 import java.util.Set;
 
 public class ProcessorTopology {
-    private final Logger log = 
LoggerFactory.getLogger(ProcessorTopology.class);
+    private static final Logger log = 
LoggerFactory.getLogger(ProcessorTopology.class);
 
     private final List<ProcessorNode<?, ?, ?, ?>> processorNodes;
     private final Map<String, SourceNode<?, ?>> sourceNodesByName;
diff --git 
a/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowStoreBuilder.java
 
b/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowStoreBuilder.java
index 10f75f2bd06..d57d5318c68 100644
--- 
a/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowStoreBuilder.java
+++ 
b/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowStoreBuilder.java
@@ -29,7 +29,7 @@ import org.slf4j.LoggerFactory;
 import java.util.Objects;
 
 public class WindowStoreBuilder<K, V> extends AbstractStoreBuilder<K, V, 
WindowStore<K, V>> {
-    private final Logger log = 
LoggerFactory.getLogger(WindowStoreBuilder.class);
+    private static final Logger log = 
LoggerFactory.getLogger(WindowStoreBuilder.class);
 
     private final WindowBytesStoreSupplier storeSupplier;
 
diff --git 
a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RepartitionOptimizingTest.java
 
b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RepartitionOptimizingTest.java
index cac44dbb5e9..91b95b87a7f 100644
--- 
a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RepartitionOptimizingTest.java
+++ 
b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RepartitionOptimizingTest.java
@@ -72,7 +72,7 @@ import static org.junit.jupiter.api.Assertions.assertEquals;
 @SuppressWarnings("deprecation")
 public class RepartitionOptimizingTest {
 
-    private final Logger log = 
LoggerFactory.getLogger(RepartitionOptimizingTest.class);
+    private static final Logger log = 
LoggerFactory.getLogger(RepartitionOptimizingTest.class);
 
     private static final String INPUT_TOPIC = "input";
     private static final String COUNT_TOPIC = "outputTopic_0";
diff --git 
a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RepartitionWithMergeOptimizingTest.java
 
b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RepartitionWithMergeOptimizingTest.java
index f6a12d6477b..a9287fc3e59 100644
--- 
a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RepartitionWithMergeOptimizingTest.java
+++ 
b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RepartitionWithMergeOptimizingTest.java
@@ -60,7 +60,7 @@ import static org.junit.jupiter.api.Assertions.assertEquals;
 
 public class RepartitionWithMergeOptimizingTest {
 
-    private final Logger log = 
LoggerFactory.getLogger(RepartitionWithMergeOptimizingTest.class);
+    private static final Logger log = 
LoggerFactory.getLogger(RepartitionWithMergeOptimizingTest.class);
 
     private static final String INPUT_A_TOPIC = "inputA";
     private static final String INPUT_B_TOPIC = "inputB";
diff --git 
a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsAssignmentScaleTest.java
 
b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsAssignmentScaleTest.java
index d61c530c92e..b195600c392 100644
--- 
a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsAssignmentScaleTest.java
+++ 
b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsAssignmentScaleTest.java
@@ -82,7 +82,7 @@ public class StreamsAssignmentScaleTest {
     static final long MAX_ASSIGNMENT_DURATION = 120 * 1000L; // we should stay 
below `max.poll.interval.ms`
     static final String APPLICATION_ID = "streams-assignment-scale-test";
 
-    private final Logger log = 
LoggerFactory.getLogger(StreamsAssignmentScaleTest.class);
+    private static final Logger log = 
LoggerFactory.getLogger(StreamsAssignmentScaleTest.class);
 
     /* HighAvailabilityTaskAssignor tests */
 
diff --git 
a/streams/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java 
b/streams/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java
index 6e216dcdbca..64cf3c7eb0b 100644
--- 
a/streams/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java
+++ 
b/streams/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java
@@ -154,7 +154,7 @@ public class StreamsUpgradeTest {
 
     public static class FutureStreamsPartitionAssignor extends 
StreamsPartitionAssignor {
         private static final Map<String, String> CLIENT_TAGS = 
mkMap(mkEntry("t1", "v1"), mkEntry("t2", "v2"));
-        private final Logger log = 
LoggerFactory.getLogger(FutureStreamsPartitionAssignor.class);
+        private static final Logger log = 
LoggerFactory.getLogger(FutureStreamsPartitionAssignor.class);
 
         private AtomicInteger usedSubscriptionMetadataVersionPeek;
         private AtomicLong nextScheduledRebalanceMs;
diff --git 
a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/TimestampRecordProcessor.java
 
b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/TimestampRecordProcessor.java
index 529e228c38c..19d1d32d174 100644
--- 
a/trogdor/src/main/java/org/apache/kafka/trogdor/workload/TimestampRecordProcessor.java
+++ 
b/trogdor/src/main/java/org/apache/kafka/trogdor/workload/TimestampRecordProcessor.java
@@ -55,7 +55,7 @@ public class TimestampRecordProcessor implements 
RecordProcessor {
     private final ByteBuffer buffer;
     private final Histogram histogram;
 
-    private final Logger log = 
LoggerFactory.getLogger(TimestampRecordProcessor.class);
+    private static final Logger log = 
LoggerFactory.getLogger(TimestampRecordProcessor.class);
 
     static final float[] PERCENTILES = {0.5f, 0.95f, 0.99f};
 

Reply via email to