Skip to content
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

KAFKA-17744: improve state updater logs when restorating state #17714

Open
wants to merge 1 commit 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
Expand Up @@ -369,10 +369,15 @@ public static StreamThread create(final TopologyMetadata topologyMetadata,
final Runnable shutdownErrorHook,
final BiConsumer<Throwable, Boolean> streamsUncaughtExceptionHandler) {

final boolean stateUpdaterEnabled = InternalConfig.stateUpdaterEnabled(config.originals());

final String threadId = clientId + THREAD_ID_SUBSTRING + threadIdx;
final String stateUpdaterId = threadId.replace(THREAD_ID_SUBSTRING, STATE_UPDATER_ID_SUBSTRING);
final String effectiveThreadId = stateUpdaterEnabled ? stateUpdaterId : threadId;

final String logPrefix = String.format("stream-thread [%s] ", threadId);
final LogContext logContext = new LogContext(logPrefix);
final LogContext effectiveLogContext = stateUpdaterEnabled ? new LogContext(String.format("state-updater [%s] ", effectiveThreadId)): logContext ;
final Logger log = logContext.logger(StreamThread.class);

final ReferenceContainer referenceContainer = new ReferenceContainer();
Expand All @@ -382,13 +387,13 @@ public static StreamThread create(final TopologyMetadata topologyMetadata,
referenceContainer.clientTags = config.getClientTags();

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

final StoreChangelogReader changelogReader = new StoreChangelogReader(
time,
config,
logContext,
effectiveLogContext,
adminClient,
restoreConsumer,
userStateRestoreListener,
Expand All @@ -397,7 +402,6 @@ public static StreamThread create(final TopologyMetadata topologyMetadata,

final ThreadCache cache = new ThreadCache(logContext, cacheSizeBytes, streamsMetrics);

final boolean stateUpdaterEnabled = InternalConfig.stateUpdaterEnabled(config.originals());
final boolean proceessingThreadsEnabled = InternalConfig.processingThreadsEnabled(config.originals());
final ActiveTaskCreator activeTaskCreator = new ActiveTaskCreator(
topologyMetadata,
Expand Down Expand Up @@ -475,7 +479,6 @@ public static StreamThread create(final TopologyMetadata topologyMetadata,
taskManager.setMainConsumer(mainConsumer);
referenceContainer.mainConsumer = mainConsumer;

final String stateUpdaterId = threadId.replace(THREAD_ID_SUBSTRING, STATE_UPDATER_ID_SUBSTRING);
final StreamsThreadMetricsDelegatingReporter reporter = new StreamsThreadMetricsDelegatingReporter(mainConsumer, threadId, stateUpdaterId);
streamsMetrics.metricsRegistry().addReporter(reporter);

Expand Down