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 <chia7712@gmail.com>
This commit is contained in:
Chung, Ming-Yen 2024-07-31 02:37:36 +08:00 committed by GitHub
parent 1084d3b9c9
commit 7c0a96d08d
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
18 changed files with 18 additions and 18 deletions

View File

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

View File

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

View File

@ -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";

View File

@ -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);

View File

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

View File

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

View File

@ -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);
}

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -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";

View File

@ -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";

View File

@ -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 */

View File

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

View File

@ -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};