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-15995: Adding KIP-877 support to Connect #17804

Draft
wants to merge 2 commits into
base: trunk
Choose a base branch
from
Draft
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
5 changes: 5 additions & 0 deletions checkstyle/import-control.xml
Original file line number Diff line number Diff line change
Expand Up @@ -84,6 +84,11 @@
<allow class="org.apache.kafka.common.record.RecordBatch" exact-match="true" />
</subpackage>

<subpackage name="internals">
<allow pkg="org.apache.kafka.common.metrics" />
<allow pkg="org.apache.kafka.common.metrics.internals" />
</subpackage>

<subpackage name="message">
<allow pkg="com.fasterxml.jackson" />
<allow pkg="org.apache.kafka.common.protocol" />
Expand Down
3 changes: 3 additions & 0 deletions checkstyle/suppressions.xml
Original file line number Diff line number Diff line change
Expand Up @@ -182,6 +182,9 @@
<suppress checks="JavaNCSS"
files="(DistributedHerder|Worker)Test.java"/>

<suppress checks="ParameterNumber"
files="WorkerSinkTaskTest.java"/>

<!-- Raft -->
<suppress checks="NPathComplexity"
files="(DynamicVoter|RecordsIterator).java"/>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,8 @@
* {@link org.apache.kafka.clients.consumer.KafkaConsumer#poll(java.time.Duration)}.
* <p>
* Implement {@link org.apache.kafka.common.ClusterResourceListener} to receive cluster metadata once it's available. Please see the class documentation for ClusterResourceListener for more information.
* Implement {@link org.apache.kafka.common.metrics.Monitorable} to enable the interceptor to register metrics. The following tags are automatically added to
* all metrics registered: <code>config</code> set to <code>interceptor.classes</code>, and <code>class</code> set to the ConsumerInterceptor class name.
*/
public interface ConsumerInterceptor<K, V> extends Configurable, AutoCloseable {

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -303,12 +303,12 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) {
this.retryBackoffMs = config.getLong(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG);

List<ConsumerInterceptor<K, V>> interceptorList = configuredConsumerInterceptors(config);
this.interceptors = new ConsumerInterceptors<>(interceptorList);
this.deserializers = new Deserializers<>(config, keyDeserializer, valueDeserializer);
this.interceptors = new ConsumerInterceptors<>(interceptorList, metrics);
this.deserializers = new Deserializers<>(config, keyDeserializer, valueDeserializer, metrics);
this.subscriptions = createSubscriptionState(config, logContext);
ClusterResourceListeners clusterResourceListeners = ClientUtils.configureClusterResourceListeners(metrics.reporters(),
interceptorList,
Arrays.asList(deserializers.keyDeserializer, deserializers.valueDeserializer));
Arrays.asList(deserializers.keyDeserializer(), deserializers.valueDeserializer()));
this.metadata = metadataFactory.build(config, subscriptions, logContext, clusterResourceListeners);
final List<InetSocketAddress> addresses = ClientUtils.parseAndValidateAddresses(config);
metadata.bootstrap(addresses);
Expand Down Expand Up @@ -460,13 +460,13 @@ private void process(final ConsumerRebalanceListenerCallbackNeededEvent event) {
this.autoCommitEnabled = config.getBoolean(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG);
this.fetchBuffer = new FetchBuffer(logContext);
this.isolationLevel = IsolationLevel.READ_UNCOMMITTED;
this.interceptors = new ConsumerInterceptors<>(Collections.emptyList());
this.time = time;
this.metrics = new Metrics(time);
this.interceptors = new ConsumerInterceptors<>(Collections.emptyList(), metrics);
this.metadata = metadata;
this.retryBackoffMs = config.getLong(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG);
this.defaultApiTimeoutMs = config.getInt(ConsumerConfig.DEFAULT_API_TIMEOUT_MS_CONFIG);
this.deserializers = new Deserializers<>(keyDeserializer, valueDeserializer);
this.deserializers = new Deserializers<>(keyDeserializer, valueDeserializer, metrics);
this.clientTelemetryReporter = Optional.empty();

ConsumerMetrics metricsRegistry = new ConsumerMetrics(CONSUMER_METRIC_GROUP_PREFIX);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -179,13 +179,13 @@ public class ClassicKafkaConsumer<K, V> implements ConsumerDelegate<K, V> {
this.retryBackoffMaxMs = config.getLong(ConsumerConfig.RETRY_BACKOFF_MAX_MS_CONFIG);

List<ConsumerInterceptor<K, V>> interceptorList = configuredConsumerInterceptors(config);
this.interceptors = new ConsumerInterceptors<>(interceptorList);
this.deserializers = new Deserializers<>(config, keyDeserializer, valueDeserializer);
this.interceptors = new ConsumerInterceptors<>(interceptorList, metrics);
this.deserializers = new Deserializers<>(config, keyDeserializer, valueDeserializer, metrics);
this.subscriptions = createSubscriptionState(config, logContext);
ClusterResourceListeners clusterResourceListeners = ClientUtils.configureClusterResourceListeners(
metrics.reporters(),
interceptorList,
Arrays.asList(this.deserializers.keyDeserializer, this.deserializers.valueDeserializer));
Arrays.asList(this.deserializers.keyDeserializer(), this.deserializers.valueDeserializer()));
this.metadata = new ConsumerMetadata(config, subscriptions, logContext, clusterResourceListeners);
List<InetSocketAddress> addresses = ClientUtils.parseAndValidateAddresses(config);
this.metadata.bootstrap(addresses);
Expand Down Expand Up @@ -289,12 +289,12 @@ public class ClassicKafkaConsumer<K, V> implements ConsumerDelegate<K, V> {
this.metrics = new Metrics(time);
this.clientId = config.getString(ConsumerConfig.CLIENT_ID_CONFIG);
this.groupId = Optional.ofNullable(config.getString(ConsumerConfig.GROUP_ID_CONFIG));
this.deserializers = new Deserializers<>(keyDeserializer, valueDeserializer);
this.deserializers = new Deserializers<>(keyDeserializer, valueDeserializer, metrics);
this.isolationLevel = ConsumerUtils.configuredIsolationLevel(config);
this.defaultApiTimeoutMs = config.getInt(ConsumerConfig.DEFAULT_API_TIMEOUT_MS_CONFIG);
this.assignors = assignors;
this.kafkaConsumerMetrics = new KafkaConsumerMetrics(metrics, CONSUMER_METRIC_GROUP_PREFIX);
this.interceptors = new ConsumerInterceptors<>(Collections.emptyList());
this.interceptors = new ConsumerInterceptors<>(Collections.emptyList(), metrics);
this.retryBackoffMs = config.getLong(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG);
this.retryBackoffMaxMs = config.getLong(ConsumerConfig.RETRY_BACKOFF_MAX_MS_CONFIG);
this.requestTimeoutMs = config.getInt(ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -319,13 +319,13 @@ <K, V> ConsumerRecord<K, V> parseRecord(Deserializers<K, V> deserializers,
K key;
V value;
try {
key = keyBytes == null ? null : deserializers.keyDeserializer.deserialize(partition.topic(), headers, keyBytes);
key = keyBytes == null ? null : deserializers.keyDeserializer().deserialize(partition.topic(), headers, keyBytes);
} catch (RuntimeException e) {
log.error("Key Deserializers with error: {}", deserializers);
throw newRecordDeserializationException(DeserializationExceptionOrigin.KEY, partition, timestampType, record, e, headers);
}
try {
value = valueBytes == null ? null : deserializers.valueDeserializer.deserialize(partition.topic(), headers, valueBytes);
value = valueBytes == null ? null : deserializers.valueDeserializer().deserialize(partition.topic(), headers, valueBytes);
} catch (RuntimeException e) {
log.error("Value Deserializers with error: {}", deserializers);
throw newRecordDeserializationException(DeserializationExceptionOrigin.VALUE, partition, timestampType, record, e, headers);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,10 +17,13 @@
package org.apache.kafka.clients.consumer.internals;


import org.apache.kafka.clients.consumer.ConsumerConfig;
import org.apache.kafka.clients.consumer.ConsumerInterceptor;
import org.apache.kafka.clients.consumer.ConsumerRecords;
import org.apache.kafka.clients.consumer.OffsetAndMetadata;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.internals.Plugin;
import org.apache.kafka.common.metrics.Metrics;

import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand All @@ -35,15 +38,15 @@
*/
public class ConsumerInterceptors<K, V> implements Closeable {
private static final Logger log = LoggerFactory.getLogger(ConsumerInterceptors.class);
private final List<ConsumerInterceptor<K, V>> interceptors;
private final List<Plugin<ConsumerInterceptor<K, V>>> interceptorPlugins;

public ConsumerInterceptors(List<ConsumerInterceptor<K, V>> interceptors) {
this.interceptors = interceptors;
public ConsumerInterceptors(List<ConsumerInterceptor<K, V>> interceptors, Metrics metrics) {
this.interceptorPlugins = Plugin.wrapInstances(interceptors, metrics, ConsumerConfig.INTERCEPTOR_CLASSES_CONFIG);
}

/** Returns true if no interceptors are defined. All other methods will be no-ops in this case. */
public boolean isEmpty() {
return interceptors.isEmpty();
return interceptorPlugins.isEmpty();
}

/**
Expand All @@ -62,9 +65,9 @@ public boolean isEmpty() {
*/
public ConsumerRecords<K, V> onConsume(ConsumerRecords<K, V> records) {
ConsumerRecords<K, V> interceptRecords = records;
for (ConsumerInterceptor<K, V> interceptor : this.interceptors) {
for (Plugin<ConsumerInterceptor<K, V>> interceptorPlugin : this.interceptorPlugins) {
try {
interceptRecords = interceptor.onConsume(interceptRecords);
interceptRecords = interceptorPlugin.get().onConsume(interceptRecords);
} catch (Exception e) {
// do not propagate interceptor exception, log and continue calling other interceptors
log.warn("Error executing interceptor onConsume callback", e);
Expand All @@ -83,9 +86,9 @@ public ConsumerRecords<K, V> onConsume(ConsumerRecords<K, V> records) {
* @param offsets A map of offsets by partition with associated metadata
*/
public void onCommit(Map<TopicPartition, OffsetAndMetadata> offsets) {
for (ConsumerInterceptor<K, V> interceptor : this.interceptors) {
for (Plugin<ConsumerInterceptor<K, V>> interceptorPlugin : this.interceptorPlugins) {
try {
interceptor.onCommit(offsets);
interceptorPlugin.get().onCommit(offsets);
} catch (Exception e) {
// do not propagate interceptor exception, just log
log.warn("Error executing interceptor onCommit callback", e);
Expand All @@ -98,9 +101,9 @@ public void onCommit(Map<TopicPartition, OffsetAndMetadata> offsets) {
*/
@Override
public void close() {
for (ConsumerInterceptor<K, V> interceptor : this.interceptors) {
for (Plugin<ConsumerInterceptor<K, V>> interceptorPlugin : this.interceptorPlugins) {
try {
interceptor.close();
interceptorPlugin.close();
} catch (Exception e) {
log.error("Failed to close consumer interceptor ", e);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,8 @@
import org.apache.kafka.clients.consumer.ConsumerConfig;
import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.errors.InterruptException;
import org.apache.kafka.common.internals.Plugin;
import org.apache.kafka.common.metrics.Metrics;
import org.apache.kafka.common.serialization.Deserializer;
import org.apache.kafka.common.utils.Utils;

Expand All @@ -28,44 +30,54 @@

public class Deserializers<K, V> implements AutoCloseable {

public final Deserializer<K> keyDeserializer;
public final Deserializer<V> valueDeserializer;
private final Plugin<Deserializer<K>> keyDeserializerPlugin;
private final Plugin<Deserializer<V>> valueDeserializerPlugin;

public Deserializers(Deserializer<K> keyDeserializer, Deserializer<V> valueDeserializer) {
this.keyDeserializer = Objects.requireNonNull(keyDeserializer, "Key deserializer provided to Deserializers should not be null");
this.valueDeserializer = Objects.requireNonNull(valueDeserializer, "Value deserializer provided to Deserializers should not be null");
}

public Deserializers(ConsumerConfig config) {
this(config, null, null);
public Deserializers(Deserializer<K> keyDeserializer, Deserializer<V> valueDeserializer, Metrics metrics) {
this.keyDeserializerPlugin = Plugin.wrapInstance(
Objects.requireNonNull(keyDeserializer, "Key deserializer provided to Deserializers should not be null"),
metrics,
ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG);
this.valueDeserializerPlugin = Plugin.wrapInstance(
Objects.requireNonNull(valueDeserializer, "Value deserializer provided to Deserializers should not be null"),
metrics,
ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG);
}

@SuppressWarnings("unchecked")
public Deserializers(ConsumerConfig config, Deserializer<K> keyDeserializer, Deserializer<V> valueDeserializer) {
public Deserializers(ConsumerConfig config, Deserializer<K> keyDeserializer, Deserializer<V> valueDeserializer, Metrics metrics) {
String clientId = config.getString(ConsumerConfig.CLIENT_ID_CONFIG);

if (keyDeserializer == null) {
this.keyDeserializer = config.getConfiguredInstance(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, Deserializer.class);
this.keyDeserializer.configure(config.originals(Collections.singletonMap(ConsumerConfig.CLIENT_ID_CONFIG, clientId)), true);
keyDeserializer = config.getConfiguredInstance(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, Deserializer.class);
keyDeserializer.configure(config.originals(Collections.singletonMap(ConsumerConfig.CLIENT_ID_CONFIG, clientId)), true);
} else {
config.ignore(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG);
this.keyDeserializer = keyDeserializer;
}
this.keyDeserializerPlugin = Plugin.wrapInstance(keyDeserializer, metrics, ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG);

if (valueDeserializer == null) {
this.valueDeserializer = config.getConfiguredInstance(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, Deserializer.class);
this.valueDeserializer.configure(config.originals(Collections.singletonMap(ConsumerConfig.CLIENT_ID_CONFIG, clientId)), false);
valueDeserializer = config.getConfiguredInstance(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, Deserializer.class);
valueDeserializer.configure(config.originals(Collections.singletonMap(ConsumerConfig.CLIENT_ID_CONFIG, clientId)), false);
} else {
config.ignore(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG);
this.valueDeserializer = valueDeserializer;
}
this.valueDeserializerPlugin = Plugin.wrapInstance(valueDeserializer, metrics, ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG);
}

public Deserializer<K> keyDeserializer() {
return keyDeserializerPlugin.get();
}

public Deserializer<V> valueDeserializer() {
return valueDeserializerPlugin.get();
}

@Override
public void close() {
AtomicReference<Throwable> firstException = new AtomicReference<>();
Utils.closeQuietly(keyDeserializer, "key deserializer", firstException);
Utils.closeQuietly(valueDeserializer, "value deserializer", firstException);
Utils.closeQuietly(keyDeserializerPlugin, "key deserializer", firstException);
Utils.closeQuietly(valueDeserializerPlugin, "value deserializer", firstException);
Throwable exception = firstException.get();

if (exception != null) {
Expand All @@ -79,8 +91,8 @@ public void close() {
@Override
public String toString() {
return "Deserializers{" +
"keyDeserializer=" + keyDeserializer +
", valueDeserializer=" + valueDeserializer +
"keyDeserializer=" + keyDeserializerPlugin.get() +
", valueDeserializer=" + valueDeserializerPlugin.get() +
'}';
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -296,13 +296,13 @@ <K, V> ConsumerRecord<K, V> parseRecord(final Deserializers<K, V> deserializers,
K key;
V value;
try {
key = keyBytes == null ? null : deserializers.keyDeserializer.deserialize(partition.topic(), headers, keyBytes);
key = keyBytes == null ? null : deserializers.keyDeserializer().deserialize(partition.topic(), headers, keyBytes);
} catch (RuntimeException e) {
log.error("Key Deserializers with error: {}", deserializers);
throw newRecordDeserializationException(RecordDeserializationException.DeserializationExceptionOrigin.KEY, partition.topicPartition(), timestampType, record, e, headers);
}
try {
value = valueBytes == null ? null : deserializers.valueDeserializer.deserialize(partition.topic(), headers, valueBytes);
value = valueBytes == null ? null : deserializers.valueDeserializer().deserialize(partition.topic(), headers, valueBytes);
} catch (RuntimeException e) {
log.error("Value Deserializers with error: {}", deserializers);
throw newRecordDeserializationException(RecordDeserializationException.DeserializationExceptionOrigin.VALUE, partition.topicPartition(), timestampType, record, e, headers);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -253,12 +253,12 @@ private enum AcknowledgementMode {
this.clientTelemetryReporter.ifPresent(reporters::add);
this.metrics = createMetrics(config, time, reporters);

this.deserializers = new Deserializers<>(config, keyDeserializer, valueDeserializer);
this.deserializers = new Deserializers<>(config, keyDeserializer, valueDeserializer, metrics);
this.currentFetch = ShareFetch.empty();
this.subscriptions = createSubscriptionState(config, logContext);
ClusterResourceListeners clusterResourceListeners = ClientUtils.configureClusterResourceListeners(
metrics.reporters(),
Arrays.asList(deserializers.keyDeserializer, deserializers.valueDeserializer));
Arrays.asList(deserializers.keyDeserializer(), deserializers.valueDeserializer()));
this.metadata = new ConsumerMetadata(config, subscriptions, logContext, clusterResourceListeners);
final List<InetSocketAddress> addresses = ClientUtils.parseAndValidateAddresses(config);
metadata.bootstrap(addresses);
Expand Down Expand Up @@ -355,7 +355,7 @@ private enum AcknowledgementMode {
this.time = time;
this.metrics = new Metrics(time);
this.clientTelemetryReporter = Optional.empty();
this.deserializers = new Deserializers<>(config, keyDeserializer, valueDeserializer);
this.deserializers = new Deserializers<>(config, keyDeserializer, valueDeserializer, metrics);
this.currentFetch = ShareFetch.empty();
this.subscriptions = subscriptions;
this.metadata = metadata;
Expand Down Expand Up @@ -451,7 +451,7 @@ private enum AcknowledgementMode {
this.metrics = metrics;
this.metadata = metadata;
this.defaultApiTimeoutMs = defaultApiTimeoutMs;
this.deserializers = new Deserializers<>(keyDeserializer, valueDeserializer);
this.deserializers = new Deserializers<>(keyDeserializer, valueDeserializer, metrics);
this.currentFetch = ShareFetch.empty();
this.applicationEventHandler = applicationEventHandler;
this.kafkaShareConsumerMetrics = new KafkaShareConsumerMetrics(metrics, CONSUMER_SHARE_METRIC_GROUP_PREFIX);
Expand Down
Loading