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

Enhanced Kafka source logging through the use of MDC and thread naming #4663

Merged
merged 1 commit into from
Jun 26, 2024
Merged
Show file tree
Hide file tree
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 @@ -3,6 +3,8 @@
* SPDX-License-Identifier: Apache-2.0
*/

package org.opensearch.dataprepper.plugins.kafka.common;public class KafkaMdc {
package org.opensearch.dataprepper.plugins.kafka.common;

public class KafkaMdc {
public static final String MDC_KAFKA_PLUGIN_KEY = "kafkaPluginType";
}
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,16 @@ public class KafkaPluginThreadFactory implements ThreadFactory {
final ThreadFactory delegateThreadFactory,
final String kafkaPluginType) {
this.delegateThreadFactory = delegateThreadFactory;
this.threadPrefix = "kafka-" + kafkaPluginType + "-";
this.threadPrefix = createPluginPart(kafkaPluginType);
this.kafkaPluginType = kafkaPluginType;
}

KafkaPluginThreadFactory(
final ThreadFactory delegateThreadFactory,
final String kafkaPluginType,
final String kafkaTopic) {
this.delegateThreadFactory = delegateThreadFactory;
this.threadPrefix = normalizeName(kafkaTopic) + "-" + createPluginPart(kafkaPluginType);
this.kafkaPluginType = kafkaPluginType;
}

Expand All @@ -39,6 +48,28 @@ public static KafkaPluginThreadFactory defaultExecutorThreadFactory(final String
return new KafkaPluginThreadFactory(Executors.defaultThreadFactory(), kafkaPluginType);
}

/**
* Creates an instance specifically for use with {@link Executors}.
*
* @param kafkaPluginType The name of the plugin type. e.g. sink, source, buffer
* @return An instance of the {@link KafkaPluginThreadFactory}.
*/
public static KafkaPluginThreadFactory defaultExecutorThreadFactory(
final String kafkaPluginType,
final String kafkaTopic) {
return new KafkaPluginThreadFactory(Executors.defaultThreadFactory(), kafkaPluginType, kafkaTopic);
}

private static String createPluginPart(final String kafkaPluginType) {
return "kafka-" + kafkaPluginType + "-";
}

private static String normalizeName(final String kafkaTopic) {
final String limitedName = kafkaTopic.length() > 20 ? kafkaTopic.substring(0, 20) : kafkaTopic;
return limitedName
.toLowerCase().replaceAll("[^a-z0-9]", "-");
}

@Override
public Thread newThread(final Runnable runnable) {
final Thread thread = delegateThreadFactory.newThread(() -> {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,8 @@
import org.opensearch.dataprepper.model.plugin.PluginConfigObservable;
import org.opensearch.dataprepper.model.record.Record;
import org.opensearch.dataprepper.model.source.Source;
import org.opensearch.dataprepper.plugins.kafka.common.KafkaMdc;
import org.opensearch.dataprepper.plugins.kafka.common.thread.KafkaPluginThreadFactory;
import org.opensearch.dataprepper.plugins.kafka.configuration.AuthConfig;
import org.opensearch.dataprepper.plugins.kafka.configuration.TopicConsumerConfig;
import org.opensearch.dataprepper.plugins.kafka.configuration.OAuthConfig;
Expand All @@ -46,6 +48,7 @@
import org.opensearch.dataprepper.plugins.kafka.util.MessageFormat;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.slf4j.MDC;

import java.io.IOException;
import java.util.ArrayList;
Expand Down Expand Up @@ -73,10 +76,10 @@
public class KafkaSource implements Source<Record<Event>> {
private static final String NO_RESOLVABLE_URLS_ERROR_MESSAGE = "No resolvable bootstrap urls given in bootstrap.servers";
private static final long RETRY_SLEEP_INTERVAL = 30000;
private static final String MDC_KAFKA_PLUGIN_VALUE = "source";
private static final Logger LOG = LoggerFactory.getLogger(KafkaSource.class);
private final KafkaSourceConfig sourceConfig;
private final AtomicBoolean shutdownInProgress;
private ExecutorService executorService;
private final PluginMetrics pluginMetrics;
private KafkaCustomConsumer consumer;
private KafkaConsumer kafkaConsumer;
Expand Down Expand Up @@ -112,59 +115,65 @@ public KafkaSource(final KafkaSourceConfig sourceConfig,

@Override
public void start(Buffer<Record<Event>> buffer) {
Properties authProperties = new Properties();
KafkaSecurityConfigurer.setDynamicSaslClientCallbackHandler(authProperties, sourceConfig, pluginConfigObservable);
KafkaSecurityConfigurer.setAuthProperties(authProperties, sourceConfig, LOG);
sourceConfig.getTopics().forEach(topic -> {
consumerGroupID = topic.getGroupId();
KafkaTopicConsumerMetrics topicMetrics = new KafkaTopicConsumerMetrics(topic.getName(), pluginMetrics, true);
Properties consumerProperties = getConsumerProperties(topic, authProperties);
MessageFormat schema = MessageFormat.getByMessageFormatByName(schemaType);
try {
int numWorkers = topic.getWorkers();
executorService = Executors.newFixedThreadPool(numWorkers);
allTopicExecutorServices.add(executorService);

IntStream.range(0, numWorkers).forEach(index -> {
while (true) {
try {
kafkaConsumer = createKafkaConsumer(schema, consumerProperties);
break;
} catch (ConfigException ce) {
if (ce.getMessage().contains(NO_RESOLVABLE_URLS_ERROR_MESSAGE)) {
LOG.warn("Exception while creating Kafka consumer: ", ce);
LOG.warn("Bootstrap URL could not be resolved. Retrying in {} ms...", RETRY_SLEEP_INTERVAL);
try {
sleep(RETRY_SLEEP_INTERVAL);
} catch (InterruptedException ie) {
Thread.currentThread().interrupt();
throw new RuntimeException(ie);
try {
setMdc();
Properties authProperties = new Properties();
KafkaSecurityConfigurer.setDynamicSaslClientCallbackHandler(authProperties, sourceConfig, pluginConfigObservable);
KafkaSecurityConfigurer.setAuthProperties(authProperties, sourceConfig, LOG);
sourceConfig.getTopics().forEach(topic -> {
consumerGroupID = topic.getGroupId();
KafkaTopicConsumerMetrics topicMetrics = new KafkaTopicConsumerMetrics(topic.getName(), pluginMetrics, true);
Properties consumerProperties = getConsumerProperties(topic, authProperties);
MessageFormat schema = MessageFormat.getByMessageFormatByName(schemaType);
try {
int numWorkers = topic.getWorkers();
final ExecutorService executorService = Executors.newFixedThreadPool(
numWorkers, KafkaPluginThreadFactory.defaultExecutorThreadFactory(MDC_KAFKA_PLUGIN_VALUE, topic.getName()));
allTopicExecutorServices.add(executorService);

IntStream.range(0, numWorkers).forEach(index -> {
while (true) {
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This part of the code is increasing, Probably good idea to move to separate helper function. Maybe a separate PR.

try {
kafkaConsumer = createKafkaConsumer(schema, consumerProperties);
break;
} catch (ConfigException ce) {
if (ce.getMessage().contains(NO_RESOLVABLE_URLS_ERROR_MESSAGE)) {
LOG.warn("Exception while creating Kafka consumer: ", ce);
LOG.warn("Bootstrap URL could not be resolved. Retrying in {} ms...", RETRY_SLEEP_INTERVAL);
try {
sleep(RETRY_SLEEP_INTERVAL);
} catch (InterruptedException ie) {
Thread.currentThread().interrupt();
throw new RuntimeException(ie);
}
} else {
throw ce;
}
} else {
throw ce;
}

}
consumer = new KafkaCustomConsumer(kafkaConsumer, shutdownInProgress, buffer, sourceConfig, topic, schemaType,
acknowledgementSetManager, null, topicMetrics, PauseConsumePredicate.noPause());
allTopicConsumers.add(consumer);

executorService.submit(consumer);
});
} catch (Exception e) {
if (e instanceof BrokerNotAvailableException || e instanceof TimeoutException) {
LOG.error("The kafka broker is not available...");
} else {
LOG.error("Failed to setup the Kafka Source Plugin.", e);
}
consumer = new KafkaCustomConsumer(kafkaConsumer, shutdownInProgress, buffer, sourceConfig, topic, schemaType,
acknowledgementSetManager, null, topicMetrics, PauseConsumePredicate.noPause());
allTopicConsumers.add(consumer);

executorService.submit(consumer);
});
} catch (Exception e) {
if (e instanceof BrokerNotAvailableException || e instanceof TimeoutException) {
LOG.error("The kafka broker is not available...");
} else {
LOG.error("Failed to setup the Kafka Source Plugin.", e);
throw new RuntimeException(e);
}
throw new RuntimeException(e);
}
LOG.info("Started Kafka source for topic " + topic.getName());
});
LOG.info("Started Kafka source for topic " + topic.getName());
});
} finally {
removeMdc();
}
}

public KafkaConsumer<?, ?> createKafkaConsumer(final MessageFormat schema, final Properties consumerProperties) {
KafkaConsumer<?, ?> createKafkaConsumer(final MessageFormat schema, final Properties consumerProperties) {
switch (schema) {
case JSON:
return new KafkaConsumer<String, JsonNode>(consumerProperties);
Expand All @@ -183,19 +192,24 @@ public void start(Buffer<Record<Event>> buffer) {

@Override
public void stop() {
shutdownInProgress.set(true);
final long shutdownWaitTime = calculateLongestThreadWaitingTime();
try {
setMdc();
shutdownInProgress.set(true);
final long shutdownWaitTime = calculateLongestThreadWaitingTime();

LOG.info("Shutting down {} Executor services", allTopicExecutorServices.size());
allTopicExecutorServices.forEach(executor -> stopExecutor(executor, shutdownWaitTime));
LOG.info("Shutting down {} Executor services", allTopicExecutorServices.size());
allTopicExecutorServices.forEach(executor -> stopExecutor(executor, shutdownWaitTime));

LOG.info("Closing {} consumers", allTopicConsumers.size());
allTopicConsumers.forEach(consumer -> consumer.closeConsumer());
LOG.info("Closing {} consumers", allTopicConsumers.size());
allTopicConsumers.forEach(consumer -> consumer.closeConsumer());

LOG.info("Kafka source shutdown successfully...");
LOG.info("Kafka source shutdown successfully...");
} finally {
removeMdc();
}
}

public void stopExecutor(final ExecutorService executorService, final long shutdownWaitTime) {
private void stopExecutor(final ExecutorService executorService, final long shutdownWaitTime) {
executorService.shutdown();
try {
if (!executorService.awaitTermination(shutdownWaitTime, TimeUnit.SECONDS)) {
Expand Down Expand Up @@ -346,7 +360,7 @@ private void setPropertiesForSchemaRegistryConnectivity(Properties properties) {
}
}

protected void sleep(final long millis) throws InterruptedException {
void sleep(final long millis) throws InterruptedException {
Thread.sleep(millis);
}

Expand All @@ -366,4 +380,12 @@ private void updateConfig(final KafkaClusterConfigSupplier kafkaClusterConfigSup
}
}
}

private static void setMdc() {
MDC.put(KafkaMdc.MDC_KAFKA_PLUGIN_KEY, MDC_KAFKA_PLUGIN_VALUE);
}

private static void removeMdc() {
MDC.remove(KafkaMdc.MDC_KAFKA_PLUGIN_KEY);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -8,6 +8,8 @@
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.extension.ExtendWith;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.CsvSource;
import org.mockito.ArgumentCaptor;
import org.mockito.Mock;
import org.mockito.junit.jupiter.MockitoExtension;
Expand Down Expand Up @@ -37,10 +39,12 @@ class KafkaPluginThreadFactoryTest {
@Mock
private Runnable runnable;
private String pluginType;
private String topic;

@BeforeEach
void setUp() {
pluginType = UUID.randomUUID().toString();
topic = UUID.randomUUID().toString();

when(delegateThreadFactory.newThread(any(Runnable.class))).thenReturn(innerThread);
}
Expand All @@ -50,11 +54,20 @@ private KafkaPluginThreadFactory createObjectUnderTest() {
return new KafkaPluginThreadFactory(delegateThreadFactory, pluginType);
}

private KafkaPluginThreadFactory createObjectUnderTestWithTopic() {
return new KafkaPluginThreadFactory(delegateThreadFactory, pluginType, topic);
}

@Test
void newThread_creates_thread_from_delegate() {
assertThat(createObjectUnderTest().newThread(runnable), equalTo(innerThread));
}

@Test
void newThread_with_topic_creates_thread_from_delegate() {
assertThat(createObjectUnderTestWithTopic().newThread(runnable), equalTo(innerThread));
}

@Test
void newThread_creates_thread_with_name() {
final KafkaPluginThreadFactory objectUnderTest = createObjectUnderTest();
Expand All @@ -69,6 +82,30 @@ void newThread_creates_thread_with_name() {
verify(thread2).setName(String.format("kafka-%s-2", pluginType));
}

@ParameterizedTest
@CsvSource({
"abcd12,abcd12",
"aBCd12,abcd12",
"abcd-12,abcd-12",
"has space,has-space",
"has!character,has-character",
"this-is-somewhat-too-long,this-is-somewhat-too"
})
void newThread_with_topic_creates_thread_with_name(
final String topicName,
final String expectedPrefix) {
this.topic = topicName;
final KafkaPluginThreadFactory objectUnderTest = createObjectUnderTestWithTopic();

final Thread thread1 = objectUnderTest.newThread(runnable);
assertThat(thread1, notNullValue());
verify(thread1).setName(String.format("%s-kafka-%s-1", expectedPrefix, pluginType));

final Thread thread2 = objectUnderTest.newThread(runnable);
assertThat(thread2, notNullValue());
verify(thread2).setName(String.format("%s-kafka-%s-2", expectedPrefix, pluginType));
}

@Test
void newThread_creates_thread_with_wrapping_runnable() {
createObjectUnderTest().newThread(runnable);
Expand All @@ -85,6 +122,22 @@ void newThread_creates_thread_with_wrapping_runnable() {
verify(runnable).run();
}

@Test
void newThread_with_topic_creates_thread_with_wrapping_runnable() {
createObjectUnderTestWithTopic().newThread(runnable);

final ArgumentCaptor<Runnable> actualRunnableCaptor = ArgumentCaptor.forClass(Runnable.class);
verify(delegateThreadFactory).newThread(actualRunnableCaptor.capture());

final Runnable actualRunnable = actualRunnableCaptor.getValue();

assertThat(actualRunnable, not(equalTo(runnable)));

verifyNoInteractions(runnable);
actualRunnable.run();
verify(runnable).run();
}

@Test
void newThread_creates_thread_that_calls_MDC_on_run() {
createObjectUnderTest().newThread(runnable);
Expand All @@ -104,4 +157,24 @@ void newThread_creates_thread_that_calls_MDC_on_run() {

assertThat(actualKafkaPluginType[0], equalTo(pluginType));
}

@Test
void newThread_with_topic_creates_thread_that_calls_MDC_on_run() {
createObjectUnderTestWithTopic().newThread(runnable);

final ArgumentCaptor<Runnable> actualRunnableCaptor = ArgumentCaptor.forClass(Runnable.class);
verify(delegateThreadFactory).newThread(actualRunnableCaptor.capture());

final Runnable actualRunnable = actualRunnableCaptor.getValue();

final String[] actualKafkaPluginType = new String[1];
doAnswer(a -> {
actualKafkaPluginType[0] = MDC.get(KafkaMdc.MDC_KAFKA_PLUGIN_KEY);
return null;
}).when(runnable).run();

actualRunnable.run();

assertThat(actualKafkaPluginType[0], equalTo(pluginType));
}
}
Loading
Loading