Skip to content

KAFKA-18066: Fix mismatched StreamThread ID in log messages #19517

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Open
wants to merge 5 commits into
base: trunk
Choose a base branch
from
Open
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
@@ -77,7 +77,7 @@ class ActiveTaskCreator {
final String threadId,
final int threadIdx,
final UUID processId,
final Logger log,
final LogContext logContext,
final boolean stateUpdaterEnabled,
final boolean processingThreadsEnabled) {
this.topologyMetadata = topologyMetadata;
@@ -91,15 +91,12 @@ class ActiveTaskCreator {
this.threadId = threadId;
this.threadIdx = threadIdx;
this.processId = processId;
this.log = log;
this.log = logContext.logger(getClass());
this.stateUpdaterEnabled = stateUpdaterEnabled;
this.processingThreadsEnabled = processingThreadsEnabled;

createTaskSensor = ThreadMetrics.createTaskSensor(threadId, streamsMetrics);

final String threadIdPrefix = String.format("stream-thread [%s] ", Thread.currentThread().getName());
final LogContext logContext = new LogContext(threadIdPrefix);

streamsProducer = new StreamsProducer(
producer(),
processingMode(applicationConfig),
Original file line number Diff line number Diff line change
@@ -52,20 +52,20 @@ class StandbyTaskCreator {
final StateDirectory stateDirectory,
final ChangelogReader storeChangelogReader,
final String threadId,
final Logger log,
final LogContext logContext,
final boolean stateUpdaterEnabled) {
this.topologyMetadata = topologyMetadata;
this.applicationConfig = applicationConfig;
this.streamsMetrics = streamsMetrics;
this.stateDirectory = stateDirectory;
this.storeChangelogReader = storeChangelogReader;
this.log = log;
this.log = logContext.logger(getClass());
this.stateUpdaterEnabled = stateUpdaterEnabled;

createTaskSensor = ThreadMetrics.createTaskSensor(threadId, streamsMetrics);

dummyCache = new ThreadCache(
new LogContext(String.format("stream-thread [%s] ", Thread.currentThread().getName())),
logContext,
0,
streamsMetrics
);
Original file line number Diff line number Diff line change
@@ -72,6 +72,7 @@
import org.apache.kafka.streams.state.internals.ThreadCache;

import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.time.Duration;
import java.util.Arrays;
@@ -396,15 +397,15 @@ public static StreamThread create(final TopologyMetadata topologyMetadata,
final String logPrefix = String.format("stream-thread [%s] ", threadId);
final LogContext logContext = new LogContext(logPrefix);
final LogContext restorationLogContext = stateUpdaterEnabled ? new LogContext(String.format("state-updater [%s] ", restorationThreadId)) : logContext;
final Logger log = logContext.logger(StreamThread.class);
final Logger log = LoggerFactory.getLogger(StreamThread.class);

final ReferenceContainer referenceContainer = new ReferenceContainer();
referenceContainer.adminClient = adminClient;
referenceContainer.streamsMetadataState = streamsMetadataState;
referenceContainer.time = time;
referenceContainer.clientTags = config.getClientTags();

log.info("Creating restore consumer client");
log.info("Creating restore consumer client for thread {}", threadId);
final Map<String, Object> restoreConsumerConfigs = config.getRestoreConsumerConfigs(restoreConsumerClientId(restorationThreadId));
final Consumer<byte[], byte[]> restoreConsumer = clientSupplier.getRestoreConsumer(restoreConsumerConfigs);

@@ -433,7 +434,7 @@ public static StreamThread create(final TopologyMetadata topologyMetadata,
threadId,
threadIdx,
processId,
log,
logContext,
stateUpdaterEnabled,
proceessingThreadsEnabled
);
@@ -444,10 +445,10 @@ public static StreamThread create(final TopologyMetadata topologyMetadata,
stateDirectory,
changelogReader,
threadId,
log,
logContext,
stateUpdaterEnabled);

final Tasks tasks = new Tasks(new LogContext(logPrefix));
final Tasks tasks = new Tasks(logContext);
final boolean processingThreadsEnabled =
InternalConfig.processingThreadsEnabled(config.originals());

@@ -482,7 +483,7 @@ public static StreamThread create(final TopologyMetadata topologyMetadata,
);
referenceContainer.taskManager = taskManager;

log.info("Creating consumer client");
log.info("Creating consumer client for thread {}", threadId);
final String applicationId = config.getString(StreamsConfig.APPLICATION_ID_CONFIG);
final Map<String, Object> consumerConfigs = config.getMainConsumerConfigs(applicationId, consumerClientId(threadId), threadIdx);
consumerConfigs.put(StreamsConfig.InternalConfig.REFERENCE_CONTAINER_PARTITION_ASSIGNOR, referenceContainer);
@@ -493,7 +494,7 @@ public static StreamThread create(final TopologyMetadata topologyMetadata,
consumerConfigs.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "none");
}

final MainConsumerSetup mainConsumerSetup = setupMainConsumer(topologyMetadata, config, clientSupplier, processId, log, consumerConfigs);
final MainConsumerSetup mainConsumerSetup = setupMainConsumer(topologyMetadata, config, clientSupplier, processId, log, threadId, consumerConfigs);

taskManager.setMainConsumer(mainConsumerSetup.mainConsumer);
referenceContainer.mainConsumer = mainConsumerSetup.mainConsumer;
@@ -534,6 +535,7 @@ private static MainConsumerSetup setupMainConsumer(final TopologyMetadata topolo
final KafkaClientSupplier clientSupplier,
final UUID processId,
final Logger log,
final String threadId,
final Map<String, Object> consumerConfigs) {
if (config.getString(StreamsConfig.GROUP_PROTOCOL_CONFIG).equalsIgnoreCase(GroupProtocol.STREAMS.name)) {
if (topologyMetadata.hasNamedTopologies()) {
Original file line number Diff line number Diff line change
@@ -279,7 +279,7 @@ private void createTasks() {
"clientId-StreamThread-0",
0,
uuid,
new LogContext().logger(ActiveTaskCreator.class),
new LogContext(),
false,
false);

Original file line number Diff line number Diff line change
@@ -116,7 +116,6 @@
import org.mockito.junit.jupiter.MockitoExtension;
import org.mockito.junit.jupiter.MockitoSettings;
import org.mockito.quality.Strictness;
import org.slf4j.Logger;

import java.io.File;
import java.io.IOException;
@@ -4022,7 +4021,6 @@ private void setupInternalTopologyWithoutState(final StreamsConfig config) {
// TODO: change return type to `StandbyTask`
private Collection<Task> createStandbyTask(final StreamsConfig config) {
final LogContext logContext = new LogContext("test");
final Logger log = logContext.logger(StreamThreadTest.class);
final StreamsMetricsImpl streamsMetrics =
new StreamsMetricsImpl(metrics, CLIENT_ID, PROCESS_ID.toString(), mockTime);
final StandbyTaskCreator standbyTaskCreator = new StandbyTaskCreator(
@@ -4032,7 +4030,7 @@ private Collection<Task> createStandbyTask(final StreamsConfig config) {
stateDirectory,
new MockChangelogReader(),
CLIENT_ID,
log,
logContext,
false);
return standbyTaskCreator.createTasks(singletonMap(new TaskId(1, 2), emptySet()));
}