KAFKA-15995: Adding KIP-877 support to Connect (#17804)

Reviewers: Greg Harris <gharris1727@gmail.com>
This commit is contained in:
Mickael Maison 2025-02-11 18:27:15 +01:00 committed by GitHub
parent 0989a6a389
commit c13324fc16
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
73 changed files with 1773 additions and 659 deletions

View File

@ -183,6 +183,9 @@
<suppress checks="JavaNCSS"
files="(DistributedHerder|Worker)Test.java"/>
<suppress checks="ParameterNumber"
files="WorkerSinkTaskTest.java"/>
<!-- Raft -->
<suppress checks="NPathComplexity"
files="(DynamicVoter|RecordsIterator).java"/>

View File

@ -16,6 +16,8 @@
*/
package org.apache.kafka.connect.connector;
import org.apache.kafka.common.metrics.PluginMetrics;
/**
* ConnectorContext allows {@link Connector}s to proactively interact with the Kafka Connect runtime.
*/
@ -33,4 +35,26 @@ public interface ConnectorContext {
* @param e Exception to be raised.
*/
void raiseError(Exception e);
/**
* Get a {@link PluginMetrics} that can be used to define metrics
*
* <p>This method was added in Apache Kafka 4.1. Connectors that use this method but want to
* maintain backward compatibility so they can also be deployed to older Connect runtimes
* should guard the call to this method with a try-catch block, since calling this method will result in a
* {@link NoSuchMethodError} or {@link NoClassDefFoundError} when the connector is deployed to
* Connect runtimes older than Kafka 4.1. For example:
* <pre>
* PluginMetrics pluginMetrics;
* try {
* pluginMetrics = context.pluginMetrics();
* } catch (NoSuchMethodError | NoClassDefFoundError e) {
* pluginMetrics = null;
* }
* </pre>
*
* @return the pluginMetrics instance
* @since 4.1
*/
PluginMetrics pluginMetrics();
}

View File

@ -30,6 +30,11 @@ import java.util.List;
* <p>Kafka Connect discovers implementations of this interface using the Java {@link java.util.ServiceLoader} mechanism.
* To support this, implementations of this interface should also contain a service provider configuration file in
* {@code META-INF/services/org.apache.kafka.connect.connector.policy.ConnectorClientConfigOverridePolicy}.
* <p>
* Implement {@link org.apache.kafka.common.metrics.Monitorable} to enable the policy to register metrics.
* The following tags are automatically added to all metrics registered: <code>config</code> set to
* <code>connector.client.config.override.policy</code>, and <code>class</code> set to the
* ConnectorClientConfigOverridePolicy class name.
*/
public interface ConnectorClientConfigOverridePolicy extends Configurable, AutoCloseable {

View File

@ -43,6 +43,10 @@ import java.util.Map;
*
* <p>When the Connect worker shuts down, it will call the extension's {@link #close} method to allow the implementation to release all of
* its resources.
*
* <p>Implement {@link org.apache.kafka.common.metrics.Monitorable} to enable the extension to register metrics.
* The following tags are automatically added to all metrics registered: <code>config</code> set to
* <code>rest.extension.classes</code>, and <code>class</code> set to the ConnectRestExtension class name.
*/
public interface ConnectRestExtension extends Configurable, Versioned, Closeable {

View File

@ -17,6 +17,7 @@
package org.apache.kafka.connect.sink;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.metrics.PluginMetrics;
import java.util.Map;
import java.util.Set;
@ -123,4 +124,26 @@ public interface SinkTaskContext {
return null;
}
/**
* Get a {@link PluginMetrics} that can be used to define metrics
*
* <p>This method was added in Apache Kafka 4.1. Tasks that use this method but want to
* maintain backward compatibility so they can also be deployed to older Connect runtimes
* should guard the call to this method with a try-catch block, since calling this method will result in a
* {@link NoSuchMethodError} or {@link NoClassDefFoundError} when the connector is deployed to
* Connect runtimes older than Kafka 4.1. For example:
* <pre>
* PluginMetrics pluginMetrics;
* try {
* pluginMetrics = context.pluginMetrics();
* } catch (NoSuchMethodError | NoClassDefFoundError e) {
* pluginMetrics = null;
* }
* </pre>
*
* @return the PluginMetrics instance
* @since 4.1
*/
PluginMetrics pluginMetrics();
}

View File

@ -16,6 +16,7 @@
*/
package org.apache.kafka.connect.source;
import org.apache.kafka.common.metrics.PluginMetrics;
import org.apache.kafka.connect.storage.OffsetStorageReader;
import java.util.Map;
@ -63,4 +64,26 @@ public interface SourceTaskContext {
default TransactionContext transactionContext() {
return null;
}
/**
* Get a {@link PluginMetrics} that can be used to define metrics
*
* <p>This method was added in Apache Kafka 4.1. Tasks that use this method but want to
* maintain backward compatibility so they can also be deployed to older Connect runtimes
* should guard the call to this method with a try-catch block, since calling this method will result in a
* {@link NoSuchMethodError} or {@link NoClassDefFoundError} when the connector is deployed to
* Connect runtimes older than Kafka 4.1. For example:
* <pre>
* PluginMetrics pluginMetrics;
* try {
* pluginMetrics = context.pluginMetrics();
* } catch (NoSuchMethodError | NoClassDefFoundError e) {
* pluginMetrics = null;
* }
* </pre>
*
* @return the pluginMetrics instance
* @since 4.1
*/
PluginMetrics pluginMetrics();
}

View File

@ -21,6 +21,8 @@ import org.apache.kafka.common.header.Headers;
import org.apache.kafka.connect.data.Schema;
import org.apache.kafka.connect.data.SchemaAndValue;
import java.io.Closeable;
import java.io.IOException;
import java.util.Map;
/**
@ -30,8 +32,12 @@ import java.util.Map;
* <p>Kafka Connect may discover implementations of this interface using the Java {@link java.util.ServiceLoader} mechanism.
* To support this, implementations of this interface should also contain a service provider configuration file in
* {@code META-INF/services/org.apache.kafka.connect.storage.Converter}.
*
* <p>Implement {@link org.apache.kafka.common.metrics.Monitorable} to enable the converter to register metrics.
* The following tags are automatically added to all metrics registered: <code>connector</code> set to connector name,
* <code>task</code> set to the task id and <code>converter</code> set to either <code>key</code> or <code>value</code>.
*/
public interface Converter {
public interface Converter extends Closeable {
/**
* Configure this class.
@ -98,4 +104,9 @@ public interface Converter {
default ConfigDef config() {
return new ConfigDef();
}
@Override
default void close() throws IOException {
// no op
}
}

View File

@ -31,6 +31,10 @@ import java.io.Closeable;
* <p>Kafka Connect may discover implementations of this interface using the Java {@link java.util.ServiceLoader} mechanism.
* To support this, implementations of this interface should also contain a service provider configuration file in
* {@code META-INF/services/org.apache.kafka.connect.storage.HeaderConverter}.
*
* <p>Implement {@link org.apache.kafka.common.metrics.Monitorable} to enable the converter to register metrics.
* The following tags are automatically added to all metrics registered: <code>connector</code> set to connector name,
* <code>task</code> set to the task id and <code>converter</code> set to <code>header</code>.
*/
public interface HeaderConverter extends Configurable, Closeable {

View File

@ -30,6 +30,10 @@ import java.io.Closeable;
* To support this, implementations of this interface should also contain a service provider configuration file in
* {@code META-INF/services/org.apache.kafka.connect.transforms.Transformation}.
*
* <p>Implement {@link org.apache.kafka.common.metrics.Monitorable} to enable the transformation to register metrics.
* The following tags are automatically added to all metrics registered: <code>connector</code> set to connector name,
* <code>task</code> set to the task id and <code>transformation</code> set to the transformation alias.
*
* @param <R> The type of record (must be an implementation of {@link ConnectRecord})
*/
public interface Transformation<R extends ConnectRecord<R>> extends Configurable, Closeable {

View File

@ -31,6 +31,10 @@ import org.apache.kafka.connect.connector.ConnectRecord;
* To support this, implementations of this interface should also contain a service provider configuration file in
* {@code META-INF/services/org.apache.kafka.connect.transforms.predicates.Predicate}.
*
* <p>Implement {@link org.apache.kafka.common.metrics.Monitorable} to enable the predicate to register metrics.
* The following tags are automatically added to all metrics registered: <code>connector</code> set to connector name,
* <code>task</code> set to the task id and <code>predicate</code> set to the predicate alias.
*
* @param <R> The type of record.
*/
public interface Predicate<R extends ConnectRecord<R>> extends Configurable, AutoCloseable {

View File

@ -17,6 +17,7 @@
package org.apache.kafka.connect.sink;
import org.apache.kafka.common.config.ConfigDef;
import org.apache.kafka.common.metrics.PluginMetrics;
import org.apache.kafka.connect.connector.ConnectorContext;
import org.apache.kafka.connect.connector.ConnectorTest;
import org.apache.kafka.connect.connector.Task;
@ -53,6 +54,12 @@ public class SinkConnectorTest extends ConnectorTest {
// Unexpected in these tests
throw new UnsupportedOperationException();
}
@Override
public PluginMetrics pluginMetrics() {
// Unexpected in these tests
throw new UnsupportedOperationException();
}
}
protected static class TestSinkConnector extends SinkConnector implements ConnectorTest.AssertableConnector {

View File

@ -17,6 +17,7 @@
package org.apache.kafka.connect.source;
import org.apache.kafka.common.config.ConfigDef;
import org.apache.kafka.common.metrics.PluginMetrics;
import org.apache.kafka.connect.connector.ConnectorContext;
import org.apache.kafka.connect.connector.ConnectorTest;
import org.apache.kafka.connect.connector.Task;
@ -55,6 +56,12 @@ public class SourceConnectorTest extends ConnectorTest {
throw new UnsupportedOperationException();
}
@Override
public PluginMetrics pluginMetrics() {
// Unexpected in these tests
throw new UnsupportedOperationException();
}
@Override
public OffsetStorageReader offsetStorageReader() {
return null;

View File

@ -26,6 +26,7 @@ import org.apache.kafka.common.config.ConfigDef.ConfigKey;
import org.apache.kafka.common.config.ConfigDef.Type;
import org.apache.kafka.common.config.ConfigTransformer;
import org.apache.kafka.common.config.ConfigValue;
import org.apache.kafka.common.internals.Plugin;
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.connect.connector.Connector;
@ -140,7 +141,7 @@ public abstract class AbstractHerder implements Herder, TaskStatus.Listener, Con
protected final StatusBackingStore statusBackingStore;
protected final ConfigBackingStore configBackingStore;
private volatile boolean ready = false;
private final ConnectorClientConfigOverridePolicy connectorClientConfigOverridePolicy;
private final Plugin<ConnectorClientConfigOverridePolicy> connectorClientConfigOverridePolicyPlugin;
private final ExecutorService connectorExecutor;
private final Time time;
protected final Loggers loggers;
@ -160,7 +161,10 @@ public abstract class AbstractHerder implements Herder, TaskStatus.Listener, Con
this.kafkaClusterId = kafkaClusterId;
this.statusBackingStore = statusBackingStore;
this.configBackingStore = configBackingStore;
this.connectorClientConfigOverridePolicy = connectorClientConfigOverridePolicy;
this.connectorClientConfigOverridePolicyPlugin = Plugin.wrapInstance(
connectorClientConfigOverridePolicy,
worker.metrics().metrics(),
WorkerConfig.CONNECTOR_CLIENT_POLICY_CLASS_CONFIG);
this.connectorExecutor = Executors.newCachedThreadPool();
this.time = time;
this.loggers = Loggers.newInstance(time);
@ -185,7 +189,7 @@ public abstract class AbstractHerder implements Herder, TaskStatus.Listener, Con
this.configBackingStore.stop();
this.worker.stop();
this.connectorExecutor.shutdown();
Utils.closeQuietly(this.connectorClientConfigOverridePolicy, "connector client config override policy");
Utils.closeQuietly(this.connectorClientConfigOverridePolicyPlugin, "connector client config override policy");
}
protected void ready() {
@ -388,6 +392,11 @@ public abstract class AbstractHerder implements Herder, TaskStatus.Listener, Con
status.workerId(), status.trace());
}
@Override
public ConnectMetrics connectMetrics() {
return worker.metrics();
}
protected Map<String, ConfigValue> validateSinkConnectorConfig(SinkConnector connector, ConfigDef configDef, Map<String, String> config) {
Map<String, ConfigValue> result = configDef.validateAll(config);
SinkConnectorConfig.validate(config, result);
@ -691,7 +700,7 @@ public abstract class AbstractHerder implements Herder, TaskStatus.Listener, Con
connectorClass,
connectorType,
ConnectorClientConfigRequest.ClientType.PRODUCER,
connectorClientConfigOverridePolicy);
connectorClientConfigOverridePolicyPlugin);
}
}
if (connectorUsesAdmin(connectorType, connectorProps)) {
@ -705,7 +714,7 @@ public abstract class AbstractHerder implements Herder, TaskStatus.Listener, Con
connectorClass,
connectorType,
ConnectorClientConfigRequest.ClientType.ADMIN,
connectorClientConfigOverridePolicy);
connectorClientConfigOverridePolicyPlugin);
}
}
if (connectorUsesConsumer(connectorType, connectorProps)) {
@ -719,7 +728,7 @@ public abstract class AbstractHerder implements Herder, TaskStatus.Listener, Con
connectorClass,
connectorType,
ConnectorClientConfigRequest.ClientType.CONSUMER,
connectorClientConfigOverridePolicy);
connectorClientConfigOverridePolicyPlugin);
}
}
return mergeConfigInfos(connType,
@ -893,7 +902,7 @@ public abstract class AbstractHerder implements Herder, TaskStatus.Listener, Con
Class<? extends Connector> connectorClass,
org.apache.kafka.connect.health.ConnectorType connectorType,
ConnectorClientConfigRequest.ClientType clientType,
ConnectorClientConfigOverridePolicy connectorClientConfigOverridePolicy) {
Plugin<ConnectorClientConfigOverridePolicy> connectorClientConfigOverridePolicyPlugin) {
Map<String, Object> clientConfigs = new HashMap<>();
for (Map.Entry<String, Object> rawClientConfig : connectorConfig.originalsWithPrefix(prefix).entrySet()) {
String configName = rawClientConfig.getKey();
@ -906,7 +915,7 @@ public abstract class AbstractHerder implements Herder, TaskStatus.Listener, Con
}
ConnectorClientConfigRequest connectorClientConfigRequest = new ConnectorClientConfigRequest(
connName, connectorType, connectorClass, clientConfigs, clientType);
List<ConfigValue> configValues = connectorClientConfigOverridePolicy.validate(connectorClientConfigRequest);
List<ConfigValue> configValues = connectorClientConfigOverridePolicyPlugin.get().validate(connectorClientConfigRequest);
return prefixedConfigInfos(configDef.configKeys(), configValues, prefix);
}

View File

@ -24,6 +24,7 @@ import org.apache.kafka.clients.producer.ProducerRecord;
import org.apache.kafka.clients.producer.RecordMetadata;
import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.header.internals.RecordHeaders;
import org.apache.kafka.common.internals.Plugin;
import org.apache.kafka.common.metrics.Sensor;
import org.apache.kafka.common.metrics.stats.Avg;
import org.apache.kafka.common.metrics.stats.CumulativeSum;
@ -46,6 +47,7 @@ import org.apache.kafka.connect.source.SourceRecord;
import org.apache.kafka.connect.source.SourceTask;
import org.apache.kafka.connect.source.SourceTaskContext;
import org.apache.kafka.connect.storage.CloseableOffsetStorageReader;
import org.apache.kafka.connect.storage.ClusterConfigState;
import org.apache.kafka.connect.storage.ConnectorOffsetBackingStore;
import org.apache.kafka.connect.storage.Converter;
import org.apache.kafka.connect.storage.HeaderConverter;
@ -184,15 +186,14 @@ public abstract class AbstractWorkerSourceTask extends WorkerTask<SourceRecord,
protected final WorkerConfig workerConfig;
protected final WorkerSourceTaskContext sourceTaskContext;
protected final ConnectorOffsetBackingStore offsetStore;
protected final OffsetStorageWriter offsetWriter;
protected final Producer<byte[], byte[]> producer;
private final SourceTask task;
private final Converter keyConverter;
private final Converter valueConverter;
private final HeaderConverter headerConverter;
private final Plugin<Converter> keyConverterPlugin;
private final Plugin<Converter> valueConverterPlugin;
private final Plugin<HeaderConverter> headerConverterPlugin;
private final TopicAdmin admin;
private final CloseableOffsetStorageReader offsetReader;
private final SourceTaskMetricsGroup sourceTaskMetricsGroup;
@ -204,6 +205,7 @@ public abstract class AbstractWorkerSourceTask extends WorkerTask<SourceRecord,
// Visible for testing
List<SourceRecord> toSend;
protected Map<String, String> taskConfig;
protected WorkerSourceTaskContext sourceTaskContext;
protected boolean started = false;
private volatile boolean producerClosed = false;
@ -211,11 +213,12 @@ public abstract class AbstractWorkerSourceTask extends WorkerTask<SourceRecord,
SourceTask task,
TaskStatus.Listener statusListener,
TargetState initialState,
Converter keyConverter,
Converter valueConverter,
HeaderConverter headerConverter,
ClusterConfigState configState,
Plugin<Converter> keyConverterPlugin,
Plugin<Converter> valueConverterPlugin,
Plugin<HeaderConverter> headerConverterPlugin,
TransformationChain<SourceRecord, SourceRecord> transformationChain,
WorkerSourceTaskContext sourceTaskContext,
WorkerTransactionContext workerTransactionContext,
Producer<byte[], byte[]> producer,
TopicAdmin admin,
Map<String, TopicCreationGroup> topicGroups,
@ -238,16 +241,16 @@ public abstract class AbstractWorkerSourceTask extends WorkerTask<SourceRecord,
this.workerConfig = workerConfig;
this.task = task;
this.keyConverter = keyConverter;
this.valueConverter = valueConverter;
this.headerConverter = headerConverter;
this.keyConverterPlugin = keyConverterPlugin;
this.valueConverterPlugin = valueConverterPlugin;
this.headerConverterPlugin = headerConverterPlugin;
this.producer = producer;
this.admin = admin;
this.offsetReader = offsetReader;
this.offsetWriter = offsetWriter;
this.offsetStore = Objects.requireNonNull(offsetStore, "offset store cannot be null for source tasks");
this.closeExecutor = closeExecutor;
this.sourceTaskContext = sourceTaskContext;
this.sourceTaskContext = new WorkerSourceTaskContext(offsetReader, id, configState, workerTransactionContext, pluginMetrics);
this.stopRequestedLatch = new CountDownLatch(1);
this.sourceTaskMetricsGroup = new SourceTaskMetricsGroup(id, connectMetrics);
this.topicTrackingEnabled = workerConfig.getBoolean(TOPIC_TRACKING_ENABLE_CONFIG);
@ -320,7 +323,10 @@ public abstract class AbstractWorkerSourceTask extends WorkerTask<SourceRecord,
}
Utils.closeQuietly(offsetReader, "offset reader");
Utils.closeQuietly(offsetStore::stop, "offset backing store");
Utils.closeQuietly(headerConverter, "header converter");
Utils.closeQuietly(headerConverterPlugin, "header converter");
Utils.closeQuietly(keyConverterPlugin, "key converter");
Utils.closeQuietly(valueConverterPlugin, "value converter");
Utils.closeQuietly(pluginMetrics, "pluginMetrics");
}
private void closeProducer(Duration duration) {
@ -483,13 +489,13 @@ public abstract class AbstractWorkerSourceTask extends WorkerTask<SourceRecord,
return null;
}
RecordHeaders headers = retryWithToleranceOperator.execute(context, () -> convertHeaderFor(record), Stage.HEADER_CONVERTER, headerConverter.getClass());
RecordHeaders headers = retryWithToleranceOperator.execute(context, () -> convertHeaderFor(record), Stage.HEADER_CONVERTER, headerConverterPlugin.get().getClass());
byte[] key = retryWithToleranceOperator.execute(context, () -> keyConverter.fromConnectData(record.topic(), headers, record.keySchema(), record.key()),
Stage.KEY_CONVERTER, keyConverter.getClass());
byte[] key = retryWithToleranceOperator.execute(context, () -> keyConverterPlugin.get().fromConnectData(record.topic(), headers, record.keySchema(), record.key()),
Stage.KEY_CONVERTER, keyConverterPlugin.get().getClass());
byte[] value = retryWithToleranceOperator.execute(context, () -> valueConverter.fromConnectData(record.topic(), headers, record.valueSchema(), record.value()),
Stage.VALUE_CONVERTER, valueConverter.getClass());
byte[] value = retryWithToleranceOperator.execute(context, () -> valueConverterPlugin.get().fromConnectData(record.topic(), headers, record.valueSchema(), record.value()),
Stage.VALUE_CONVERTER, valueConverterPlugin.get().getClass());
if (context.failed()) {
return null;
@ -545,7 +551,7 @@ public abstract class AbstractWorkerSourceTask extends WorkerTask<SourceRecord,
String topic = record.topic();
for (Header header : headers) {
String key = header.key();
byte[] rawHeader = headerConverter.fromConnectHeader(topic, key, header.schema(), header.value());
byte[] rawHeader = headerConverterPlugin.get().fromConnectHeader(topic, key, header.schema(), header.value());
result.add(key, rawHeader);
}
}

View File

@ -19,6 +19,7 @@ package org.apache.kafka.connect.runtime;
import org.apache.kafka.clients.CommonClientConfigs;
import org.apache.kafka.common.MetricName;
import org.apache.kafka.common.MetricNameTemplate;
import org.apache.kafka.common.internals.Plugin;
import org.apache.kafka.common.metrics.Gauge;
import org.apache.kafka.common.metrics.KafkaMetricsContext;
import org.apache.kafka.common.metrics.MetricConfig;
@ -27,9 +28,16 @@ import org.apache.kafka.common.metrics.MetricsContext;
import org.apache.kafka.common.metrics.MetricsReporter;
import org.apache.kafka.common.metrics.Sensor;
import org.apache.kafka.common.metrics.internals.MetricsUtils;
import org.apache.kafka.common.metrics.internals.PluginMetricsImpl;
import org.apache.kafka.common.utils.AppInfoParser;
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.connect.connector.ConnectRecord;
import org.apache.kafka.connect.runtime.distributed.DistributedConfig;
import org.apache.kafka.connect.storage.Converter;
import org.apache.kafka.connect.storage.HeaderConverter;
import org.apache.kafka.connect.transforms.Transformation;
import org.apache.kafka.connect.transforms.predicates.Predicate;
import org.apache.kafka.connect.util.ConnectorTaskId;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -45,6 +53,7 @@ import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.TimeUnit;
import java.util.function.Supplier;
/**
* The Connect metrics with configurable {@link MetricsReporter}s.
@ -166,6 +175,74 @@ public class ConnectMetrics {
AppInfoParser.unregisterAppInfo(JMX_PREFIX, workerId, metrics);
}
public PluginMetricsImpl connectorPluginMetrics(String connectorId) {
return new PluginMetricsImpl(metrics, connectorPluginTags(connectorId));
}
private static Map<String, String> connectorPluginTags(String connectorId) {
Map<String, String> tags = new LinkedHashMap<>();
tags.put("connector", connectorId);
return tags;
}
PluginMetricsImpl taskPluginMetrics(ConnectorTaskId connectorTaskId) {
return new PluginMetricsImpl(metrics, taskPluginTags(connectorTaskId));
}
private static Map<String, String> taskPluginTags(ConnectorTaskId connectorTaskId) {
Map<String, String> tags = connectorPluginTags(connectorTaskId.connector());
tags.put("task", String.valueOf(connectorTaskId.task()));
return tags;
}
private static Supplier<Map<String, String>> converterPluginTags(ConnectorTaskId connectorTaskId, boolean isKey) {
return () -> {
Map<String, String> tags = taskPluginTags(connectorTaskId);
tags.put("converter", isKey ? "key" : "value");
return tags;
};
}
private static Supplier<Map<String, String>> headerConverterPluginTags(ConnectorTaskId connectorTaskId) {
return () -> {
Map<String, String> tags = taskPluginTags(connectorTaskId);
tags.put("converter", "header");
return tags;
};
}
private static Supplier<Map<String, String>> transformationPluginTags(ConnectorTaskId connectorTaskId, String transformationAlias) {
return () -> {
Map<String, String> tags = taskPluginTags(connectorTaskId);
tags.put("transformation", transformationAlias);
return tags;
};
}
private static Supplier<Map<String, String>> predicatePluginTags(ConnectorTaskId connectorTaskId, String predicateAlias) {
return () -> {
Map<String, String> tags = taskPluginTags(connectorTaskId);
tags.put("predicate", predicateAlias);
return tags;
};
}
public Plugin<HeaderConverter> wrap(HeaderConverter headerConverter, ConnectorTaskId connectorTaskId) {
return Plugin.wrapInstance(headerConverter, metrics, headerConverterPluginTags(connectorTaskId));
}
public Plugin<Converter> wrap(Converter converter, ConnectorTaskId connectorTaskId, boolean isKey) {
return Plugin.wrapInstance(converter, metrics, converterPluginTags(connectorTaskId, isKey));
}
public <R extends ConnectRecord<R>> Plugin<Transformation<R>> wrap(Transformation<R> transformation, ConnectorTaskId connectorTaskId, String alias) {
return Plugin.wrapInstance(transformation, metrics, transformationPluginTags(connectorTaskId, alias));
}
public <R extends ConnectRecord<R>> Plugin<Predicate<R>> wrap(Predicate<R> predicate, ConnectorTaskId connectorTaskId, String alias) {
return Plugin.wrapInstance(predicate, metrics, predicatePluginTags(connectorTaskId, alias));
}
public static class MetricGroupId {
private final String groupName;
private final Map<String, String> tags;

View File

@ -22,6 +22,7 @@ import org.apache.kafka.common.config.ConfigDef.Importance;
import org.apache.kafka.common.config.ConfigDef.Type;
import org.apache.kafka.common.config.ConfigDef.Width;
import org.apache.kafka.common.config.ConfigException;
import org.apache.kafka.common.internals.Plugin;
import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.connect.connector.ConnectRecord;
import org.apache.kafka.connect.errors.ConnectException;
@ -37,6 +38,7 @@ import org.apache.kafka.connect.storage.HeaderConverter;
import org.apache.kafka.connect.transforms.Transformation;
import org.apache.kafka.connect.transforms.predicates.Predicate;
import org.apache.kafka.connect.util.ConcreteSubClassValidator;
import org.apache.kafka.connect.util.ConnectorTaskId;
import org.apache.kafka.connect.util.InstantiableClassValidator;
import org.apache.maven.artifact.versioning.InvalidVersionSpecificationException;
@ -360,7 +362,7 @@ public class ConnectorConfig extends AbstractConfig {
* {@link Transformation transformations} and {@link Predicate predicates}
* as they are specified in the {@link #TRANSFORMS_CONFIG} and {@link #PREDICATES_CONFIG}
*/
public <R extends ConnectRecord<R>> List<TransformationStage<R>> transformationStages() {
public <R extends ConnectRecord<R>> List<TransformationStage<R>> transformationStages(ConnectorTaskId connectorTaskId, ConnectMetrics metrics) {
final List<String> transformAliases = getList(TRANSFORMS_CONFIG);
final List<TransformationStage<R>> transformations = new ArrayList<>(transformAliases.size());
@ -374,14 +376,16 @@ public class ConnectorConfig extends AbstractConfig {
Object predicateAlias = configs.remove(TransformationStage.PREDICATE_CONFIG);
Object negate = configs.remove(TransformationStage.NEGATE_CONFIG);
transformation.configure(configs);
Plugin<Transformation<R>> transformationPlugin = metrics.wrap(transformation, connectorTaskId, alias);
if (predicateAlias != null) {
String predicatePrefix = PREDICATES_PREFIX + predicateAlias + ".";
@SuppressWarnings("unchecked")
Predicate<R> predicate = Utils.newInstance(getClass(predicatePrefix + "type"), Predicate.class);
predicate.configure(originalsWithPrefix(predicatePrefix));
transformations.add(new TransformationStage<>(predicate, negate != null && Boolean.parseBoolean(negate.toString()), transformation));
Plugin<Predicate<R>> predicatePlugin = metrics.wrap(predicate, connectorTaskId, (String) predicateAlias);
transformations.add(new TransformationStage<>(predicatePlugin, negate != null && Boolean.parseBoolean(negate.toString()), transformationPlugin));
} else {
transformations.add(new TransformationStage<>(transformation));
transformations.add(new TransformationStage<>(transformationPlugin));
}
} catch (Exception e) {
throw new ConnectException(e);

View File

@ -20,6 +20,7 @@ import org.apache.kafka.clients.producer.Producer;
import org.apache.kafka.clients.producer.ProducerRecord;
import org.apache.kafka.clients.producer.RecordMetadata;
import org.apache.kafka.common.errors.InvalidProducerEpochException;
import org.apache.kafka.common.internals.Plugin;
import org.apache.kafka.common.metrics.Sensor;
import org.apache.kafka.common.metrics.stats.Avg;
import org.apache.kafka.common.metrics.stats.Max;
@ -78,9 +79,9 @@ class ExactlyOnceWorkerSourceTask extends AbstractWorkerSourceTask {
SourceTask task,
TaskStatus.Listener statusListener,
TargetState initialState,
Converter keyConverter,
Converter valueConverter,
HeaderConverter headerConverter,
Plugin<Converter> keyConverterPlugin,
Plugin<Converter> valueConverterPlugin,
Plugin<HeaderConverter> headerConverterPlugin,
TransformationChain<SourceRecord, SourceRecord> transformationChain,
Producer<byte[], byte[]> producer,
TopicAdmin admin,
@ -101,8 +102,8 @@ class ExactlyOnceWorkerSourceTask extends AbstractWorkerSourceTask {
Runnable preProducerCheck,
Runnable postProducerCheck,
Supplier<List<ErrorReporter<SourceRecord>>> errorReportersSupplier) {
super(id, task, statusListener, initialState, keyConverter, valueConverter, headerConverter, transformationChain,
new WorkerSourceTaskContext(offsetReader, id, configState, buildTransactionContext(sourceConfig)),
super(id, task, statusListener, initialState, configState, keyConverterPlugin, valueConverterPlugin, headerConverterPlugin, transformationChain,
buildTransactionContext(sourceConfig),
producer, admin, topicGroups, offsetReader, offsetWriter, offsetStore, workerConfig, connectMetrics, errorMetrics,
loader, time, retryWithToleranceOperator, statusBackingStore, closeExecutor, errorReportersSupplier);

View File

@ -381,6 +381,12 @@ public interface Herder {
*/
void setClusterLoggerLevel(String namespace, String level);
/**
* Get the ConnectMetrics from the worker for this herder
* @return the ConnectMetrics
*/
ConnectMetrics connectMetrics();
enum ConfigReloadAction {
NONE,
RESTART

View File

@ -16,6 +16,9 @@
*/
package org.apache.kafka.connect.runtime;
import org.apache.kafka.common.metrics.PluginMetrics;
import org.apache.kafka.common.metrics.internals.PluginMetricsImpl;
import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.connect.errors.ConnectException;
import org.slf4j.Logger;
@ -30,11 +33,13 @@ public class HerderConnectorContext implements CloseableConnectorContext {
private final AbstractHerder herder;
private final String connectorName;
private final PluginMetricsImpl pluginMetrics;
private volatile boolean closed;
public HerderConnectorContext(AbstractHerder herder, String connectorName) {
public HerderConnectorContext(AbstractHerder herder, String connectorName, PluginMetricsImpl pluginMetrics) {
this.herder = herder;
this.connectorName = connectorName;
this.pluginMetrics = pluginMetrics;
this.closed = false;
}
@ -63,8 +68,14 @@ public class HerderConnectorContext implements CloseableConnectorContext {
herder.onFailure(connectorName, e);
}
@Override
public PluginMetrics pluginMetrics() {
return pluginMetrics;
}
@Override
public void close() {
Utils.closeQuietly(pluginMetrics, "Plugin metrics for " + connectorName);
closed = true;
}
}

View File

@ -17,6 +17,7 @@
package org.apache.kafka.connect.runtime;
import org.apache.kafka.common.internals.Plugin;
import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.connect.connector.ConnectRecord;
import org.apache.kafka.connect.transforms.Transformation;
@ -32,44 +33,44 @@ public class TransformationStage<R extends ConnectRecord<R>> implements AutoClos
static final String PREDICATE_CONFIG = "predicate";
static final String NEGATE_CONFIG = "negate";
private final Predicate<R> predicate;
private final Transformation<R> transformation;
private final Plugin<Predicate<R>> predicatePlugin;
private final Plugin<Transformation<R>> transformationPlugin;
private final boolean negate;
TransformationStage(Transformation<R> transformation) {
this(null, false, transformation);
TransformationStage(Plugin<Transformation<R>> transformationPlugin) {
this(null, false, transformationPlugin);
}
TransformationStage(Predicate<R> predicate, boolean negate, Transformation<R> transformation) {
this.predicate = predicate;
TransformationStage(Plugin<Predicate<R>> predicatePlugin, boolean negate, Plugin<Transformation<R>> transformationPlugin) {
this.predicatePlugin = predicatePlugin;
this.negate = negate;
this.transformation = transformation;
this.transformationPlugin = transformationPlugin;
}
public Class<? extends Transformation<R>> transformClass() {
@SuppressWarnings("unchecked")
Class<? extends Transformation<R>> transformClass = (Class<? extends Transformation<R>>) transformation.getClass();
Class<? extends Transformation<R>> transformClass = (Class<? extends Transformation<R>>) transformationPlugin.get().getClass();
return transformClass;
}
public R apply(R record) {
if (predicate == null || negate ^ predicate.test(record)) {
return transformation.apply(record);
if (predicatePlugin == null || predicatePlugin.get() == null || negate ^ predicatePlugin.get().test(record)) {
return transformationPlugin.get().apply(record);
}
return record;
}
@Override
public void close() {
Utils.closeQuietly(transformation, "transformation");
Utils.closeQuietly(predicate, "predicate");
Utils.closeQuietly(transformationPlugin, "transformation");
Utils.closeQuietly(predicatePlugin, "predicate");
}
@Override
public String toString() {
return "TransformationStage{" +
"predicate=" + predicate +
", transformation=" + transformation +
"predicate=" + predicatePlugin.get() +
", transformation=" + transformationPlugin.get() +
", negate=" + negate +
'}';
}

View File

@ -44,6 +44,7 @@ import org.apache.kafka.common.errors.GroupIdNotFoundException;
import org.apache.kafka.common.errors.GroupNotEmptyException;
import org.apache.kafka.common.errors.GroupSubscribedToTopicException;
import org.apache.kafka.common.errors.UnknownMemberIdException;
import org.apache.kafka.common.internals.Plugin;
import org.apache.kafka.common.utils.ThreadUtils;
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.common.utils.Timer;
@ -276,6 +277,8 @@ public final class Worker {
workerConfigTransformer.close();
ThreadUtils.shutdownExecutorServiceQuietly(executor, EXECUTOR_SHUTDOWN_TERMINATION_TIMEOUT_MS, TimeUnit.MILLISECONDS);
Utils.closeQuietly(internalKeyConverter, "internal key converter");
Utils.closeQuietly(internalValueConverter, "internal value converter");
}
public WorkerConfig config() {
@ -699,9 +702,9 @@ public final class Worker {
workerTask = taskBuilder
.withTask(task)
.withConnectorConfig(connConfig)
.withKeyConverter(keyConverter)
.withValueConverter(valueConverter)
.withHeaderConverter(headerConverter)
.withKeyConverterPlugin(metrics.wrap(keyConverter, id, true))
.withValueConverterPlugin(metrics.wrap(valueConverter, id, false))
.withHeaderConverterPlugin(metrics.wrap(headerConverter, id))
.withClassloader(connectorLoader)
.build();
@ -1734,9 +1737,9 @@ public final class Worker {
private Task task = null;
private ConnectorConfig connectorConfig = null;
private Converter keyConverter = null;
private Converter valueConverter = null;
private HeaderConverter headerConverter = null;
private Plugin<Converter> keyConverterPlugin = null;
private Plugin<Converter> valueConverterPlugin = null;
private Plugin<HeaderConverter> headerConverterPlugin = null;
private ClassLoader classLoader = null;
public TaskBuilder(ConnectorTaskId id,
@ -1759,18 +1762,18 @@ public final class Worker {
return this;
}
public TaskBuilder<T, R> withKeyConverter(Converter keyConverter) {
this.keyConverter = keyConverter;
public TaskBuilder<T, R> withKeyConverterPlugin(Plugin<Converter> keyConverterPlugin) {
this.keyConverterPlugin = keyConverterPlugin;
return this;
}
public TaskBuilder<T, R> withValueConverter(Converter valueConverter) {
this.valueConverter = valueConverter;
public TaskBuilder<T, R> withValueConverterPlugin(Plugin<Converter> valueConverterPlugin) {
this.valueConverterPlugin = valueConverterPlugin;
return this;
}
public TaskBuilder<T, R> withHeaderConverter(HeaderConverter headerConverter) {
this.headerConverter = headerConverter;
public TaskBuilder<T, R> withHeaderConverterPlugin(Plugin<HeaderConverter> headerConverterPlugin) {
this.headerConverterPlugin = headerConverterPlugin;
return this;
}
@ -1782,9 +1785,9 @@ public final class Worker {
public WorkerTask<T, R> build() {
Objects.requireNonNull(task, "Task cannot be null");
Objects.requireNonNull(connectorConfig, "Connector config used by task cannot be null");
Objects.requireNonNull(keyConverter, "Key converter used by task cannot be null");
Objects.requireNonNull(valueConverter, "Value converter used by task cannot be null");
Objects.requireNonNull(headerConverter, "Header converter used by task cannot be null");
Objects.requireNonNull(keyConverterPlugin.get(), "Key converter used by task cannot be null");
Objects.requireNonNull(valueConverterPlugin.get(), "Value converter used by task cannot be null");
Objects.requireNonNull(headerConverterPlugin.get(), "Header converter used by task cannot be null");
Objects.requireNonNull(classLoader, "Classloader used by task cannot be null");
ErrorHandlingMetrics errorHandlingMetrics = errorHandlingMetrics(id);
@ -1794,11 +1797,11 @@ public final class Worker {
RetryWithToleranceOperator<T> retryWithToleranceOperator = new RetryWithToleranceOperator<>(connectorConfig.errorRetryTimeout(),
connectorConfig.errorMaxDelayInMillis(), connectorConfig.errorToleranceType(), Time.SYSTEM, errorHandlingMetrics);
TransformationChain<T, R> transformationChain = new TransformationChain<>(connectorConfig.<R>transformationStages(), retryWithToleranceOperator);
TransformationChain<T, R> transformationChain = new TransformationChain<>(connectorConfig.<R>transformationStages(id, metrics), retryWithToleranceOperator);
log.info("Initializing: {}", transformationChain);
return doBuild(task, id, configState, statusListener, initialState,
connectorConfig, keyConverter, valueConverter, headerConverter, classLoader,
connectorConfig, keyConverterPlugin, valueConverterPlugin, headerConverterPlugin, classLoader,
retryWithToleranceOperator, transformationChain,
errorHandlingMetrics, connectorClass);
}
@ -1810,9 +1813,9 @@ public final class Worker {
TaskStatus.Listener statusListener,
TargetState initialState,
ConnectorConfig connectorConfig,
Converter keyConverter,
Converter valueConverter,
HeaderConverter headerConverter,
Plugin<Converter> keyConverterPlugin,
Plugin<Converter> valueConverterPlugin,
Plugin<HeaderConverter> headerConverterPlugin,
ClassLoader classLoader,
RetryWithToleranceOperator<T> retryWithToleranceOperator,
TransformationChain<T, R> transformationChain,
@ -1838,9 +1841,9 @@ public final class Worker {
TaskStatus.Listener statusListener,
TargetState initialState,
ConnectorConfig connectorConfig,
Converter keyConverter,
Converter valueConverter,
HeaderConverter headerConverter,
Plugin<Converter> keyConverterPlugin,
Plugin<Converter> valueConverterPlugin,
Plugin<HeaderConverter> headerConverterPlugin,
ClassLoader classLoader,
RetryWithToleranceOperator<ConsumerRecord<byte[], byte[]>> retryWithToleranceOperator,
TransformationChain<ConsumerRecord<byte[], byte[]>, SinkRecord> transformationChain,
@ -1849,15 +1852,15 @@ public final class Worker {
) {
SinkConnectorConfig sinkConfig = new SinkConnectorConfig(plugins, connectorConfig.originalsStrings());
WorkerErrantRecordReporter workerErrantRecordReporter = createWorkerErrantRecordReporter(sinkConfig, retryWithToleranceOperator,
keyConverter, valueConverter, headerConverter);
keyConverterPlugin.get(), valueConverterPlugin.get(), headerConverterPlugin.get());
Map<String, Object> consumerProps = baseConsumerConfigs(
id.connector(), "connector-consumer-" + id, config, connectorConfig, connectorClass,
connectorClientConfigOverridePolicy, kafkaClusterId, ConnectorType.SINK);
KafkaConsumer<byte[], byte[]> consumer = new KafkaConsumer<>(consumerProps);
return new WorkerSinkTask(id, (SinkTask) task, statusListener, initialState, config, configState, metrics, keyConverter,
valueConverter, errorHandlingMetrics, headerConverter, transformationChain, consumer, classLoader, time,
return new WorkerSinkTask(id, (SinkTask) task, statusListener, initialState, config, configState, metrics, keyConverterPlugin,
valueConverterPlugin, errorHandlingMetrics, headerConverterPlugin, transformationChain, consumer, classLoader, time,
retryWithToleranceOperator, workerErrantRecordReporter, herder.statusBackingStore(),
() -> sinkTaskReporters(id, sinkConfig, errorHandlingMetrics, connectorClass));
}
@ -1879,9 +1882,9 @@ public final class Worker {
TaskStatus.Listener statusListener,
TargetState initialState,
ConnectorConfig connectorConfig,
Converter keyConverter,
Converter valueConverter,
HeaderConverter headerConverter,
Plugin<Converter> keyConverterPlugin,
Plugin<Converter> valueConverterPlugin,
Plugin<HeaderConverter> headerConverterPlugin,
ClassLoader classLoader,
RetryWithToleranceOperator<SourceRecord> retryWithToleranceOperator,
TransformationChain<SourceRecord, SourceRecord> transformationChain,
@ -1916,8 +1919,8 @@ public final class Worker {
OffsetStorageWriter offsetWriter = new OffsetStorageWriter(offsetStore, id.connector(), internalKeyConverter, internalValueConverter);
// Note we pass the configState as it performs dynamic transformations under the covers
return new WorkerSourceTask(id, (SourceTask) task, statusListener, initialState, keyConverter, valueConverter, errorHandlingMetrics,
headerConverter, transformationChain, producer, topicAdmin, topicCreationGroups,
return new WorkerSourceTask(id, (SourceTask) task, statusListener, initialState, keyConverterPlugin, valueConverterPlugin, errorHandlingMetrics,
headerConverterPlugin, transformationChain, producer, topicAdmin, topicCreationGroups,
offsetReader, offsetWriter, offsetStore, config, configState, metrics, classLoader, time,
retryWithToleranceOperator, herder.statusBackingStore(), executor, () -> sourceTaskReporters(id, sourceConfig, errorHandlingMetrics));
}
@ -1946,9 +1949,9 @@ public final class Worker {
TaskStatus.Listener statusListener,
TargetState initialState,
ConnectorConfig connectorConfig,
Converter keyConverter,
Converter valueConverter,
HeaderConverter headerConverter,
Plugin<Converter> keyConverterPlugin,
Plugin<Converter> valueConverterPlugin,
Plugin<HeaderConverter> headerConverterPlugin,
ClassLoader classLoader,
RetryWithToleranceOperator<SourceRecord> retryWithToleranceOperator,
TransformationChain<SourceRecord, SourceRecord> transformationChain,
@ -1980,8 +1983,8 @@ public final class Worker {
OffsetStorageWriter offsetWriter = new OffsetStorageWriter(offsetStore, id.connector(), internalKeyConverter, internalValueConverter);
// Note we pass the configState as it performs dynamic transformations under the covers
return new ExactlyOnceWorkerSourceTask(id, (SourceTask) task, statusListener, initialState, keyConverter, valueConverter,
headerConverter, transformationChain, producer, topicAdmin, topicCreationGroups,
return new ExactlyOnceWorkerSourceTask(id, (SourceTask) task, statusListener, initialState, keyConverterPlugin, valueConverterPlugin,
headerConverterPlugin, transformationChain, producer, topicAdmin, topicCreationGroups,
offsetReader, offsetWriter, offsetStore, config, configState, metrics, errorHandlingMetrics, classLoader, time, retryWithToleranceOperator,
herder.statusBackingStore(), sourceConfig, executor, preProducerCheck, postProducerCheck,
() -> sourceTaskReporters(id, sourceConfig, errorHandlingMetrics));

View File

@ -16,6 +16,7 @@
*/
package org.apache.kafka.connect.runtime;
import org.apache.kafka.common.metrics.PluginMetrics;
import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.connect.connector.Connector;
import org.apache.kafka.connect.connector.ConnectorContext;
@ -86,7 +87,7 @@ public class WorkerConnector implements Runnable {
Connector connector,
ConnectorConfig connectorConfig,
CloseableConnectorContext ctx,
ConnectMetrics metrics,
ConnectMetrics connectMetrics,
ConnectorStatus.Listener statusListener,
CloseableOffsetStorageReader offsetStorageReader,
ConnectorOffsetBackingStore offsetStore,
@ -97,7 +98,7 @@ public class WorkerConnector implements Runnable {
this.ctx = ctx;
this.connector = connector;
this.state = State.INIT;
this.metrics = new ConnectorMetricsGroup(metrics, AbstractStatus.State.UNASSIGNED, statusListener);
this.metrics = new ConnectorMetricsGroup(connectMetrics, AbstractStatus.State.UNASSIGNED, statusListener);
this.statusListener = this.metrics;
this.offsetStorageReader = offsetStorageReader;
this.offsetStore = offsetStore;
@ -582,6 +583,11 @@ public class WorkerConnector implements Runnable {
onFailure(e);
WorkerConnector.this.ctx.raiseError(e);
}
@Override
public PluginMetrics pluginMetrics() {
return WorkerConnector.this.ctx.pluginMetrics();
}
}
private class WorkerSinkConnectorContext extends WorkerConnectorContext implements SinkConnectorContext {

View File

@ -25,6 +25,7 @@ import org.apache.kafka.clients.consumer.OffsetCommitCallback;
import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.errors.WakeupException;
import org.apache.kafka.common.internals.Plugin;
import org.apache.kafka.common.metrics.Sensor;
import org.apache.kafka.common.metrics.stats.Avg;
import org.apache.kafka.common.metrics.stats.CumulativeSum;
@ -82,9 +83,9 @@ class WorkerSinkTask extends WorkerTask<ConsumerRecord<byte[], byte[]>, SinkReco
private final SinkTask task;
private final ClusterConfigState configState;
private Map<String, String> taskConfig;
private final Converter keyConverter;
private final Converter valueConverter;
private final HeaderConverter headerConverter;
private final Plugin<Converter> keyConverterPlugin;
private final Plugin<Converter> valueConverterPlugin;
private final Plugin<HeaderConverter> headerConverterPlugin;
private final SinkTaskMetricsGroup sinkTaskMetricsGroup;
private final boolean isTopicTrackingEnabled;
private final Consumer<byte[], byte[]> consumer;
@ -110,10 +111,10 @@ class WorkerSinkTask extends WorkerTask<ConsumerRecord<byte[], byte[]>, SinkReco
WorkerConfig workerConfig,
ClusterConfigState configState,
ConnectMetrics connectMetrics,
Converter keyConverter,
Converter valueConverter,
Plugin<Converter> keyConverterPlugin,
Plugin<Converter> valueConverterPlugin,
ErrorHandlingMetrics errorMetrics,
HeaderConverter headerConverter,
Plugin<HeaderConverter> headerConverterPlugin,
TransformationChain<ConsumerRecord<byte[], byte[]>, SinkRecord> transformationChain,
Consumer<byte[], byte[]> consumer,
ClassLoader loader,
@ -128,9 +129,9 @@ class WorkerSinkTask extends WorkerTask<ConsumerRecord<byte[], byte[]>, SinkReco
this.workerConfig = workerConfig;
this.task = task;
this.configState = configState;
this.keyConverter = keyConverter;
this.valueConverter = valueConverter;
this.headerConverter = headerConverter;
this.keyConverterPlugin = keyConverterPlugin;
this.valueConverterPlugin = valueConverterPlugin;
this.headerConverterPlugin = headerConverterPlugin;
this.messageBatch = new ArrayList<>();
this.lastCommittedOffsets = new HashMap<>();
this.currentOffsets = new HashMap<>();
@ -180,7 +181,10 @@ class WorkerSinkTask extends WorkerTask<ConsumerRecord<byte[], byte[]>, SinkReco
}
taskStopped = true;
Utils.closeQuietly(consumer, "consumer");
Utils.closeQuietly(headerConverter, "header converter");
Utils.closeQuietly(headerConverterPlugin, "header converter");
Utils.closeQuietly(keyConverterPlugin, "key converter");
Utils.closeQuietly(valueConverterPlugin, "value converter");
Utils.closeQuietly(pluginMetrics, "plugin metrics");
/*
Setting partition count explicitly to 0 to handle the case,
when the task fails, which would cause its consumer to leave the group.
@ -535,13 +539,13 @@ class WorkerSinkTask extends WorkerTask<ConsumerRecord<byte[], byte[]>, SinkReco
}
private SinkRecord convertAndTransformRecord(ProcessingContext<ConsumerRecord<byte[], byte[]>> context, final ConsumerRecord<byte[], byte[]> msg) {
SchemaAndValue keyAndSchema = retryWithToleranceOperator.execute(context, () -> keyConverter.toConnectData(msg.topic(), msg.headers(), msg.key()),
Stage.KEY_CONVERTER, keyConverter.getClass());
SchemaAndValue keyAndSchema = retryWithToleranceOperator.execute(context, () -> keyConverterPlugin.get().toConnectData(msg.topic(), msg.headers(), msg.key()),
Stage.KEY_CONVERTER, keyConverterPlugin.get().getClass());
SchemaAndValue valueAndSchema = retryWithToleranceOperator.execute(context, () -> valueConverter.toConnectData(msg.topic(), msg.headers(), msg.value()),
Stage.VALUE_CONVERTER, valueConverter.getClass());
SchemaAndValue valueAndSchema = retryWithToleranceOperator.execute(context, () -> valueConverterPlugin.get().toConnectData(msg.topic(), msg.headers(), msg.value()),
Stage.VALUE_CONVERTER, valueConverterPlugin.get().getClass());
Headers headers = retryWithToleranceOperator.execute(context, () -> convertHeadersFor(msg), Stage.HEADER_CONVERTER, headerConverter.getClass());
Headers headers = retryWithToleranceOperator.execute(context, () -> convertHeadersFor(msg), Stage.HEADER_CONVERTER, headerConverterPlugin.get().getClass());
if (context.failed()) {
return null;
@ -576,7 +580,7 @@ class WorkerSinkTask extends WorkerTask<ConsumerRecord<byte[], byte[]>, SinkReco
if (recordHeaders != null) {
String topic = record.topic();
for (org.apache.kafka.common.header.Header recordHeader : recordHeaders) {
SchemaAndValue schemaAndValue = headerConverter.toConnectHeader(topic, recordHeader.key(), recordHeader.value());
SchemaAndValue schemaAndValue = headerConverterPlugin.get().toConnectHeader(topic, recordHeader.key(), recordHeader.value());
result.add(recordHeader.key(), schemaAndValue);
}
}

View File

@ -18,6 +18,7 @@ package org.apache.kafka.connect.runtime;
import org.apache.kafka.clients.consumer.Consumer;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.metrics.PluginMetrics;
import org.apache.kafka.connect.errors.IllegalWorkerStateException;
import org.apache.kafka.connect.sink.ErrantRecordReporter;
import org.apache.kafka.connect.sink.SinkTaskContext;
@ -165,6 +166,11 @@ public class WorkerSinkTaskContext implements SinkTaskContext {
return sinkTask.workerErrantRecordReporter();
}
@Override
public PluginMetrics pluginMetrics() {
return sinkTask.pluginMetrics();
}
@Override
public String toString() {
return "WorkerSinkTaskContext{" +

View File

@ -19,6 +19,7 @@ package org.apache.kafka.connect.runtime;
import org.apache.kafka.clients.producer.Producer;
import org.apache.kafka.clients.producer.ProducerRecord;
import org.apache.kafka.clients.producer.RecordMetadata;
import org.apache.kafka.common.internals.Plugin;
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.connect.errors.ConnectException;
import org.apache.kafka.connect.runtime.errors.ErrorHandlingMetrics;
@ -71,10 +72,10 @@ class WorkerSourceTask extends AbstractWorkerSourceTask {
SourceTask task,
TaskStatus.Listener statusListener,
TargetState initialState,
Converter keyConverter,
Converter valueConverter,
Plugin<Converter> keyConverterPlugin,
Plugin<Converter> valueConverterPlugin,
ErrorHandlingMetrics errorMetrics,
HeaderConverter headerConverter,
Plugin<HeaderConverter> headerConverterPlugin,
TransformationChain<SourceRecord, SourceRecord> transformationChain,
Producer<byte[], byte[]> producer,
TopicAdmin admin,
@ -92,8 +93,8 @@ class WorkerSourceTask extends AbstractWorkerSourceTask {
Executor closeExecutor,
Supplier<List<ErrorReporter<SourceRecord>>> errorReportersSupplier) {
super(id, task, statusListener, initialState, keyConverter, valueConverter, headerConverter, transformationChain,
new WorkerSourceTaskContext(offsetReader, id, configState, null), producer,
super(id, task, statusListener, initialState, configState, keyConverterPlugin, valueConverterPlugin, headerConverterPlugin, transformationChain,
null, producer,
admin, topicGroups, offsetReader, offsetWriter, offsetStore, workerConfig, connectMetrics, errorMetrics, loader,
time, retryWithToleranceOperator, statusBackingStore, closeExecutor, errorReportersSupplier);

View File

@ -16,6 +16,7 @@
*/
package org.apache.kafka.connect.runtime;
import org.apache.kafka.common.metrics.PluginMetrics;
import org.apache.kafka.connect.source.SourceTaskContext;
import org.apache.kafka.connect.storage.ClusterConfigState;
import org.apache.kafka.connect.storage.OffsetStorageReader;
@ -29,15 +30,18 @@ public class WorkerSourceTaskContext implements SourceTaskContext {
private final ConnectorTaskId id;
private final ClusterConfigState configState;
private final WorkerTransactionContext transactionContext;
private final PluginMetrics pluginMetrics;
public WorkerSourceTaskContext(OffsetStorageReader reader,
ConnectorTaskId id,
ClusterConfigState configState,
WorkerTransactionContext transactionContext) {
WorkerTransactionContext transactionContext,
PluginMetrics pluginMetrics) {
this.reader = reader;
this.id = id;
this.configState = configState;
this.transactionContext = transactionContext;
this.pluginMetrics = pluginMetrics;
}
@Override
@ -54,4 +58,9 @@ public class WorkerSourceTaskContext implements SourceTaskContext {
public WorkerTransactionContext transactionContext() {
return transactionContext;
}
@Override
public PluginMetrics pluginMetrics() {
return pluginMetrics;
}
}

View File

@ -19,7 +19,9 @@ package org.apache.kafka.connect.runtime;
import org.apache.kafka.common.MetricName;
import org.apache.kafka.common.MetricNameTemplate;
import org.apache.kafka.common.metrics.Gauge;
import org.apache.kafka.common.metrics.PluginMetrics;
import org.apache.kafka.common.metrics.Sensor;
import org.apache.kafka.common.metrics.internals.PluginMetricsImpl;
import org.apache.kafka.common.metrics.stats.Avg;
import org.apache.kafka.common.metrics.stats.Frequencies;
import org.apache.kafka.common.metrics.stats.Max;
@ -76,6 +78,7 @@ abstract class WorkerTask<T, R extends ConnectRecord<R>> implements Runnable {
protected final RetryWithToleranceOperator<T> retryWithToleranceOperator;
protected final TransformationChain<T, R> transformationChain;
private final Supplier<List<ErrorReporter<T>>> errorReportersSupplier;
protected final PluginMetricsImpl pluginMetrics;
public WorkerTask(ConnectorTaskId id,
TaskStatus.Listener statusListener,
@ -103,6 +106,7 @@ abstract class WorkerTask<T, R extends ConnectRecord<R>> implements Runnable {
this.errorReportersSupplier = errorReportersSupplier;
this.time = time;
this.statusBackingStore = statusBackingStore;
this.pluginMetrics = connectMetrics.taskPluginMetrics(id);
}
public ConnectorTaskId id() {
@ -113,6 +117,10 @@ abstract class WorkerTask<T, R extends ConnectRecord<R>> implements Runnable {
return loader;
}
public PluginMetrics pluginMetrics() {
return pluginMetrics;
}
/**
* Initialize the task for execution.
*

View File

@ -2079,7 +2079,7 @@ public class DistributedHerder extends AbstractHerder implements Runnable {
private void startConnector(String connectorName, Callback<Void> callback) {
log.info("Starting connector {}", connectorName);
final Map<String, String> configProps = configState.connectorConfig(connectorName);
final CloseableConnectorContext ctx = new HerderConnectorContext(this, connectorName);
final CloseableConnectorContext ctx = new HerderConnectorContext(this, connectorName, worker.metrics().connectorPluginMetrics(connectorName));
final TargetState initialState = configState.targetState(connectorName);
final Callback<TargetState> onInitialStateChange = (error, newState) -> {
if (error != null) {

View File

@ -17,6 +17,7 @@
package org.apache.kafka.connect.runtime.rest;
import org.apache.kafka.common.config.ConfigException;
import org.apache.kafka.common.internals.Plugin;
import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.connect.errors.ConnectException;
import org.apache.kafka.connect.health.ConnectClusterDetails;
@ -96,7 +97,7 @@ public abstract class RestServer {
private final Server jettyServer;
private final RequestTimeout requestTimeout;
private List<ConnectRestExtension> connectRestExtensions = Collections.emptyList();
private List<Plugin<ConnectRestExtension>> connectRestExtensionPlugins = Collections.emptyList();
/**
* Create a REST server for this herder using the specified configs.
@ -217,10 +218,10 @@ public abstract class RestServer {
throw new ConnectException("Unable to initialize REST server", e);
}
log.info("REST server listening at " + jettyServer.getURI() + ", advertising URL " + advertisedUrl());
log.info("REST server listening at {}, advertising URL {}", jettyServer.getURI(), advertisedUrl());
URI adminUrl = adminUrl();
if (adminUrl != null)
log.info("REST admin endpoints at " + adminUrl);
log.info("REST admin endpoints at {}", adminUrl);
}
protected final void initializeResources() {
@ -370,11 +371,11 @@ public abstract class RestServer {
}
}
}
for (ConnectRestExtension connectRestExtension : connectRestExtensions) {
for (Plugin<ConnectRestExtension> connectRestExtensionPlugin : connectRestExtensionPlugins) {
try {
connectRestExtension.close();
connectRestExtensionPlugin.close();
} catch (IOException e) {
log.warn("Error while invoking close on " + connectRestExtension.getClass(), e);
log.warn("Error while invoking close on {}", connectRestExtensionPlugin.get().getClass(), e);
}
}
jettyServer.stop();
@ -504,9 +505,14 @@ public abstract class RestServer {
}
protected final void registerRestExtensions(Herder herder, ResourceConfig resourceConfig) {
connectRestExtensions = herder.plugins().newPlugins(
config.restExtensions(),
config, ConnectRestExtension.class);
connectRestExtensionPlugins = Plugin.wrapInstances(
herder.plugins().newPlugins(
config.restExtensions(),
config,
ConnectRestExtension.class
),
herder.connectMetrics().metrics(),
RestServerConfig.REST_EXTENSION_CLASSES_CONFIG);
long herderRequestTimeoutMs = DEFAULT_REST_REQUEST_TIMEOUT_MS;
@ -525,8 +531,8 @@ public abstract class RestServer {
new ConnectRestConfigurable(resourceConfig),
new ConnectClusterStateImpl(herderRequestTimeoutMs, connectClusterDetails, herder)
);
for (ConnectRestExtension connectRestExtension : connectRestExtensions) {
connectRestExtension.register(connectRestExtensionContext);
for (Plugin<ConnectRestExtension> connectRestExtensionPlugin : connectRestExtensionPlugins) {
connectRestExtensionPlugin.get().register(connectRestExtensionContext);
}
}

View File

@ -463,7 +463,7 @@ public final class StandaloneHerder extends AbstractHerder {
private void startConnector(String connName, Callback<TargetState> onStart) {
Map<String, String> connConfigs = configState.connectorConfig(connName);
TargetState targetState = configState.targetState(connName);
worker.startConnector(connName, connConfigs, new HerderConnectorContext(this, connName), this, targetState, onStart);
worker.startConnector(connName, connConfigs, new HerderConnectorContext(this, connName, worker.metrics().connectorPluginMetrics(connName)), this, targetState, onStart);
}
private List<Map<String, String>> recomputeTaskConfigs(String connName) {

View File

@ -323,9 +323,9 @@ public class BlockingConnectorTest {
normalConnectorHandle.expectedCommits(NUM_RECORDS_PRODUCED);
Map<String, String> props = new HashMap<>();
props.put(CONNECTOR_CLASS_CONFIG, MonitorableSourceConnector.class.getName());
props.put(CONNECTOR_CLASS_CONFIG, TestableSourceConnector.class.getName());
props.put(TASKS_MAX_CONFIG, "1");
props.put(MonitorableSourceConnector.TOPIC_CONFIG, TEST_TOPIC);
props.put(TestableSourceConnector.TOPIC_CONFIG, TEST_TOPIC);
log.info("Creating normal connector");
try {
connect.configureConnector(NORMAL_CONNECTOR_NAME, props);

View File

@ -79,7 +79,7 @@ import static org.apache.kafka.common.config.AbstractConfig.CONFIG_PROVIDERS_CON
import static org.apache.kafka.common.config.TopicConfig.DELETE_RETENTION_MS_CONFIG;
import static org.apache.kafka.common.config.TopicConfig.SEGMENT_MS_CONFIG;
import static org.apache.kafka.connect.integration.BlockingConnectorTest.TASK_STOP;
import static org.apache.kafka.connect.integration.MonitorableSourceConnector.TOPIC_CONFIG;
import static org.apache.kafka.connect.integration.TestableSourceConnector.TOPIC_CONFIG;
import static org.apache.kafka.connect.runtime.ConnectorConfig.CONNECTOR_CLASS_CONFIG;
import static org.apache.kafka.connect.runtime.ConnectorConfig.CONNECTOR_CLIENT_PRODUCER_OVERRIDES_PREFIX;
import static org.apache.kafka.connect.runtime.ConnectorConfig.HEADER_CONVERTER_CLASS_CONFIG;
@ -326,7 +326,7 @@ public class ConnectWorkerIntegrationTest {
// base connector props
Map<String, String> props = defaultSourceConnectorProps(TOPIC_NAME);
props.put(CONNECTOR_CLASS_CONFIG, MonitorableSourceConnector.class.getSimpleName());
props.put(CONNECTOR_CLASS_CONFIG, TestableSourceConnector.class.getSimpleName());
// start the connector with only one task
int initialNumTasks = 1;
@ -792,7 +792,7 @@ public class ConnectWorkerIntegrationTest {
private Map<String, String> defaultSinkConnectorProps(String topics) {
// setup props for the sink connector
Map<String, String> props = new HashMap<>();
props.put(CONNECTOR_CLASS_CONFIG, MonitorableSinkConnector.class.getSimpleName());
props.put(CONNECTOR_CLASS_CONFIG, TestableSinkConnector.class.getSimpleName());
props.put(TASKS_MAX_CONFIG, String.valueOf(NUM_TASKS));
props.put(TOPICS_CONFIG, topics);
@ -987,7 +987,7 @@ public class ConnectWorkerIntegrationTest {
int maxTasks = 1;
connectorProps.put(TASKS_MAX_CONFIG, Integer.toString(maxTasks));
int numTasks = 2;
connectorProps.put(MonitorableSourceConnector.NUM_TASKS, Integer.toString(numTasks));
connectorProps.put(TestableSourceConnector.NUM_TASKS, Integer.toString(numTasks));
connect.configureConnector(CONNECTOR_NAME, connectorProps);
// A connector that generates excessive tasks will be failed with an expected error message
@ -1019,7 +1019,7 @@ public class ConnectWorkerIntegrationTest {
);
for (int i = 0; i < numTasks; i++) {
Map<String, String> taskConfig = MonitorableSourceConnector.taskConfig(
Map<String, String> taskConfig = TestableSourceConnector.taskConfig(
connectorProps,
CONNECTOR_NAME,
i
@ -1070,7 +1070,7 @@ public class ConnectWorkerIntegrationTest {
);
numTasks++;
connectorProps.put(MonitorableSourceConnector.NUM_TASKS, Integer.toString(numTasks));
connectorProps.put(TestableSourceConnector.NUM_TASKS, Integer.toString(numTasks));
connect.configureConnector(CONNECTOR_NAME, connectorProps);
// A connector will be allowed to generate excessive tasks when tasks.max.enforce is set to false
@ -1081,7 +1081,7 @@ public class ConnectWorkerIntegrationTest {
);
numTasks = maxTasks;
connectorProps.put(MonitorableSourceConnector.NUM_TASKS, Integer.toString(numTasks));
connectorProps.put(TestableSourceConnector.NUM_TASKS, Integer.toString(numTasks));
connectorProps.put(TASKS_MAX_ENFORCE_CONFIG, "true");
connect.configureConnector(CONNECTOR_NAME, connectorProps);
@ -1092,7 +1092,7 @@ public class ConnectWorkerIntegrationTest {
);
numTasks = maxTasks + 1;
connectorProps.put(MonitorableSourceConnector.NUM_TASKS, Integer.toString(numTasks));
connectorProps.put(TestableSourceConnector.NUM_TASKS, Integer.toString(numTasks));
connect.configureConnector(CONNECTOR_NAME, connectorProps);
// A connector that generates excessive tasks after being reconfigured will be failed, but its existing tasks will continue running
@ -1386,7 +1386,7 @@ public class ConnectWorkerIntegrationTest {
final String sourceConnectorName = "plugins-alias-test-source";
Map<String, String> sourceConnectorConfig = new HashMap<>(baseConnectorConfig);
// Aliased source connector class
sourceConnectorConfig.put(CONNECTOR_CLASS_CONFIG, MonitorableSourceConnector.class.getSimpleName());
sourceConnectorConfig.put(CONNECTOR_CLASS_CONFIG, TestableSourceConnector.class.getSimpleName());
// Connector-specific properties
sourceConnectorConfig.put(TOPIC_CONFIG, topic);
sourceConnectorConfig.put("throughput", "10");
@ -1400,7 +1400,7 @@ public class ConnectWorkerIntegrationTest {
final String sinkConnectorName = "plugins-alias-test-sink";
Map<String, String> sinkConnectorConfig = new HashMap<>(baseConnectorConfig);
// Aliased sink connector class
sinkConnectorConfig.put(CONNECTOR_CLASS_CONFIG, MonitorableSinkConnector.class.getSimpleName());
sinkConnectorConfig.put(CONNECTOR_CLASS_CONFIG, TestableSinkConnector.class.getSimpleName());
// Connector-specific properties
sinkConnectorConfig.put(TOPICS_CONFIG, topic);
// Create the connector and ensure it and its tasks can start
@ -1412,7 +1412,7 @@ public class ConnectWorkerIntegrationTest {
private Map<String, String> defaultSourceConnectorProps(String topic) {
// setup props for the source connector
Map<String, String> props = new HashMap<>();
props.put(CONNECTOR_CLASS_CONFIG, MonitorableSourceConnector.class.getSimpleName());
props.put(CONNECTOR_CLASS_CONFIG, TestableSourceConnector.class.getSimpleName());
props.put(TASKS_MAX_CONFIG, String.valueOf(NUM_TASKS));
props.put(TOPIC_CONFIG, topic);
props.put("throughput", "10");

View File

@ -148,7 +148,7 @@ public class ConnectorClientPolicyIntegrationTest {
public Map<String, String> basicConnectorConfig() {
Map<String, String> props = new HashMap<>();
props.put(CONNECTOR_CLASS_CONFIG, MonitorableSinkConnector.class.getSimpleName());
props.put(CONNECTOR_CLASS_CONFIG, TestableSinkConnector.class.getSimpleName());
props.put(TASKS_MAX_CONFIG, String.valueOf(NUM_TASKS));
props.put(TOPICS_CONFIG, "test-topic");
props.put(KEY_CONVERTER_CLASS_CONFIG, StringConverter.class.getName());

View File

@ -42,7 +42,7 @@ import java.util.stream.Collectors;
import jakarta.ws.rs.core.Response;
import static org.apache.kafka.connect.integration.MonitorableSourceConnector.TOPIC_CONFIG;
import static org.apache.kafka.connect.integration.TestableSourceConnector.TOPIC_CONFIG;
import static org.apache.kafka.connect.runtime.ConnectorConfig.CONNECTOR_CLASS_CONFIG;
import static org.apache.kafka.connect.runtime.ConnectorConfig.KEY_CONVERTER_CLASS_CONFIG;
import static org.apache.kafka.connect.runtime.ConnectorConfig.TASKS_MAX_CONFIG;
@ -415,7 +415,7 @@ public class ConnectorRestartApiIntegrationTest {
private Map<String, String> defaultSourceConnectorProps(String topic) {
// setup up props for the source connector
Map<String, String> props = new HashMap<>();
props.put(CONNECTOR_CLASS_CONFIG, MonitorableSourceConnector.class.getSimpleName());
props.put(CONNECTOR_CLASS_CONFIG, TestableSourceConnector.class.getSimpleName());
props.put(TASKS_MAX_CONFIG, String.valueOf(NUM_TASKS));
props.put(TOPIC_CONFIG, topic);
props.put("throughput", "10");

View File

@ -46,7 +46,7 @@ import java.util.Set;
import java.util.stream.Collectors;
import java.util.stream.StreamSupport;
import static org.apache.kafka.connect.integration.MonitorableSourceConnector.TOPIC_CONFIG;
import static org.apache.kafka.connect.integration.TestableSourceConnector.TOPIC_CONFIG;
import static org.apache.kafka.connect.runtime.ConnectorConfig.CONNECTOR_CLASS_CONFIG;
import static org.apache.kafka.connect.runtime.ConnectorConfig.KEY_CONVERTER_CLASS_CONFIG;
import static org.apache.kafka.connect.runtime.ConnectorConfig.TASKS_MAX_CONFIG;
@ -296,7 +296,7 @@ public class ConnectorTopicsIntegrationTest {
private Map<String, String> defaultSourceConnectorProps(String topic) {
// setup up props for the source connector
Map<String, String> props = new HashMap<>();
props.put(CONNECTOR_CLASS_CONFIG, MonitorableSourceConnector.class.getSimpleName());
props.put(CONNECTOR_CLASS_CONFIG, TestableSourceConnector.class.getSimpleName());
props.put(TASKS_MAX_CONFIG, String.valueOf(NUM_TASKS));
props.put(TOPIC_CONFIG, topic);
props.put("throughput", String.valueOf(10));
@ -311,7 +311,7 @@ public class ConnectorTopicsIntegrationTest {
private Map<String, String> defaultSinkConnectorProps(String... topics) {
// setup up props for the sink connector
Map<String, String> props = new HashMap<>();
props.put(CONNECTOR_CLASS_CONFIG, MonitorableSinkConnector.class.getSimpleName());
props.put(CONNECTOR_CLASS_CONFIG, TestableSinkConnector.class.getSimpleName());
props.put(TASKS_MAX_CONFIG, String.valueOf(NUM_TASKS));
props.put(TOPICS_CONFIG, String.join(",", topics));
props.put(KEY_CONVERTER_CLASS_CONFIG, StringConverter.class.getName());

View File

@ -39,7 +39,7 @@ import java.util.HashMap;
import java.util.Map;
import static org.apache.kafka.clients.consumer.ConsumerConfig.GROUP_ID_CONFIG;
import static org.apache.kafka.connect.integration.MonitorableSourceConnector.TOPIC_CONFIG;
import static org.apache.kafka.connect.integration.TestableSourceConnector.TOPIC_CONFIG;
import static org.apache.kafka.connect.runtime.ConnectorConfig.CONNECTOR_CLASS_CONFIG;
import static org.apache.kafka.connect.runtime.ConnectorConfig.CONNECTOR_CLIENT_CONSUMER_OVERRIDES_PREFIX;
import static org.apache.kafka.connect.runtime.ConnectorConfig.HEADER_CONVERTER_CLASS_CONFIG;
@ -228,7 +228,7 @@ public class ConnectorValidationIntegrationTest {
Map<String, String> config = defaultSinkConnectorProps();
String transformName = "t";
config.put(TRANSFORMS_CONFIG, transformName);
config.put(TRANSFORMS_CONFIG + "." + transformName + ".type", MonitorableSinkConnector.class.getName());
config.put(TRANSFORMS_CONFIG + "." + transformName + ".type", TestableSinkConnector.class.getName());
connect.assertions().assertExactlyNumErrorsOnConnectorConfigValidation(
config.get(CONNECTOR_CLASS_CONFIG),
config,
@ -289,7 +289,7 @@ public class ConnectorValidationIntegrationTest {
Map<String, String> config = defaultSinkConnectorProps();
String predicateName = "p";
config.put(PREDICATES_CONFIG, predicateName);
config.put(PREDICATES_CONFIG + "." + predicateName + ".type", MonitorableSinkConnector.class.getName());
config.put(PREDICATES_CONFIG + "." + predicateName + ".type", TestableSinkConnector.class.getName());
connect.assertions().assertExactlyNumErrorsOnConnectorConfigValidation(
config.get(CONNECTOR_CLASS_CONFIG),
config,
@ -315,7 +315,7 @@ public class ConnectorValidationIntegrationTest {
@Test
public void testConnectorHasInvalidConverterClassType() throws InterruptedException {
Map<String, String> config = defaultSinkConnectorProps();
config.put(KEY_CONVERTER_CLASS_CONFIG, MonitorableSinkConnector.class.getName());
config.put(KEY_CONVERTER_CLASS_CONFIG, TestableSinkConnector.class.getName());
connect.assertions().assertExactlyNumErrorsOnConnectorConfigValidation(
config.get(CONNECTOR_CLASS_CONFIG),
config,
@ -413,7 +413,7 @@ public class ConnectorValidationIntegrationTest {
@Test
public void testConnectorHasInvalidHeaderConverterClassType() throws InterruptedException {
Map<String, String> config = defaultSinkConnectorProps();
config.put(HEADER_CONVERTER_CLASS_CONFIG, MonitorableSinkConnector.class.getName());
config.put(HEADER_CONVERTER_CLASS_CONFIG, TestableSinkConnector.class.getName());
connect.assertions().assertExactlyNumErrorsOnConnectorConfigValidation(
config.get(CONNECTOR_CLASS_CONFIG),
config,
@ -560,7 +560,7 @@ public class ConnectorValidationIntegrationTest {
// setup up props for the source connector
Map<String, String> props = new HashMap<>();
props.put(NAME_CONFIG, "source-connector");
props.put(CONNECTOR_CLASS_CONFIG, MonitorableSourceConnector.class.getSimpleName());
props.put(CONNECTOR_CLASS_CONFIG, TestableSourceConnector.class.getSimpleName());
props.put(TASKS_MAX_CONFIG, "1");
props.put(TOPIC_CONFIG, "t1");
props.put(KEY_CONVERTER_CLASS_CONFIG, StringConverter.class.getName());
@ -572,7 +572,7 @@ public class ConnectorValidationIntegrationTest {
// setup up props for the sink connector
Map<String, String> props = new HashMap<>();
props.put(NAME_CONFIG, "sink-connector");
props.put(CONNECTOR_CLASS_CONFIG, MonitorableSinkConnector.class.getSimpleName());
props.put(CONNECTOR_CLASS_CONFIG, TestableSinkConnector.class.getSimpleName());
props.put(TASKS_MAX_CONFIG, "1");
props.put(TOPICS_CONFIG, "t1");
props.put(KEY_CONVERTER_CLASS_CONFIG, StringConverter.class.getName());

View File

@ -30,14 +30,14 @@ import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit;
public class ErrantRecordSinkConnector extends MonitorableSinkConnector {
public class ErrantRecordSinkConnector extends TestableSinkConnector {
@Override
public Class<? extends Task> taskClass() {
return ErrantRecordSinkTask.class;
}
public static class ErrantRecordSinkTask extends MonitorableSinkTask {
public static class ErrantRecordSinkTask extends TestableSinkTask {
private ErrantRecordReporter reporter;
private ExecutorService executorService;

View File

@ -110,7 +110,7 @@ public class ErrorHandlingIntegrationTest {
// setup connector config
Map<String, String> props = new HashMap<>();
props.put(CONNECTOR_CLASS_CONFIG, MonitorableSinkConnector.class.getSimpleName());
props.put(CONNECTOR_CLASS_CONFIG, TestableSinkConnector.class.getSimpleName());
props.put(TASKS_MAX_CONFIG, String.valueOf(NUM_TASKS));
props.put(TOPICS_CONFIG, "test-topic");
props.put(KEY_CONVERTER_CLASS_CONFIG, StringConverter.class.getName());

View File

@ -84,11 +84,11 @@ import static org.apache.kafka.clients.producer.ProducerConfig.BOOTSTRAP_SERVERS
import static org.apache.kafka.clients.producer.ProducerConfig.CLIENT_ID_CONFIG;
import static org.apache.kafka.clients.producer.ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG;
import static org.apache.kafka.clients.producer.ProducerConfig.TRANSACTIONAL_ID_CONFIG;
import static org.apache.kafka.connect.integration.MonitorableSourceConnector.CUSTOM_EXACTLY_ONCE_SUPPORT_CONFIG;
import static org.apache.kafka.connect.integration.MonitorableSourceConnector.CUSTOM_TRANSACTION_BOUNDARIES_CONFIG;
import static org.apache.kafka.connect.integration.MonitorableSourceConnector.MAX_MESSAGES_PER_SECOND_CONFIG;
import static org.apache.kafka.connect.integration.MonitorableSourceConnector.MESSAGES_PER_POLL_CONFIG;
import static org.apache.kafka.connect.integration.MonitorableSourceConnector.TOPIC_CONFIG;
import static org.apache.kafka.connect.integration.TestableSourceConnector.CUSTOM_EXACTLY_ONCE_SUPPORT_CONFIG;
import static org.apache.kafka.connect.integration.TestableSourceConnector.CUSTOM_TRANSACTION_BOUNDARIES_CONFIG;
import static org.apache.kafka.connect.integration.TestableSourceConnector.MAX_MESSAGES_PER_SECOND_CONFIG;
import static org.apache.kafka.connect.integration.TestableSourceConnector.MESSAGES_PER_POLL_CONFIG;
import static org.apache.kafka.connect.integration.TestableSourceConnector.TOPIC_CONFIG;
import static org.apache.kafka.connect.runtime.ConnectorConfig.CONNECTOR_CLASS_CONFIG;
import static org.apache.kafka.connect.runtime.ConnectorConfig.CONNECTOR_CLIENT_ADMIN_OVERRIDES_PREFIX;
import static org.apache.kafka.connect.runtime.ConnectorConfig.CONNECTOR_CLIENT_CONSUMER_OVERRIDES_PREFIX;
@ -183,7 +183,7 @@ public class ExactlyOnceSourceIntegrationTest {
startConnect();
Map<String, String> props = new HashMap<>();
props.put(CONNECTOR_CLASS_CONFIG, MonitorableSourceConnector.class.getName());
props.put(CONNECTOR_CLASS_CONFIG, TestableSourceConnector.class.getName());
props.put(TASKS_MAX_CONFIG, "1");
props.put(TOPIC_CONFIG, "topic");
props.put(KEY_CONVERTER_CLASS_CONFIG, StringConverter.class.getName());
@ -194,8 +194,8 @@ public class ExactlyOnceSourceIntegrationTest {
props.put(EXACTLY_ONCE_SUPPORT_CONFIG, "required");
// Connector will return null from SourceConnector::exactlyOnceSupport
props.put(CUSTOM_EXACTLY_ONCE_SUPPORT_CONFIG, MonitorableSourceConnector.EXACTLY_ONCE_NULL);
ConfigInfos validation = connect.validateConnectorConfig(MonitorableSourceConnector.class.getSimpleName(), props);
props.put(CUSTOM_EXACTLY_ONCE_SUPPORT_CONFIG, TestableSourceConnector.EXACTLY_ONCE_NULL);
ConfigInfos validation = connect.validateConnectorConfig(TestableSourceConnector.class.getSimpleName(), props);
assertEquals(1, validation.errorCount(),
"Preflight validation should have exactly one error");
ConfigInfo propertyValidation = findConfigInfo(EXACTLY_ONCE_SUPPORT_CONFIG, validation);
@ -203,56 +203,56 @@ public class ExactlyOnceSourceIntegrationTest {
"Preflight validation for exactly-once support property should have at least one error message");
// Connector will return UNSUPPORTED from SourceConnector::exactlyOnceSupport
props.put(CUSTOM_EXACTLY_ONCE_SUPPORT_CONFIG, MonitorableSourceConnector.EXACTLY_ONCE_UNSUPPORTED);
validation = connect.validateConnectorConfig(MonitorableSourceConnector.class.getSimpleName(), props);
props.put(CUSTOM_EXACTLY_ONCE_SUPPORT_CONFIG, TestableSourceConnector.EXACTLY_ONCE_UNSUPPORTED);
validation = connect.validateConnectorConfig(TestableSourceConnector.class.getSimpleName(), props);
assertEquals(1, validation.errorCount(), "Preflight validation should have exactly one error");
propertyValidation = findConfigInfo(EXACTLY_ONCE_SUPPORT_CONFIG, validation);
assertFalse(propertyValidation.configValue().errors().isEmpty(),
"Preflight validation for exactly-once support property should have at least one error message");
// Connector will throw an exception from SourceConnector::exactlyOnceSupport
props.put(CUSTOM_EXACTLY_ONCE_SUPPORT_CONFIG, MonitorableSourceConnector.EXACTLY_ONCE_FAIL);
validation = connect.validateConnectorConfig(MonitorableSourceConnector.class.getSimpleName(), props);
props.put(CUSTOM_EXACTLY_ONCE_SUPPORT_CONFIG, TestableSourceConnector.EXACTLY_ONCE_FAIL);
validation = connect.validateConnectorConfig(TestableSourceConnector.class.getSimpleName(), props);
assertEquals(1, validation.errorCount(), "Preflight validation should have exactly one error");
propertyValidation = findConfigInfo(EXACTLY_ONCE_SUPPORT_CONFIG, validation);
assertFalse(propertyValidation.configValue().errors().isEmpty(),
"Preflight validation for exactly-once support property should have at least one error message");
// Connector will return SUPPORTED from SourceConnector::exactlyOnceSupport
props.put(CUSTOM_EXACTLY_ONCE_SUPPORT_CONFIG, MonitorableSourceConnector.EXACTLY_ONCE_SUPPORTED);
validation = connect.validateConnectorConfig(MonitorableSourceConnector.class.getSimpleName(), props);
props.put(CUSTOM_EXACTLY_ONCE_SUPPORT_CONFIG, TestableSourceConnector.EXACTLY_ONCE_SUPPORTED);
validation = connect.validateConnectorConfig(TestableSourceConnector.class.getSimpleName(), props);
assertEquals(0, validation.errorCount(), "Preflight validation should have zero errors");
// Test out the transaction boundary definition property
props.put(TRANSACTION_BOUNDARY_CONFIG, CONNECTOR.toString());
// Connector will return null from SourceConnector::canDefineTransactionBoundaries
props.put(CUSTOM_TRANSACTION_BOUNDARIES_CONFIG, MonitorableSourceConnector.TRANSACTION_BOUNDARIES_NULL);
validation = connect.validateConnectorConfig(MonitorableSourceConnector.class.getSimpleName(), props);
props.put(CUSTOM_TRANSACTION_BOUNDARIES_CONFIG, TestableSourceConnector.TRANSACTION_BOUNDARIES_NULL);
validation = connect.validateConnectorConfig(TestableSourceConnector.class.getSimpleName(), props);
assertEquals(1, validation.errorCount(), "Preflight validation should have exactly one error");
propertyValidation = findConfigInfo(TRANSACTION_BOUNDARY_CONFIG, validation);
assertFalse(propertyValidation.configValue().errors().isEmpty(),
"Preflight validation for transaction boundary property should have at least one error message");
// Connector will return UNSUPPORTED from SourceConnector::canDefineTransactionBoundaries
props.put(CUSTOM_TRANSACTION_BOUNDARIES_CONFIG, MonitorableSourceConnector.TRANSACTION_BOUNDARIES_UNSUPPORTED);
validation = connect.validateConnectorConfig(MonitorableSourceConnector.class.getSimpleName(), props);
props.put(CUSTOM_TRANSACTION_BOUNDARIES_CONFIG, TestableSourceConnector.TRANSACTION_BOUNDARIES_UNSUPPORTED);
validation = connect.validateConnectorConfig(TestableSourceConnector.class.getSimpleName(), props);
assertEquals(1, validation.errorCount(), "Preflight validation should have exactly one error");
propertyValidation = findConfigInfo(TRANSACTION_BOUNDARY_CONFIG, validation);
assertFalse(propertyValidation.configValue().errors().isEmpty(),
"Preflight validation for transaction boundary property should have at least one error message");
// Connector will throw an exception from SourceConnector::canDefineTransactionBoundaries
props.put(CUSTOM_TRANSACTION_BOUNDARIES_CONFIG, MonitorableSourceConnector.TRANSACTION_BOUNDARIES_FAIL);
validation = connect.validateConnectorConfig(MonitorableSourceConnector.class.getSimpleName(), props);
props.put(CUSTOM_TRANSACTION_BOUNDARIES_CONFIG, TestableSourceConnector.TRANSACTION_BOUNDARIES_FAIL);
validation = connect.validateConnectorConfig(TestableSourceConnector.class.getSimpleName(), props);
assertEquals(1, validation.errorCount(), "Preflight validation should have exactly one error");
propertyValidation = findConfigInfo(TRANSACTION_BOUNDARY_CONFIG, validation);
assertFalse(propertyValidation.configValue().errors().isEmpty(),
"Preflight validation for transaction boundary property should have at least one error message");
// Connector will return SUPPORTED from SourceConnector::canDefineTransactionBoundaries
props.put(CUSTOM_TRANSACTION_BOUNDARIES_CONFIG, MonitorableSourceConnector.TRANSACTION_BOUNDARIES_SUPPORTED);
validation = connect.validateConnectorConfig(MonitorableSourceConnector.class.getSimpleName(), props);
props.put(CUSTOM_TRANSACTION_BOUNDARIES_CONFIG, TestableSourceConnector.TRANSACTION_BOUNDARIES_SUPPORTED);
validation = connect.validateConnectorConfig(TestableSourceConnector.class.getSimpleName(), props);
assertEquals(0, validation.errorCount(), "Preflight validation should have zero errors");
}
@ -274,7 +274,7 @@ public class ExactlyOnceSourceIntegrationTest {
int numTasks = 1;
Map<String, String> props = new HashMap<>();
props.put(CONNECTOR_CLASS_CONFIG, MonitorableSourceConnector.class.getName());
props.put(CONNECTOR_CLASS_CONFIG, TestableSourceConnector.class.getName());
props.put(TASKS_MAX_CONFIG, Integer.toString(numTasks));
props.put(TOPIC_CONFIG, topic);
props.put(KEY_CONVERTER_CLASS_CONFIG, StringConverter.class.getName());
@ -333,7 +333,7 @@ public class ExactlyOnceSourceIntegrationTest {
int numTasks = 1;
Map<String, String> props = new HashMap<>();
props.put(CONNECTOR_CLASS_CONFIG, MonitorableSourceConnector.class.getName());
props.put(CONNECTOR_CLASS_CONFIG, TestableSourceConnector.class.getName());
props.put(TASKS_MAX_CONFIG, Integer.toString(numTasks));
props.put(TOPIC_CONFIG, topic);
props.put(KEY_CONVERTER_CLASS_CONFIG, StringConverter.class.getName());
@ -393,14 +393,14 @@ public class ExactlyOnceSourceIntegrationTest {
connect.kafka().createTopic(topic, 3);
Map<String, String> props = new HashMap<>();
props.put(CONNECTOR_CLASS_CONFIG, MonitorableSourceConnector.class.getName());
props.put(CONNECTOR_CLASS_CONFIG, TestableSourceConnector.class.getName());
props.put(TASKS_MAX_CONFIG, "1");
props.put(TOPIC_CONFIG, topic);
props.put(KEY_CONVERTER_CLASS_CONFIG, StringConverter.class.getName());
props.put(VALUE_CONVERTER_CLASS_CONFIG, StringConverter.class.getName());
props.put(NAME_CONFIG, CONNECTOR_NAME);
props.put(TRANSACTION_BOUNDARY_CONFIG, CONNECTOR.toString());
props.put(CUSTOM_TRANSACTION_BOUNDARIES_CONFIG, MonitorableSourceConnector.TRANSACTION_BOUNDARIES_SUPPORTED);
props.put(CUSTOM_TRANSACTION_BOUNDARIES_CONFIG, TestableSourceConnector.TRANSACTION_BOUNDARIES_SUPPORTED);
props.put(MESSAGES_PER_POLL_CONFIG, MESSAGES_PER_POLL);
props.put(MAX_MESSAGES_PER_SECOND_CONFIG, MESSAGES_PER_SECOND);
@ -495,7 +495,7 @@ public class ExactlyOnceSourceIntegrationTest {
int numTasks = 1;
Map<String, String> props = new HashMap<>();
props.put(CONNECTOR_CLASS_CONFIG, MonitorableSourceConnector.class.getName());
props.put(CONNECTOR_CLASS_CONFIG, TestableSourceConnector.class.getName());
props.put(TASKS_MAX_CONFIG, Integer.toString(numTasks));
props.put(TOPIC_CONFIG, topic);
props.put(KEY_CONVERTER_CLASS_CONFIG, StringConverter.class.getName());
@ -562,7 +562,7 @@ public class ExactlyOnceSourceIntegrationTest {
connect.kafka().createTopic(topic, 3);
Map<String, String> props = new HashMap<>();
props.put(CONNECTOR_CLASS_CONFIG, MonitorableSourceConnector.class.getName());
props.put(CONNECTOR_CLASS_CONFIG, TestableSourceConnector.class.getName());
props.put(TOPIC_CONFIG, topic);
props.put(KEY_CONVERTER_CLASS_CONFIG, StringConverter.class.getName());
props.put(VALUE_CONVERTER_CLASS_CONFIG, StringConverter.class.getName());
@ -669,7 +669,7 @@ public class ExactlyOnceSourceIntegrationTest {
Map<String, String> props = new HashMap<>();
int tasksMax = 2; // Use two tasks since single-task connectors don't require zombie fencing
props.put(CONNECTOR_CLASS_CONFIG, MonitorableSourceConnector.class.getName());
props.put(CONNECTOR_CLASS_CONFIG, TestableSourceConnector.class.getName());
props.put(TOPIC_CONFIG, topic);
props.put(KEY_CONVERTER_CLASS_CONFIG, StringConverter.class.getName());
props.put(VALUE_CONVERTER_CLASS_CONFIG, StringConverter.class.getName());
@ -824,7 +824,7 @@ public class ExactlyOnceSourceIntegrationTest {
int numTasks = 1;
Map<String, String> props = new HashMap<>();
props.put(CONNECTOR_CLASS_CONFIG, MonitorableSourceConnector.class.getName());
props.put(CONNECTOR_CLASS_CONFIG, TestableSourceConnector.class.getName());
props.put(TASKS_MAX_CONFIG, Integer.toString(numTasks));
props.put(TOPIC_CONFIG, topic);
props.put(KEY_CONVERTER_CLASS_CONFIG, StringConverter.class.getName());
@ -1123,7 +1123,7 @@ public class ExactlyOnceSourceIntegrationTest {
@SuppressWarnings("unchecked")
Map<String, Object> partition = assertAndCast(key.get(1), Map.class, "Key[1]");
Object taskIdObject = partition.get("task.id");
assertNotNull(taskIdObject, "Serialized source partition should contain 'task.id' field from MonitorableSourceConnector");
assertNotNull(taskIdObject, "Serialized source partition should contain 'task.id' field from TestableSourceConnector");
String taskId = assertAndCast(taskIdObject, String.class, "task ID");
assertTrue(taskId.startsWith(CONNECTOR_NAME + "-"), "task ID should match pattern '<connectorName>-<taskId>");
String taskIdRemainder = taskId.substring(CONNECTOR_NAME.length() + 1);
@ -1138,7 +1138,7 @@ public class ExactlyOnceSourceIntegrationTest {
Map<String, Object> value = assertAndCast(valueObject, Map.class, "Value");
Object seqnoObject = value.get("saved");
assertNotNull(seqnoObject, "Serialized source offset should contain 'seqno' field from MonitorableSourceConnector");
assertNotNull(seqnoObject, "Serialized source offset should contain 'seqno' field from TestableSourceConnector");
long seqno = assertAndCast(seqnoObject, Long.class, "Seqno offset field");
result.computeIfAbsent(taskNum, t -> new ArrayList<>()).add(seqno);
@ -1163,7 +1163,7 @@ public class ExactlyOnceSourceIntegrationTest {
private StartAndStopLatch connectorAndTaskStart(int numTasks) {
connectorHandle.clearTasks();
IntStream.range(0, numTasks)
.mapToObj(i -> MonitorableSourceConnector.taskId(CONNECTOR_NAME, i))
.mapToObj(i -> TestableSourceConnector.taskId(CONNECTOR_NAME, i))
.forEach(connectorHandle::taskHandle);
return connectorHandle.expectedStarts(1, true);
}

View File

@ -63,8 +63,8 @@ public class ExampleConnectIntegrationTest {
private static final int NUM_TASKS = 3;
private static final int NUM_WORKERS = 3;
private static final String CONNECTOR_NAME = "simple-conn";
private static final String SINK_CONNECTOR_CLASS_NAME = MonitorableSinkConnector.class.getSimpleName();
private static final String SOURCE_CONNECTOR_CLASS_NAME = MonitorableSourceConnector.class.getSimpleName();
private static final String SINK_CONNECTOR_CLASS_NAME = TestableSinkConnector.class.getSimpleName();
private static final String SOURCE_CONNECTOR_CLASS_NAME = TestableSourceConnector.class.getSimpleName();
private EmbeddedConnectCluster connect;
private ConnectorHandle connectorHandle;

View File

@ -16,48 +16,27 @@
*/
package org.apache.kafka.connect.integration;
import org.apache.kafka.clients.consumer.OffsetAndMetadata;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.config.ConfigDef;
import org.apache.kafka.common.MetricName;
import org.apache.kafka.common.metrics.Gauge;
import org.apache.kafka.common.metrics.Measurable;
import org.apache.kafka.common.metrics.PluginMetrics;
import org.apache.kafka.connect.connector.Task;
import org.apache.kafka.connect.runtime.SampleSinkConnector;
import org.apache.kafka.connect.sink.SinkRecord;
import org.apache.kafka.connect.sink.SinkTask;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.ArrayList;
import java.util.Collection;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
/**
* A sink connector that is used in Apache Kafka integration tests to verify the behavior of the
* Connect framework, but that can be used in other integration tests as a simple connector that
* consumes and counts records. This class provides methods to find task instances
* which are initiated by the embedded connector, and wait for them to consume a desired number of
* messages.
*/
public class MonitorableSinkConnector extends SampleSinkConnector {
public class MonitorableSinkConnector extends TestableSinkConnector {
private static final Logger log = LoggerFactory.getLogger(MonitorableSinkConnector.class);
// Boolean valued configuration that determines whether MonitorableSinkConnector::alterOffsets should return true or false
public static final String ALTER_OFFSETS_RESULT = "alter.offsets.result";
private String connectorName;
private Map<String, String> commonConfigs;
private ConnectorHandle connectorHandle;
public static final String VALUE = "started";
public static MetricName metricsName = null;
@Override
public void start(Map<String, String> props) {
connectorHandle = RuntimeHandles.get().connectorHandle(props.get("name"));
connectorName = props.get("name");
commonConfigs = props;
log.info("Starting connector {}", props.get("name"));
connectorHandle.recordConnectorStart();
super.start(props);
PluginMetrics pluginMetrics = context.pluginMetrics();
metricsName = pluginMetrics.metricName("start", "description", Map.of());
pluginMetrics.addMetric(metricsName, (Gauge<Object>) (config, now) -> VALUE);
}
@Override
@ -65,103 +44,26 @@ public class MonitorableSinkConnector extends SampleSinkConnector {
return MonitorableSinkTask.class;
}
@Override
public List<Map<String, String>> taskConfigs(int maxTasks) {
List<Map<String, String>> configs = new ArrayList<>();
for (int i = 0; i < maxTasks; i++) {
Map<String, String> config = new HashMap<>(commonConfigs);
config.put("connector.name", connectorName);
config.put("task.id", connectorName + "-" + i);
configs.add(config);
}
return configs;
}
public static class MonitorableSinkTask extends TestableSinkTask {
@Override
public void stop() {
log.info("Stopped {} connector {}", this.getClass().getSimpleName(), connectorName);
connectorHandle.recordConnectorStop();
}
@Override
public ConfigDef config() {
return new ConfigDef();
}
@Override
public boolean alterOffsets(Map<String, String> connectorConfig, Map<TopicPartition, Long> offsets) {
return Boolean.parseBoolean(connectorConfig.get(ALTER_OFFSETS_RESULT));
}
public static class MonitorableSinkTask extends SinkTask {
private String taskId;
TaskHandle taskHandle;
Map<TopicPartition, Integer> committedOffsets;
Map<String, Map<Integer, TopicPartition>> cachedTopicPartitions;
public MonitorableSinkTask() {
this.committedOffsets = new HashMap<>();
this.cachedTopicPartitions = new HashMap<>();
}
@Override
public String version() {
return "unknown";
}
public static MetricName metricsName = null;
private int count = 0;
@Override
public void start(Map<String, String> props) {
taskId = props.get("task.id");
String connectorName = props.get("connector.name");
taskHandle = RuntimeHandles.get().connectorHandle(connectorName).taskHandle(taskId);
log.debug("Starting task {}", taskId);
taskHandle.recordTaskStart();
}
@Override
public void open(Collection<TopicPartition> partitions) {
log.debug("Opening partitions {}", partitions);
taskHandle.partitionsAssigned(partitions);
}
@Override
public void close(Collection<TopicPartition> partitions) {
log.debug("Closing partitions {}", partitions);
taskHandle.partitionsRevoked(partitions);
partitions.forEach(committedOffsets::remove);
super.start(props);
PluginMetrics pluginMetrics = context.pluginMetrics();
metricsName = pluginMetrics.metricName("put", "description", Map.of());
pluginMetrics.addMetric(metricsName, (Measurable) (config, now) -> count);
}
@Override
public void put(Collection<SinkRecord> records) {
for (SinkRecord rec : records) {
taskHandle.record(rec);
TopicPartition tp = cachedTopicPartitions
.computeIfAbsent(rec.topic(), v -> new HashMap<>())
.computeIfAbsent(rec.kafkaPartition(), v -> new TopicPartition(rec.topic(), rec.kafkaPartition()));
committedOffsets.put(tp, committedOffsets.getOrDefault(tp, 0) + 1);
log.trace("Task {} obtained record (key='{}' value='{}')", taskId, rec.key(), rec.value());
super.put(records);
for (SinkRecord ignore : records) {
count++;
}
}
@Override
public Map<TopicPartition, OffsetAndMetadata> preCommit(Map<TopicPartition, OffsetAndMetadata> offsets) {
taskHandle.partitionsCommitted(offsets.keySet());
offsets.forEach((tp, offset) -> {
int recordsSinceLastCommit = committedOffsets.getOrDefault(tp, 0);
if (recordsSinceLastCommit != 0) {
taskHandle.commit(recordsSinceLastCommit);
log.debug("Forwarding to framework request to commit {} records for {}", recordsSinceLastCommit, tp);
committedOffsets.put(tp, 0);
}
});
return offsets;
}
@Override
public void stop() {
log.info("Stopped {} task {}", this.getClass().getSimpleName(), taskId);
taskHandle.recordTaskStop();
}
}
}

View File

@ -0,0 +1,157 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.connect.integration;
import org.apache.kafka.common.MetricName;
import org.apache.kafka.common.metrics.KafkaMetric;
import org.apache.kafka.connect.runtime.rest.entities.ConnectorStateInfo;
import org.apache.kafka.connect.storage.StringConverter;
import org.apache.kafka.connect.util.clusters.EmbeddedConnectStandalone;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Tag;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.Timeout;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.Map;
import java.util.concurrent.TimeUnit;
import static org.apache.kafka.connect.runtime.ConnectorConfig.CONNECTOR_CLASS_CONFIG;
import static org.apache.kafka.connect.runtime.ConnectorConfig.KEY_CONVERTER_CLASS_CONFIG;
import static org.apache.kafka.connect.runtime.ConnectorConfig.TASKS_MAX_CONFIG;
import static org.apache.kafka.connect.runtime.ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG;
import static org.apache.kafka.connect.runtime.SinkConnectorConfig.TOPICS_CONFIG;
import static org.apache.kafka.test.TestUtils.waitForCondition;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertTrue;
/**
* Integration test for a sink connector defining metrics via the PluginMetrics API
*/
@Tag("integration")
@Timeout(value = 600)
public class MonitorableSinkIntegrationTest {
private static final Logger log = LoggerFactory.getLogger(MonitorableSinkIntegrationTest.class);
private static final String CONNECTOR_NAME = "monitorable-sink";
private static final String TASK_ID = CONNECTOR_NAME + "-0";
private static final int NUM_RECORDS_PRODUCED = 1000;
private static final int NUM_TASKS = 1;
private static final long CONNECTOR_SETUP_DURATION_MS = TimeUnit.SECONDS.toMillis(60);
private static final long CONSUME_MAX_DURATION_MS = TimeUnit.SECONDS.toMillis(30);
private EmbeddedConnectStandalone connect;
private ConnectorHandle connectorHandle;
@BeforeEach
public void setup() throws InterruptedException {
// setup Connect cluster with defaults
connect = new EmbeddedConnectStandalone.Builder().build();
// start Connect cluster
connect.start();
// get connector handles before starting test.
connectorHandle = RuntimeHandles.get().connectorHandle(CONNECTOR_NAME);
}
@AfterEach
public void close() {
connect.stop();
}
@Test
public void testMonitorableSinkConnectorAndTask() throws Exception {
connect.kafka().createTopic("test-topic");
Map<String, String> props = Map.of(
CONNECTOR_CLASS_CONFIG, MonitorableSinkConnector.class.getSimpleName(),
TASKS_MAX_CONFIG, "1",
TOPICS_CONFIG, "test-topic",
KEY_CONVERTER_CLASS_CONFIG, StringConverter.class.getName(),
VALUE_CONVERTER_CLASS_CONFIG, StringConverter.class.getName());
// set expected records to successfully reach the task
connectorHandle.taskHandle(TASK_ID).expectedRecords(NUM_RECORDS_PRODUCED);
connect.configureConnector(CONNECTOR_NAME, props);
connect.assertions().assertConnectorAndAtLeastNumTasksAreRunning(CONNECTOR_NAME, NUM_TASKS,
"Connector tasks did not start in time.");
waitForCondition(this::checkForPartitionAssignment,
CONNECTOR_SETUP_DURATION_MS,
"Connector task was not assigned a partition.");
// check connector metric
Map<MetricName, KafkaMetric> metrics = connect.connectMetrics().metrics().metrics();
MetricName connectorMetric = MonitorableSinkConnector.metricsName;
assertTrue(metrics.containsKey(connectorMetric));
assertEquals(CONNECTOR_NAME, connectorMetric.tags().get("connector"));
KafkaMetric kafkaMetric = metrics.get(connectorMetric);
assertEquals(MonitorableSinkConnector.VALUE, kafkaMetric.metricValue());
// produce some records
for (int i = 0; i < NUM_RECORDS_PRODUCED; i++) {
connect.kafka().produce("test-topic", "key-" + i, "value-" + i);
}
// wait for records to reach the task
connectorHandle.taskHandle(TASK_ID).awaitRecords(CONSUME_MAX_DURATION_MS);
// check task metric
metrics = connect.connectMetrics().metrics().metrics();
MetricName taskMetric = MonitorableSinkConnector.MonitorableSinkTask.metricsName;
assertTrue(metrics.containsKey(taskMetric));
assertEquals(CONNECTOR_NAME, taskMetric.tags().get("connector"));
assertEquals("0", taskMetric.tags().get("task"));
assertEquals((double) NUM_RECORDS_PRODUCED, metrics.get(taskMetric).metricValue());
connect.deleteConnector(CONNECTOR_NAME);
connect.assertions().assertConnectorDoesNotExist(CONNECTOR_NAME,
"Connector wasn't deleted in time.");
// verify connector and task metrics have been deleted
metrics = connect.connectMetrics().metrics().metrics();
assertFalse(metrics.containsKey(connectorMetric));
assertFalse(metrics.containsKey(taskMetric));
}
/**
* Check if a partition was assigned to each task. This method swallows exceptions since it is invoked from a
* {@link org.apache.kafka.test.TestUtils#waitForCondition} that will throw an error if this method continued
* to return false after the specified duration has elapsed.
*
* @return true if each task was assigned a partition each, false if this was not true or an error occurred when
* executing this operation.
*/
private boolean checkForPartitionAssignment() {
try {
ConnectorStateInfo info = connect.connectorStatus(CONNECTOR_NAME);
return info != null && info.tasks().size() == NUM_TASKS
&& connectorHandle.taskHandle(TASK_ID).numPartitionsAssigned() == 1;
} catch (Exception e) {
// Log the exception and return that the partitions were not assigned
log.error("Could not check connector state info.", e);
return false;
}
}
}

View File

@ -16,76 +16,27 @@
*/
package org.apache.kafka.connect.integration;
import org.apache.kafka.clients.producer.RecordMetadata;
import org.apache.kafka.common.config.ConfigDef;
import org.apache.kafka.common.MetricName;
import org.apache.kafka.common.metrics.Gauge;
import org.apache.kafka.common.metrics.Measurable;
import org.apache.kafka.common.metrics.PluginMetrics;
import org.apache.kafka.connect.connector.Task;
import org.apache.kafka.connect.data.Schema;
import org.apache.kafka.connect.errors.ConnectException;
import org.apache.kafka.connect.header.ConnectHeaders;
import org.apache.kafka.connect.runtime.SampleSourceConnector;
import org.apache.kafka.connect.source.ConnectorTransactionBoundaries;
import org.apache.kafka.connect.source.ExactlyOnceSupport;
import org.apache.kafka.connect.source.SourceRecord;
import org.apache.kafka.connect.source.SourceTask;
import org.apache.kafka.server.util.ThroughputThrottler;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Optional;
import java.util.stream.Collectors;
import java.util.stream.LongStream;
/**
* A source connector that is used in Apache Kafka integration tests to verify the behavior of
* the Connect framework, but that can be used in other integration tests as a simple connector
* that generates records of a fixed structure. The rate of record production can be adjusted
* through the configs 'throughput' and 'messages.per.poll'
*/
public class MonitorableSourceConnector extends SampleSourceConnector {
private static final Logger log = LoggerFactory.getLogger(MonitorableSourceConnector.class);
public class MonitorableSourceConnector extends TestableSourceConnector {
public static final String TOPIC_CONFIG = "topic";
public static final String NUM_TASKS = "num.tasks";
public static final String MESSAGES_PER_POLL_CONFIG = "messages.per.poll";
public static final String MAX_MESSAGES_PER_SECOND_CONFIG = "throughput";
public static final String MAX_MESSAGES_PRODUCED_CONFIG = "max.messages";
public static final String CUSTOM_EXACTLY_ONCE_SUPPORT_CONFIG = "custom.exactly.once.support";
public static final String EXACTLY_ONCE_SUPPORTED = "supported";
public static final String EXACTLY_ONCE_UNSUPPORTED = "unsupported";
public static final String EXACTLY_ONCE_NULL = "null";
public static final String EXACTLY_ONCE_FAIL = "fail";
public static final String CUSTOM_TRANSACTION_BOUNDARIES_CONFIG = "custom.transaction.boundaries";
public static final String TRANSACTION_BOUNDARIES_SUPPORTED = "supported";
public static final String TRANSACTION_BOUNDARIES_UNSUPPORTED = "unsupported";
public static final String TRANSACTION_BOUNDARIES_NULL = "null";
public static final String TRANSACTION_BOUNDARIES_FAIL = "fail";
// Boolean valued configuration that determines whether MonitorableSourceConnector::alterOffsets should return true or false
public static final String ALTER_OFFSETS_RESULT = "alter.offsets.result";
private String connectorName;
private ConnectorHandle connectorHandle;
private Map<String, String> commonConfigs;
public static MetricName metricsName = null;
public static final String VALUE = "started";
@Override
public void start(Map<String, String> props) {
connectorHandle = RuntimeHandles.get().connectorHandle(props.get("name"));
connectorName = connectorHandle.name();
commonConfigs = props;
log.info("Started {} connector {}", this.getClass().getSimpleName(), connectorName);
connectorHandle.recordConnectorStart();
if (Boolean.parseBoolean(props.getOrDefault("connector.start.inject.error", "false"))) {
throw new RuntimeException("Injecting errors during connector start");
}
super.start(props);
PluginMetrics pluginMetrics = context.pluginMetrics();
metricsName = pluginMetrics.metricName("start", "description", Map.of());
pluginMetrics.addMetric(metricsName, (Gauge<Object>) (config, now) -> VALUE);
}
@Override
@ -93,222 +44,27 @@ public class MonitorableSourceConnector extends SampleSourceConnector {
return MonitorableSourceTask.class;
}
@Override
public List<Map<String, String>> taskConfigs(int maxTasks) {
String numTasksProp = commonConfigs.get(NUM_TASKS);
int numTasks = numTasksProp != null ? Integer.parseInt(numTasksProp) : maxTasks;
List<Map<String, String>> configs = new ArrayList<>();
for (int i = 0; i < numTasks; i++) {
Map<String, String> config = taskConfig(commonConfigs, connectorName, i);
configs.add(config);
}
return configs;
}
public static class MonitorableSourceTask extends TestableSourceTask {
public static Map<String, String> taskConfig(
Map<String, String> connectorProps,
String connectorName,
int taskNum
) {
Map<String, String> result = new HashMap<>(connectorProps);
result.put("connector.name", connectorName);
result.put("task.id", taskId(connectorName, taskNum));
return result;
}
@Override
public void stop() {
log.info("Stopped {} connector {}", this.getClass().getSimpleName(), connectorName);
connectorHandle.recordConnectorStop();
if (Boolean.parseBoolean(commonConfigs.getOrDefault("connector.stop.inject.error", "false"))) {
throw new RuntimeException("Injecting errors during connector stop");
}
}
@Override
public ConfigDef config() {
log.info("Configured {} connector {}", this.getClass().getSimpleName(), connectorName);
return new ConfigDef();
}
@Override
public ExactlyOnceSupport exactlyOnceSupport(Map<String, String> connectorConfig) {
String supportLevel = connectorConfig.getOrDefault(CUSTOM_EXACTLY_ONCE_SUPPORT_CONFIG, "null").toLowerCase(Locale.ROOT);
switch (supportLevel) {
case EXACTLY_ONCE_SUPPORTED:
return ExactlyOnceSupport.SUPPORTED;
case EXACTLY_ONCE_UNSUPPORTED:
return ExactlyOnceSupport.UNSUPPORTED;
case EXACTLY_ONCE_FAIL:
throw new ConnectException("oops");
default:
case EXACTLY_ONCE_NULL:
return null;
}
}
@Override
public ConnectorTransactionBoundaries canDefineTransactionBoundaries(Map<String, String> connectorConfig) {
String supportLevel = connectorConfig.getOrDefault(CUSTOM_TRANSACTION_BOUNDARIES_CONFIG, TRANSACTION_BOUNDARIES_UNSUPPORTED).toLowerCase(Locale.ROOT);
switch (supportLevel) {
case TRANSACTION_BOUNDARIES_SUPPORTED:
return ConnectorTransactionBoundaries.SUPPORTED;
case TRANSACTION_BOUNDARIES_FAIL:
throw new ConnectException("oh no :(");
case TRANSACTION_BOUNDARIES_NULL:
return null;
default:
case TRANSACTION_BOUNDARIES_UNSUPPORTED:
return ConnectorTransactionBoundaries.UNSUPPORTED;
}
}
@Override
public boolean alterOffsets(Map<String, String> connectorConfig, Map<Map<String, ?>, Map<String, ?>> offsets) {
return Boolean.parseBoolean(connectorConfig.get(ALTER_OFFSETS_RESULT));
}
public static String taskId(String connectorName, int taskId) {
return connectorName + "-" + taskId;
}
public static class MonitorableSourceTask extends SourceTask {
private String taskId;
private String topicName;
private TaskHandle taskHandle;
private volatile boolean stopped;
private long startingSeqno;
private long seqno;
private int batchSize;
private ThroughputThrottler throttler;
private long maxMessages;
private long priorTransactionBoundary;
private long nextTransactionBoundary;
@Override
public String version() {
return "unknown";
}
public static MetricName metricsName = null;
private int count = 0;
@Override
public void start(Map<String, String> props) {
taskId = props.get("task.id");
String connectorName = props.get("connector.name");
topicName = props.getOrDefault(TOPIC_CONFIG, "sequential-topic");
batchSize = Integer.parseInt(props.getOrDefault(MESSAGES_PER_POLL_CONFIG, "1"));
taskHandle = RuntimeHandles.get().connectorHandle(connectorName).taskHandle(taskId);
Map<String, Object> offset = Optional.ofNullable(
context.offsetStorageReader().offset(sourcePartition(taskId)))
.orElse(Collections.emptyMap());
startingSeqno = Optional.ofNullable((Long) offset.get("saved")).orElse(0L);
seqno = startingSeqno;
log.info("Started {} task {} with properties {}", this.getClass().getSimpleName(), taskId, props);
throttler = new ThroughputThrottler(Long.parseLong(props.getOrDefault(MAX_MESSAGES_PER_SECOND_CONFIG, "-1")), System.currentTimeMillis());
maxMessages = Long.parseLong(props.getOrDefault(MAX_MESSAGES_PRODUCED_CONFIG, String.valueOf(Long.MAX_VALUE)));
taskHandle.recordTaskStart();
priorTransactionBoundary = 0;
nextTransactionBoundary = 1;
if (Boolean.parseBoolean(props.getOrDefault("task-" + taskId + ".start.inject.error", "false"))) {
throw new RuntimeException("Injecting errors during task start");
}
calculateNextBoundary();
super.start(props);
PluginMetrics pluginMetrics = context.pluginMetrics();
metricsName = pluginMetrics.metricName("poll", "description", Map.of());
pluginMetrics.addMetric(metricsName, (Measurable) (config, now) -> count);
}
@Override
public List<SourceRecord> poll() {
if (!stopped) {
// Don't return any more records since we've already produced the configured maximum number.
if (seqno >= maxMessages) {
return null;
}
if (throttler.shouldThrottle(seqno - startingSeqno, System.currentTimeMillis())) {
throttler.throttle();
}
int currentBatchSize = (int) Math.min(maxMessages - seqno, batchSize);
taskHandle.record(currentBatchSize);
log.trace("Returning batch of {} records", currentBatchSize);
return LongStream.range(0, currentBatchSize)
.mapToObj(i -> {
seqno++;
SourceRecord record = new SourceRecord(
sourcePartition(taskId),
sourceOffset(seqno),
topicName,
null,
Schema.STRING_SCHEMA,
"key-" + taskId + "-" + seqno,
Schema.STRING_SCHEMA,
"value-" + taskId + "-" + seqno,
null,
new ConnectHeaders().addLong("header-" + seqno, seqno));
maybeDefineTransactionBoundary(record);
return record;
})
.collect(Collectors.toList());
List<SourceRecord> records = super.poll();
if (records != null) {
count += records.size();
}
return null;
return records;
}
@Override
public void commit() {
log.info("Task {} committing offsets", taskId);
//TODO: save progress outside the offset topic, potentially in the task handle
}
@Override
public void commitRecord(SourceRecord record, RecordMetadata metadata) {
log.trace("Committing record: {}", record);
taskHandle.commit();
}
@Override
public void stop() {
log.info("Stopped {} task {}", this.getClass().getSimpleName(), taskId);
stopped = true;
taskHandle.recordTaskStop();
}
/**
* Calculate the next transaction boundary, i.e., the seqno whose corresponding source record should be used to
* either {@link org.apache.kafka.connect.source.TransactionContext#commitTransaction(SourceRecord) commit}
* or {@link org.apache.kafka.connect.source.TransactionContext#abortTransaction(SourceRecord) abort} the next transaction.
* <p>
* This connector defines transactions whose size correspond to successive elements of the Fibonacci sequence,
* where transactions with an even number of records are aborted, and those with an odd number of records are committed.
*/
private void calculateNextBoundary() {
while (nextTransactionBoundary <= seqno) {
nextTransactionBoundary += priorTransactionBoundary;
priorTransactionBoundary = nextTransactionBoundary - priorTransactionBoundary;
}
}
private void maybeDefineTransactionBoundary(SourceRecord record) {
if (context.transactionContext() == null || seqno != nextTransactionBoundary) {
return;
}
long transactionSize = nextTransactionBoundary - priorTransactionBoundary;
// If the transaction boundary ends on an even-numbered offset, abort it
// Otherwise, commit
boolean abort = nextTransactionBoundary % 2 == 0;
calculateNextBoundary();
if (abort) {
log.info("Aborting transaction of {} records", transactionSize);
context.transactionContext().abortTransaction(record);
} else {
log.info("Committing transaction of {} records", transactionSize);
context.transactionContext().commitTransaction(record);
}
}
}
public static Map<String, Object> sourcePartition(String taskId) {
return Collections.singletonMap("task.id", taskId);
}
public static Map<String, Object> sourceOffset(long seqno) {
return Collections.singletonMap("saved", seqno);
}
}

View File

@ -0,0 +1,130 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.connect.integration;
import org.apache.kafka.common.MetricName;
import org.apache.kafka.common.metrics.KafkaMetric;
import org.apache.kafka.connect.storage.StringConverter;
import org.apache.kafka.connect.util.clusters.EmbeddedConnectStandalone;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Tag;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.Timeout;
import java.util.Map;
import java.util.concurrent.TimeUnit;
import static org.apache.kafka.connect.integration.TestableSourceConnector.MAX_MESSAGES_PER_SECOND_CONFIG;
import static org.apache.kafka.connect.integration.TestableSourceConnector.MESSAGES_PER_POLL_CONFIG;
import static org.apache.kafka.connect.integration.TestableSourceConnector.TOPIC_CONFIG;
import static org.apache.kafka.connect.runtime.ConnectorConfig.CONNECTOR_CLASS_CONFIG;
import static org.apache.kafka.connect.runtime.ConnectorConfig.KEY_CONVERTER_CLASS_CONFIG;
import static org.apache.kafka.connect.runtime.ConnectorConfig.TASKS_MAX_CONFIG;
import static org.apache.kafka.connect.runtime.ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertTrue;
/**
* Integration test for a source connector defining metrics via the PluginMetrics API
*/
@Tag("integration")
@Timeout(value = 600)
public class MonitorableSourceIntegrationTest {
private static final String CONNECTOR_NAME = "monitorable-source";
private static final String TASK_ID = CONNECTOR_NAME + "-0";
private static final int NUM_TASKS = 1;
private static final long SOURCE_TASK_PRODUCE_TIMEOUT_MS = TimeUnit.SECONDS.toMillis(30);
private static final int MINIMUM_MESSAGES = 100;
private static final String MESSAGES_PER_POLL = Integer.toString(MINIMUM_MESSAGES);
private static final String MESSAGES_PER_SECOND = Long.toString(MINIMUM_MESSAGES / 2);
private EmbeddedConnectStandalone connect;
private ConnectorHandle connectorHandle;
@BeforeEach
public void setup() throws InterruptedException {
// setup Connect cluster with defaults
connect = new EmbeddedConnectStandalone.Builder().build();
// start Connect cluster
connect.start();
// get connector handles before starting test.
connectorHandle = RuntimeHandles.get().connectorHandle(CONNECTOR_NAME);
}
@AfterEach
public void close() {
connect.stop();
}
@Test
public void testMonitorableSourceConnectorAndTask() throws Exception {
connect.kafka().createTopic("test-topic");
Map<String, String> props = Map.of(
CONNECTOR_CLASS_CONFIG, MonitorableSourceConnector.class.getSimpleName(),
TASKS_MAX_CONFIG, "1",
TOPIC_CONFIG, "test-topic",
KEY_CONVERTER_CLASS_CONFIG, StringConverter.class.getName(),
VALUE_CONVERTER_CLASS_CONFIG, StringConverter.class.getName(),
MESSAGES_PER_POLL_CONFIG, MESSAGES_PER_POLL,
MAX_MESSAGES_PER_SECOND_CONFIG, MESSAGES_PER_SECOND);
// set expected records to successfully reach the task
// expect all records to be consumed and committed by the task
connectorHandle.taskHandle(TASK_ID).expectedRecords(MINIMUM_MESSAGES);
connectorHandle.taskHandle(TASK_ID).expectedCommits(MINIMUM_MESSAGES);
connect.configureConnector(CONNECTOR_NAME, props);
connect.assertions().assertConnectorAndAtLeastNumTasksAreRunning(CONNECTOR_NAME, NUM_TASKS,
"Connector tasks did not start in time.");
// wait for the connector tasks to produce enough records
connectorHandle.taskHandle(TASK_ID).awaitRecords(SOURCE_TASK_PRODUCE_TIMEOUT_MS);
connectorHandle.taskHandle(TASK_ID).awaitCommits(TimeUnit.MINUTES.toMillis(1));
// check connector metric
Map<MetricName, KafkaMetric> metrics = connect.connectMetrics().metrics().metrics();
MetricName connectorMetric = MonitorableSourceConnector.metricsName;
assertTrue(metrics.containsKey(connectorMetric));
assertEquals(CONNECTOR_NAME, connectorMetric.tags().get("connector"));
assertEquals(MonitorableSourceConnector.VALUE, metrics.get(connectorMetric).metricValue());
// check task metric
metrics = connect.connectMetrics().metrics().metrics();
MetricName taskMetric = MonitorableSourceConnector.MonitorableSourceTask.metricsName;
assertTrue(metrics.containsKey(taskMetric));
assertEquals(CONNECTOR_NAME, taskMetric.tags().get("connector"));
assertEquals("0", taskMetric.tags().get("task"));
assertTrue(MINIMUM_MESSAGES <= (double) metrics.get(taskMetric).metricValue());
connect.deleteConnector(CONNECTOR_NAME);
connect.assertions().assertConnectorDoesNotExist(CONNECTOR_NAME,
"Connector wasn't deleted in time.");
// verify the connector and task metrics have been deleted
metrics = connect.connectMetrics().metrics().metrics();
assertFalse(metrics.containsKey(connectorMetric));
assertFalse(metrics.containsKey(taskMetric));
}
}

View File

@ -58,7 +58,7 @@ import java.util.concurrent.atomic.AtomicReference;
import jakarta.ws.rs.core.Response;
import static jakarta.ws.rs.core.Response.Status.INTERNAL_SERVER_ERROR;
import static org.apache.kafka.connect.integration.MonitorableSourceConnector.TOPIC_CONFIG;
import static org.apache.kafka.connect.integration.TestableSourceConnector.TOPIC_CONFIG;
import static org.apache.kafka.connect.runtime.ConnectorConfig.CONNECTOR_CLASS_CONFIG;
import static org.apache.kafka.connect.runtime.ConnectorConfig.TASKS_MAX_CONFIG;
import static org.apache.kafka.connect.runtime.SinkConnectorConfig.TOPICS_CONFIG;
@ -277,7 +277,7 @@ public class OffsetsApiIntegrationTest {
"Source connector offsets should reflect the expected number of records produced");
// Each task should produce more records
connectorConfigs.put(MonitorableSourceConnector.MAX_MESSAGES_PRODUCED_CONFIG, String.valueOf(2 * NUM_RECORDS_PER_PARTITION));
connectorConfigs.put(TestableSourceConnector.MAX_MESSAGES_PRODUCED_CONFIG, String.valueOf(2 * NUM_RECORDS_PER_PARTITION));
connect.configureConnector(connectorName, connectorConfigs);
verifyExpectedSourceConnectorOffsets(connectorName, NUM_TASKS, 2 * NUM_RECORDS_PER_PARTITION,
@ -300,7 +300,7 @@ public class OffsetsApiIntegrationTest {
"Connector tasks did not start in time.");
List<ConnectorOffset> offsets = new ArrayList<>();
// The MonitorableSourceConnector has a source partition per task
// The TestableSourceConnector has a source partition per task
for (int i = 0; i < NUM_TASKS; i++) {
offsets.add(
new ConnectorOffset(Collections.singletonMap("task.id", connectorName + "-" + i),
@ -415,7 +415,7 @@ public class OffsetsApiIntegrationTest {
"Sink connector consumer group offsets should reflect the altered offsets");
// Update the connector's configs; this time expect SinkConnector::alterOffsets to return true
connectorConfigs.put(MonitorableSinkConnector.ALTER_OFFSETS_RESULT, "true");
connectorConfigs.put(TestableSinkConnector.ALTER_OFFSETS_RESULT, "true");
connect.configureConnector(connectorName, connectorConfigs);
// Alter offsets again while the connector is still in a stopped state
@ -598,7 +598,7 @@ public class OffsetsApiIntegrationTest {
);
List<ConnectorOffset> offsetsToAlter = new ArrayList<>();
// The MonitorableSourceConnector has a source partition per task
// The TestableSourceConnector has a source partition per task
for (int i = 0; i < NUM_TASKS; i++) {
offsetsToAlter.add(
new ConnectorOffset(Collections.singletonMap("task.id", connectorName + "-" + i),
@ -614,12 +614,12 @@ public class OffsetsApiIntegrationTest {
"Source connector offsets should reflect the altered offsets");
// Update the connector's configs; this time expect SourceConnector::alterOffsets to return true
connectorConfigs.put(MonitorableSourceConnector.ALTER_OFFSETS_RESULT, "true");
connectorConfigs.put(TestableSourceConnector.ALTER_OFFSETS_RESULT, "true");
connect.configureConnector(connectorName, connectorConfigs);
// Alter offsets again while connector is in stopped state
offsetsToAlter = new ArrayList<>();
// The MonitorableSourceConnector has a source partition per task
// The TestableSourceConnector has a source partition per task
for (int i = 0; i < NUM_TASKS; i++) {
offsetsToAlter.add(
new ConnectorOffset(Collections.singletonMap("task.id", connectorName + "-" + i),
@ -908,7 +908,7 @@ public class OffsetsApiIntegrationTest {
private Map<String, String> baseSinkConnectorConfigs() {
Map<String, String> configs = new HashMap<>();
configs.put(CONNECTOR_CLASS_CONFIG, MonitorableSinkConnector.class.getSimpleName());
configs.put(CONNECTOR_CLASS_CONFIG, TestableSinkConnector.class.getSimpleName());
configs.put(TASKS_MAX_CONFIG, String.valueOf(NUM_TASKS));
configs.put(TOPICS_CONFIG, topic);
configs.put(KEY_CONVERTER_CLASS_CONFIG, StringConverter.class.getName());
@ -918,11 +918,11 @@ public class OffsetsApiIntegrationTest {
private Map<String, String> baseSourceConnectorConfigs() {
Map<String, String> props = new HashMap<>();
props.put(CONNECTOR_CLASS_CONFIG, MonitorableSourceConnector.class.getSimpleName());
props.put(CONNECTOR_CLASS_CONFIG, TestableSourceConnector.class.getSimpleName());
props.put(TASKS_MAX_CONFIG, String.valueOf(NUM_TASKS));
props.put(TOPIC_CONFIG, topic);
props.put(MonitorableSourceConnector.MESSAGES_PER_POLL_CONFIG, "3");
props.put(MonitorableSourceConnector.MAX_MESSAGES_PRODUCED_CONFIG, String.valueOf(NUM_RECORDS_PER_PARTITION));
props.put(TestableSourceConnector.MESSAGES_PER_POLL_CONFIG, "3");
props.put(TestableSourceConnector.MAX_MESSAGES_PRODUCED_CONFIG, String.valueOf(NUM_RECORDS_PER_PARTITION));
props.put(ConnectorConfig.KEY_CONVERTER_CLASS_CONFIG, StringConverter.class.getName());
props.put(ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG, StringConverter.class.getName());
props.put(DEFAULT_TOPIC_CREATION_PREFIX + REPLICATION_FACTOR_CONFIG, "1");
@ -1044,7 +1044,7 @@ public class OffsetsApiIntegrationTest {
* Verify whether the actual offsets for a source connector match the expected offsets. The verification is done using the
* <strong><em>GET /connectors/{connector}/offsets</em></strong> REST API which is repeatedly queried until the offsets match
* or the {@link #OFFSET_READ_TIMEOUT_MS timeout} is reached. Note that this assumes that the source connector is a
* {@link MonitorableSourceConnector}
* {@link TestableSourceConnector}
*
* @param connectorName the name of the source connector whose offsets are to be verified
* @param numTasks the number of tasks for the source connector
@ -1057,7 +1057,7 @@ public class OffsetsApiIntegrationTest {
int expectedOffset, String conditionDetails) throws InterruptedException {
waitForCondition(() -> {
ConnectorOffsets offsets = connect.connectorOffsets(connectorName);
// The MonitorableSourceConnector has a source partition per task
// The TestableSourceConnector has a source partition per task
if (offsets.offsets().size() != numTasks) {
return false;
}

View File

@ -38,7 +38,7 @@ import java.util.concurrent.TimeUnit;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
import static org.apache.kafka.connect.integration.MonitorableSourceConnector.TOPIC_CONFIG;
import static org.apache.kafka.connect.integration.TestableSourceConnector.TOPIC_CONFIG;
import static org.apache.kafka.connect.runtime.ConnectorConfig.CONNECTOR_CLASS_CONFIG;
import static org.apache.kafka.connect.runtime.ConnectorConfig.KEY_CONVERTER_CLASS_CONFIG;
import static org.apache.kafka.connect.runtime.ConnectorConfig.TASKS_MAX_CONFIG;
@ -298,7 +298,7 @@ public class RebalanceSourceConnectorsIntegrationTest {
private Map<String, String> defaultSourceConnectorProps(String topic) {
// setup up props for the source connector
Map<String, String> props = new HashMap<>();
props.put(CONNECTOR_CLASS_CONFIG, MonitorableSourceConnector.class.getSimpleName());
props.put(CONNECTOR_CLASS_CONFIG, TestableSourceConnector.class.getSimpleName());
props.put(TASKS_MAX_CONFIG, String.valueOf(NUM_TASKS));
props.put(TOPIC_CONFIG, topic);
props.put("throughput", String.valueOf(10));

View File

@ -92,7 +92,7 @@ public class RestExtensionIntegrationTest {
try {
// setup up props for the connector
Map<String, String> connectorProps = new HashMap<>();
connectorProps.put(CONNECTOR_CLASS_CONFIG, MonitorableSinkConnector.class.getSimpleName());
connectorProps.put(CONNECTOR_CLASS_CONFIG, TestableSinkConnector.class.getSimpleName());
connectorProps.put(TASKS_MAX_CONFIG, String.valueOf(1));
connectorProps.put(TOPICS_CONFIG, "test-topic");

View File

@ -21,6 +21,7 @@ import org.apache.kafka.common.config.types.Password;
import org.apache.kafka.common.network.ConnectionMode;
import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.connect.runtime.Herder;
import org.apache.kafka.connect.runtime.MockConnectMetrics;
import org.apache.kafka.connect.runtime.WorkerConfig;
import org.apache.kafka.connect.runtime.distributed.DistributedConfig;
import org.apache.kafka.connect.runtime.distributed.NotLeaderException;
@ -75,6 +76,7 @@ import static org.mockito.ArgumentMatchers.any;
import static org.mockito.ArgumentMatchers.anyBoolean;
import static org.mockito.ArgumentMatchers.isNull;
import static org.mockito.Mockito.doAnswer;
import static org.mockito.Mockito.doReturn;
import static org.mockito.Mockito.when;
@ExtendWith(MockitoExtension.class)
@ -167,6 +169,7 @@ public class RestForwardingIntegrationTest {
followerServer = new ConnectRestServer(null, followerClient, followerConfig.originals());
followerServer.initializeServer();
when(followerHerder.plugins()).thenReturn(plugins);
doReturn(new MockConnectMetrics()).when(followerHerder).connectMetrics();
followerServer.initializeResources(followerHerder);
// Leader worker setup
@ -174,6 +177,7 @@ public class RestForwardingIntegrationTest {
leaderServer = new ConnectRestServer(null, leaderClient, leaderConfig.originals());
leaderServer.initializeServer();
when(leaderHerder.plugins()).thenReturn(plugins);
doReturn(new MockConnectMetrics()).when(leaderHerder).connectMetrics();
leaderServer.initializeResources(leaderHerder);
// External client setup

View File

@ -122,7 +122,7 @@ public class SessionedProtocolIntegrationTest {
// Create the connector now
// setup up props for the sink connector
Map<String, String> connectorProps = new HashMap<>();
connectorProps.put(CONNECTOR_CLASS_CONFIG, MonitorableSinkConnector.class.getSimpleName());
connectorProps.put(CONNECTOR_CLASS_CONFIG, TestableSinkConnector.class.getSimpleName());
connectorProps.put(TASKS_MAX_CONFIG, String.valueOf(1));
connectorProps.put(TOPICS_CONFIG, "test-topic");
connectorProps.put(KEY_CONVERTER_CLASS_CONFIG, StringConverter.class.getName());

View File

@ -309,7 +309,7 @@ public class SinkConnectorsIntegrationTest {
private Map<String, String> baseSinkConnectorProps(String topics) {
Map<String, String> props = new HashMap<>();
props.put(CONNECTOR_CLASS_CONFIG, MonitorableSinkConnector.class.getSimpleName());
props.put(CONNECTOR_CLASS_CONFIG, TestableSinkConnector.class.getSimpleName());
props.put(TASKS_MAX_CONFIG, String.valueOf(NUM_TASKS));
props.put(TOPICS_CONFIG, topics);
props.put(KEY_CONVERTER_CLASS_CONFIG, StringConverter.class.getName());

View File

@ -31,7 +31,7 @@ import java.util.Map;
import java.util.Properties;
import java.util.stream.IntStream;
import static org.apache.kafka.connect.integration.MonitorableSourceConnector.TOPIC_CONFIG;
import static org.apache.kafka.connect.integration.TestableSourceConnector.TOPIC_CONFIG;
import static org.apache.kafka.connect.runtime.ConnectorConfig.CONNECTOR_CLASS_CONFIG;
import static org.apache.kafka.connect.runtime.ConnectorConfig.KEY_CONVERTER_CLASS_CONFIG;
import static org.apache.kafka.connect.runtime.ConnectorConfig.NAME_CONFIG;
@ -208,7 +208,7 @@ public class SourceConnectorsIntegrationTest {
private Map<String, String> defaultSourceConnectorProps(String topic) {
// setup up props for the source connector
Map<String, String> props = new HashMap<>();
props.put(CONNECTOR_CLASS_CONFIG, MonitorableSourceConnector.class.getSimpleName());
props.put(CONNECTOR_CLASS_CONFIG, TestableSourceConnector.class.getSimpleName());
props.put(TASKS_MAX_CONFIG, String.valueOf(NUM_TASKS));
props.put(TOPIC_CONFIG, topic);
props.put("throughput", String.valueOf(10));

View File

@ -45,7 +45,7 @@ import jakarta.ws.rs.core.Response;
import static org.apache.kafka.connect.integration.BlockingConnectorTest.Block.BLOCK_CONFIG;
import static org.apache.kafka.connect.integration.BlockingConnectorTest.CONNECTOR_START;
import static org.apache.kafka.connect.integration.BlockingConnectorTest.CONNECTOR_TASK_CONFIGS;
import static org.apache.kafka.connect.integration.MonitorableSourceConnector.TOPIC_CONFIG;
import static org.apache.kafka.connect.integration.TestableSourceConnector.TOPIC_CONFIG;
import static org.apache.kafka.connect.runtime.ConnectorConfig.CONNECTOR_CLASS_CONFIG;
import static org.apache.kafka.connect.runtime.ConnectorConfig.KEY_CONVERTER_CLASS_CONFIG;
import static org.apache.kafka.connect.runtime.ConnectorConfig.NAME_CONFIG;
@ -370,7 +370,7 @@ public class StandaloneWorkerIntegrationTest {
// setup props for the source connector
Map<String, String> props = new HashMap<>();
props.put(NAME_CONFIG, CONNECTOR_NAME);
props.put(CONNECTOR_CLASS_CONFIG, MonitorableSourceConnector.class.getSimpleName());
props.put(CONNECTOR_CLASS_CONFIG, TestableSourceConnector.class.getSimpleName());
props.put(TASKS_MAX_CONFIG, String.valueOf(NUM_TASKS));
props.put(TOPIC_CONFIG, topic);
props.put(KEY_CONVERTER_CLASS_CONFIG, StringConverter.class.getName());

View File

@ -0,0 +1,167 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.connect.integration;
import org.apache.kafka.clients.consumer.OffsetAndMetadata;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.config.ConfigDef;
import org.apache.kafka.connect.connector.Task;
import org.apache.kafka.connect.runtime.SampleSinkConnector;
import org.apache.kafka.connect.sink.SinkRecord;
import org.apache.kafka.connect.sink.SinkTask;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.ArrayList;
import java.util.Collection;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
/**
* A sink connector that is used in Apache Kafka integration tests to verify the behavior of the
* Connect framework, but that can be used in other integration tests as a simple connector that
* consumes and counts records. This class provides methods to find task instances
* which are initiated by the embedded connector, and wait for them to consume a desired number of
* messages.
*/
public class TestableSinkConnector extends SampleSinkConnector {
private static final Logger log = LoggerFactory.getLogger(TestableSinkConnector.class);
// Boolean valued configuration that determines whether TestableSinkConnector::alterOffsets should return true or false
public static final String ALTER_OFFSETS_RESULT = "alter.offsets.result";
private String connectorName;
private Map<String, String> commonConfigs;
private ConnectorHandle connectorHandle;
@Override
public void start(Map<String, String> props) {
connectorHandle = RuntimeHandles.get().connectorHandle(props.get("name"));
connectorName = props.get("name");
commonConfigs = props;
log.info("Starting connector {}", props.get("name"));
connectorHandle.recordConnectorStart();
}
@Override
public Class<? extends Task> taskClass() {
return TestableSinkTask.class;
}
@Override
public List<Map<String, String>> taskConfigs(int maxTasks) {
List<Map<String, String>> configs = new ArrayList<>();
for (int i = 0; i < maxTasks; i++) {
Map<String, String> config = new HashMap<>(commonConfigs);
config.put("connector.name", connectorName);
config.put("task.id", connectorName + "-" + i);
configs.add(config);
}
return configs;
}
@Override
public void stop() {
log.info("Stopped {} connector {}", this.getClass().getSimpleName(), connectorName);
connectorHandle.recordConnectorStop();
}
@Override
public ConfigDef config() {
return new ConfigDef();
}
@Override
public boolean alterOffsets(Map<String, String> connectorConfig, Map<TopicPartition, Long> offsets) {
return Boolean.parseBoolean(connectorConfig.get(ALTER_OFFSETS_RESULT));
}
public static class TestableSinkTask extends SinkTask {
private String taskId;
TaskHandle taskHandle;
Map<TopicPartition, Integer> committedOffsets;
Map<String, Map<Integer, TopicPartition>> cachedTopicPartitions;
public TestableSinkTask() {
this.committedOffsets = new HashMap<>();
this.cachedTopicPartitions = new HashMap<>();
}
@Override
public String version() {
return "unknown";
}
@Override
public void start(Map<String, String> props) {
taskId = props.get("task.id");
String connectorName = props.get("connector.name");
taskHandle = RuntimeHandles.get().connectorHandle(connectorName).taskHandle(taskId);
log.debug("Starting task {}", taskId);
taskHandle.recordTaskStart();
}
@Override
public void open(Collection<TopicPartition> partitions) {
log.debug("Opening partitions {}", partitions);
taskHandle.partitionsAssigned(partitions);
}
@Override
public void close(Collection<TopicPartition> partitions) {
log.debug("Closing partitions {}", partitions);
taskHandle.partitionsRevoked(partitions);
partitions.forEach(committedOffsets::remove);
}
@Override
public void put(Collection<SinkRecord> records) {
for (SinkRecord rec : records) {
taskHandle.record(rec);
TopicPartition tp = cachedTopicPartitions
.computeIfAbsent(rec.topic(), v -> new HashMap<>())
.computeIfAbsent(rec.kafkaPartition(), v -> new TopicPartition(rec.topic(), rec.kafkaPartition()));
committedOffsets.put(tp, committedOffsets.getOrDefault(tp, 0) + 1);
log.trace("Task {} obtained record (key='{}' value='{}')", taskId, rec.key(), rec.value());
}
}
@Override
public Map<TopicPartition, OffsetAndMetadata> preCommit(Map<TopicPartition, OffsetAndMetadata> offsets) {
taskHandle.partitionsCommitted(offsets.keySet());
offsets.forEach((tp, offset) -> {
int recordsSinceLastCommit = committedOffsets.getOrDefault(tp, 0);
if (recordsSinceLastCommit != 0) {
taskHandle.commit(recordsSinceLastCommit);
log.debug("Forwarding to framework request to commit {} records for {}", recordsSinceLastCommit, tp);
committedOffsets.put(tp, 0);
}
});
return offsets;
}
@Override
public void stop() {
log.info("Stopped {} task {}", this.getClass().getSimpleName(), taskId);
taskHandle.recordTaskStop();
}
}
}

View File

@ -0,0 +1,314 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.kafka.connect.integration;
import org.apache.kafka.clients.producer.RecordMetadata;
import org.apache.kafka.common.config.ConfigDef;
import org.apache.kafka.connect.connector.Task;
import org.apache.kafka.connect.data.Schema;
import org.apache.kafka.connect.errors.ConnectException;
import org.apache.kafka.connect.header.ConnectHeaders;
import org.apache.kafka.connect.runtime.SampleSourceConnector;
import org.apache.kafka.connect.source.ConnectorTransactionBoundaries;
import org.apache.kafka.connect.source.ExactlyOnceSupport;
import org.apache.kafka.connect.source.SourceRecord;
import org.apache.kafka.connect.source.SourceTask;
import org.apache.kafka.server.util.ThroughputThrottler;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Optional;
import java.util.stream.Collectors;
import java.util.stream.LongStream;
/**
* A source connector that is used in Apache Kafka integration tests to verify the behavior of
* the Connect framework, but that can be used in other integration tests as a simple connector
* that generates records of a fixed structure. The rate of record production can be adjusted
* through the configs 'throughput' and 'messages.per.poll'
*/
public class TestableSourceConnector extends SampleSourceConnector {
private static final Logger log = LoggerFactory.getLogger(TestableSourceConnector.class);
public static final String TOPIC_CONFIG = "topic";
public static final String NUM_TASKS = "num.tasks";
public static final String MESSAGES_PER_POLL_CONFIG = "messages.per.poll";
public static final String MAX_MESSAGES_PER_SECOND_CONFIG = "throughput";
public static final String MAX_MESSAGES_PRODUCED_CONFIG = "max.messages";
public static final String CUSTOM_EXACTLY_ONCE_SUPPORT_CONFIG = "custom.exactly.once.support";
public static final String EXACTLY_ONCE_SUPPORTED = "supported";
public static final String EXACTLY_ONCE_UNSUPPORTED = "unsupported";
public static final String EXACTLY_ONCE_NULL = "null";
public static final String EXACTLY_ONCE_FAIL = "fail";
public static final String CUSTOM_TRANSACTION_BOUNDARIES_CONFIG = "custom.transaction.boundaries";
public static final String TRANSACTION_BOUNDARIES_SUPPORTED = "supported";
public static final String TRANSACTION_BOUNDARIES_UNSUPPORTED = "unsupported";
public static final String TRANSACTION_BOUNDARIES_NULL = "null";
public static final String TRANSACTION_BOUNDARIES_FAIL = "fail";
// Boolean valued configuration that determines whether TestableSourceConnector::alterOffsets should return true or false
public static final String ALTER_OFFSETS_RESULT = "alter.offsets.result";
private String connectorName;
private ConnectorHandle connectorHandle;
private Map<String, String> commonConfigs;
@Override
public void start(Map<String, String> props) {
connectorHandle = RuntimeHandles.get().connectorHandle(props.get("name"));
connectorName = connectorHandle.name();
commonConfigs = props;
log.info("Started {} connector {}", this.getClass().getSimpleName(), connectorName);
connectorHandle.recordConnectorStart();
if (Boolean.parseBoolean(props.getOrDefault("connector.start.inject.error", "false"))) {
throw new RuntimeException("Injecting errors during connector start");
}
}
@Override
public Class<? extends Task> taskClass() {
return TestableSourceTask.class;
}
@Override
public List<Map<String, String>> taskConfigs(int maxTasks) {
String numTasksProp = commonConfigs.get(NUM_TASKS);
int numTasks = numTasksProp != null ? Integer.parseInt(numTasksProp) : maxTasks;
List<Map<String, String>> configs = new ArrayList<>();
for (int i = 0; i < numTasks; i++) {
Map<String, String> config = taskConfig(commonConfigs, connectorName, i);
configs.add(config);
}
return configs;
}
public static Map<String, String> taskConfig(
Map<String, String> connectorProps,
String connectorName,
int taskNum
) {
Map<String, String> result = new HashMap<>(connectorProps);
result.put("connector.name", connectorName);
result.put("task.id", taskId(connectorName, taskNum));
return result;
}
@Override
public void stop() {
log.info("Stopped {} connector {}", this.getClass().getSimpleName(), connectorName);
connectorHandle.recordConnectorStop();
if (Boolean.parseBoolean(commonConfigs.getOrDefault("connector.stop.inject.error", "false"))) {
throw new RuntimeException("Injecting errors during connector stop");
}
}
@Override
public ConfigDef config() {
log.info("Configured {} connector {}", this.getClass().getSimpleName(), connectorName);
return new ConfigDef();
}
@Override
public ExactlyOnceSupport exactlyOnceSupport(Map<String, String> connectorConfig) {
String supportLevel = connectorConfig.getOrDefault(CUSTOM_EXACTLY_ONCE_SUPPORT_CONFIG, "null").toLowerCase(Locale.ROOT);
switch (supportLevel) {
case EXACTLY_ONCE_SUPPORTED:
return ExactlyOnceSupport.SUPPORTED;
case EXACTLY_ONCE_UNSUPPORTED:
return ExactlyOnceSupport.UNSUPPORTED;
case EXACTLY_ONCE_FAIL:
throw new ConnectException("oops");
default:
case EXACTLY_ONCE_NULL:
return null;
}
}
@Override
public ConnectorTransactionBoundaries canDefineTransactionBoundaries(Map<String, String> connectorConfig) {
String supportLevel = connectorConfig.getOrDefault(CUSTOM_TRANSACTION_BOUNDARIES_CONFIG, TRANSACTION_BOUNDARIES_UNSUPPORTED).toLowerCase(Locale.ROOT);
switch (supportLevel) {
case TRANSACTION_BOUNDARIES_SUPPORTED:
return ConnectorTransactionBoundaries.SUPPORTED;
case TRANSACTION_BOUNDARIES_FAIL:
throw new ConnectException("oh no :(");
case TRANSACTION_BOUNDARIES_NULL:
return null;
default:
case TRANSACTION_BOUNDARIES_UNSUPPORTED:
return ConnectorTransactionBoundaries.UNSUPPORTED;
}
}
@Override
public boolean alterOffsets(Map<String, String> connectorConfig, Map<Map<String, ?>, Map<String, ?>> offsets) {
return Boolean.parseBoolean(connectorConfig.get(ALTER_OFFSETS_RESULT));
}
public static String taskId(String connectorName, int taskId) {
return connectorName + "-" + taskId;
}
public static class TestableSourceTask extends SourceTask {
private String taskId;
private String topicName;
private TaskHandle taskHandle;
private volatile boolean stopped;
private long startingSeqno;
private long seqno;
private int batchSize;
private ThroughputThrottler throttler;
private long maxMessages;
private long priorTransactionBoundary;
private long nextTransactionBoundary;
@Override
public String version() {
return "unknown";
}
@Override
public void start(Map<String, String> props) {
taskId = props.get("task.id");
String connectorName = props.get("connector.name");
topicName = props.getOrDefault(TOPIC_CONFIG, "sequential-topic");
batchSize = Integer.parseInt(props.getOrDefault(MESSAGES_PER_POLL_CONFIG, "1"));
taskHandle = RuntimeHandles.get().connectorHandle(connectorName).taskHandle(taskId);
Map<String, Object> offset = Optional.ofNullable(
context.offsetStorageReader().offset(sourcePartition(taskId)))
.orElse(Collections.emptyMap());
startingSeqno = Optional.ofNullable((Long) offset.get("saved")).orElse(0L);
seqno = startingSeqno;
log.info("Started {} task {} with properties {}", this.getClass().getSimpleName(), taskId, props);
throttler = new ThroughputThrottler(Long.parseLong(props.getOrDefault(MAX_MESSAGES_PER_SECOND_CONFIG, "-1")), System.currentTimeMillis());
maxMessages = Long.parseLong(props.getOrDefault(MAX_MESSAGES_PRODUCED_CONFIG, String.valueOf(Long.MAX_VALUE)));
taskHandle.recordTaskStart();
priorTransactionBoundary = 0;
nextTransactionBoundary = 1;
if (Boolean.parseBoolean(props.getOrDefault("task-" + taskId + ".start.inject.error", "false"))) {
throw new RuntimeException("Injecting errors during task start");
}
calculateNextBoundary();
}
@Override
public List<SourceRecord> poll() {
if (!stopped) {
// Don't return any more records since we've already produced the configured maximum number.
if (seqno >= maxMessages) {
return null;
}
if (throttler.shouldThrottle(seqno - startingSeqno, System.currentTimeMillis())) {
throttler.throttle();
}
int currentBatchSize = (int) Math.min(maxMessages - seqno, batchSize);
taskHandle.record(currentBatchSize);
log.trace("Returning batch of {} records", currentBatchSize);
return LongStream.range(0, currentBatchSize)
.mapToObj(i -> {
seqno++;
SourceRecord record = new SourceRecord(
sourcePartition(taskId),
sourceOffset(seqno),
topicName,
null,
Schema.STRING_SCHEMA,
"key-" + taskId + "-" + seqno,
Schema.STRING_SCHEMA,
"value-" + taskId + "-" + seqno,
null,
new ConnectHeaders().addLong("header-" + seqno, seqno));
maybeDefineTransactionBoundary(record);
return record;
})
.collect(Collectors.toList());
}
return null;
}
@Override
public void commit() {
log.info("Task {} committing offsets", taskId);
//TODO: save progress outside the offset topic, potentially in the task handle
}
@Override
public void commitRecord(SourceRecord record, RecordMetadata metadata) {
log.trace("Committing record: {}", record);
taskHandle.commit();
}
@Override
public void stop() {
log.info("Stopped {} task {}", this.getClass().getSimpleName(), taskId);
stopped = true;
taskHandle.recordTaskStop();
}
/**
* Calculate the next transaction boundary, i.e., the seqno whose corresponding source record should be used to
* either {@link org.apache.kafka.connect.source.TransactionContext#commitTransaction(SourceRecord) commit}
* or {@link org.apache.kafka.connect.source.TransactionContext#abortTransaction(SourceRecord) abort} the next transaction.
* <p>
* This connector defines transactions whose size correspond to successive elements of the Fibonacci sequence,
* where transactions with an even number of records are aborted, and those with an odd number of records are committed.
*/
private void calculateNextBoundary() {
while (nextTransactionBoundary <= seqno) {
nextTransactionBoundary += priorTransactionBoundary;
priorTransactionBoundary = nextTransactionBoundary - priorTransactionBoundary;
}
}
private void maybeDefineTransactionBoundary(SourceRecord record) {
if (context.transactionContext() == null || seqno != nextTransactionBoundary) {
return;
}
long transactionSize = nextTransactionBoundary - priorTransactionBoundary;
// If the transaction boundary ends on an even-numbered offset, abort it
// Otherwise, commit
boolean abort = nextTransactionBoundary % 2 == 0;
calculateNextBoundary();
if (abort) {
log.info("Aborting transaction of {} records", transactionSize);
context.transactionContext().abortTransaction(record);
} else {
log.info("Committing transaction of {} records", transactionSize);
context.transactionContext().commitTransaction(record);
}
}
}
public static Map<String, Object> sourcePartition(String taskId) {
return Collections.singletonMap("task.id", taskId);
}
public static Map<String, Object> sourceOffset(long seqno) {
return Collections.singletonMap("saved", seqno);
}
}

View File

@ -63,8 +63,8 @@ public class TransformationIntegrationTest {
private static final int NUM_TASKS = 1;
private static final int NUM_WORKERS = 3;
private static final String CONNECTOR_NAME = "simple-conn";
private static final String SINK_CONNECTOR_CLASS_NAME = MonitorableSinkConnector.class.getSimpleName();
private static final String SOURCE_CONNECTOR_CLASS_NAME = MonitorableSourceConnector.class.getSimpleName();
private static final String SINK_CONNECTOR_CLASS_NAME = TestableSinkConnector.class.getSimpleName();
private static final String SOURCE_CONNECTOR_CLASS_NAME = TestableSourceConnector.class.getSimpleName();
private EmbeddedConnectCluster connect;
private ConnectorHandle connectorHandle;

View File

@ -18,11 +18,16 @@ package org.apache.kafka.connect.runtime;
import org.apache.kafka.clients.CommonClientConfigs;
import org.apache.kafka.clients.producer.ProducerConfig;
import org.apache.kafka.common.MetricName;
import org.apache.kafka.common.config.ConfigDef;
import org.apache.kafka.common.config.ConfigTransformer;
import org.apache.kafka.common.config.ConfigValue;
import org.apache.kafka.common.config.SaslConfigs;
import org.apache.kafka.common.config.provider.DirectoryConfigProvider;
import org.apache.kafka.common.metrics.KafkaMetric;
import org.apache.kafka.common.metrics.Measurable;
import org.apache.kafka.common.metrics.Monitorable;
import org.apache.kafka.common.metrics.PluginMetrics;
import org.apache.kafka.common.security.oauthbearer.internals.unsecured.OAuthBearerUnsecuredLoginCallbackHandler;
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.connect.connector.Connector;
@ -781,6 +786,51 @@ public class AbstractHerderTest {
verifyValidationIsolation();
}
static final class TestClientConfigOverridePolicy extends AllConnectorClientConfigOverridePolicy implements Monitorable {
private static MetricName metricName = null;
private int count = 0;
@Override
protected boolean isAllowed(ConfigValue configValue) {
count++;
return super.isAllowed(configValue);
}
@Override
public void withPluginMetrics(PluginMetrics metrics) {
metricName = metrics.metricName("name", "description", Map.of());
metrics.addMetric(metricName, (Measurable) (config, now) -> count);
}
}
@Test
public void testClientConfigOverridePolicyWithMetrics() {
final Class<? extends Connector> connectorClass = SampleSourceConnector.class;
AbstractHerder herder = createConfigValidationHerder(connectorClass, new TestClientConfigOverridePolicy());
Map<String, String> config = new HashMap<>();
config.put(ConnectorConfig.CONNECTOR_CLASS_CONFIG, connectorClass.getName());
config.put(ConnectorConfig.NAME_CONFIG, "connector-name");
config.put("required", "value");
Map<String, String> overrides = Map.of(
producerOverrideKey(ProducerConfig.MAX_REQUEST_SIZE_CONFIG), "420",
producerOverrideKey(ProducerConfig.MAX_BLOCK_MS_CONFIG), "28980",
producerOverrideKey(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG), "true");
config.putAll(overrides);
herder.validateConnectorConfig(config, s -> null, false);
Map<MetricName, KafkaMetric> metrics = herder.worker.metrics().metrics().metrics();
assertTrue(metrics.containsKey(TestClientConfigOverridePolicy.metricName));
assertEquals((double) overrides.size(), metrics.get(TestClientConfigOverridePolicy.metricName).metricValue());
herder.stopServices();
metrics = herder.worker.metrics().metrics().metrics();
assertFalse(metrics.containsKey(TestClientConfigOverridePolicy.metricName));
}
@Test
public void testReverseTransformConfigs() {
// Construct a task config with constant values for TEST_KEY and TEST_KEY2
@ -1280,6 +1330,8 @@ public class AbstractHerderTest {
}
private AbstractHerder testHerder(ConnectorClientConfigOverridePolicy connectorClientConfigOverridePolicy) {
ConnectMetrics connectMetrics = new MockConnectMetrics();
when(worker.metrics()).thenReturn(connectMetrics);
return mock(AbstractHerder.class, withSettings()
.useConstructor(worker, workerId, kafkaClusterId, statusStore, configStore, connectorClientConfigOverridePolicy, Time.SYSTEM)
.defaultAnswer(CALLS_REAL_METHODS));

View File

@ -29,13 +29,14 @@ import org.apache.kafka.common.TopicPartitionInfo;
import org.apache.kafka.common.errors.TopicAuthorizationException;
import org.apache.kafka.common.header.Headers;
import org.apache.kafka.common.header.internals.RecordHeaders;
import org.apache.kafka.common.internals.Plugin;
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.connect.data.Schema;
import org.apache.kafka.connect.data.SchemaAndValue;
import org.apache.kafka.connect.errors.ConnectException;
import org.apache.kafka.connect.errors.RetriableException;
import org.apache.kafka.connect.header.ConnectHeaders;
import org.apache.kafka.connect.integration.MonitorableSourceConnector;
import org.apache.kafka.connect.integration.TestableSourceConnector;
import org.apache.kafka.connect.runtime.errors.ErrorHandlingMetrics;
import org.apache.kafka.connect.runtime.errors.ErrorReporter;
import org.apache.kafka.connect.runtime.errors.ProcessingContext;
@ -46,6 +47,7 @@ import org.apache.kafka.connect.runtime.standalone.StandaloneConfig;
import org.apache.kafka.connect.source.SourceRecord;
import org.apache.kafka.connect.source.SourceTask;
import org.apache.kafka.connect.storage.CloseableOffsetStorageReader;
import org.apache.kafka.connect.storage.ClusterConfigState;
import org.apache.kafka.connect.storage.ConnectorOffsetBackingStore;
import org.apache.kafka.connect.storage.Converter;
import org.apache.kafka.connect.storage.HeaderConverter;
@ -82,7 +84,7 @@ import java.util.function.Supplier;
import java.util.stream.Collectors;
import java.util.stream.Stream;
import static org.apache.kafka.connect.integration.MonitorableSourceConnector.TOPIC_CONFIG;
import static org.apache.kafka.connect.integration.TestableSourceConnector.TOPIC_CONFIG;
import static org.apache.kafka.connect.runtime.ConnectorConfig.CONNECTOR_CLASS_CONFIG;
import static org.apache.kafka.connect.runtime.ConnectorConfig.KEY_CONVERTER_CLASS_CONFIG;
import static org.apache.kafka.connect.runtime.ConnectorConfig.TASKS_MAX_CONFIG;
@ -131,8 +133,7 @@ public class AbstractWorkerSourceTaskTest {
private static final byte[] SERIALIZED_KEY = "converted-key".getBytes();
private static final byte[] SERIALIZED_RECORD = "converted-record".getBytes();
@Mock
private SourceTask sourceTask;
@Mock private SourceTask sourceTask;
@Mock private TopicAdmin admin;
@Mock private KafkaProducer<byte[], byte[]> producer;
@Mock private Converter keyConverter;
@ -143,8 +144,9 @@ public class AbstractWorkerSourceTaskTest {
@Mock private OffsetStorageWriter offsetWriter;
@Mock private ConnectorOffsetBackingStore offsetStore;
@Mock private StatusBackingStore statusBackingStore;
@Mock private WorkerSourceTaskContext sourceTaskContext;
@Mock private WorkerTransactionContext workerTransactionContext;
@Mock private TaskStatus.Listener statusListener;
@Mock private ClusterConfigState configState;
private final ConnectorTaskId taskId = new ConnectorTaskId("job", 0);
private final ConnectorTaskId taskId1 = new ConnectorTaskId("job", 1);
@ -179,7 +181,7 @@ public class AbstractWorkerSourceTaskTest {
// setup up props for the source connector
Map<String, String> props = new HashMap<>();
props.put("name", "foo-connector");
props.put(CONNECTOR_CLASS_CONFIG, MonitorableSourceConnector.class.getSimpleName());
props.put(CONNECTOR_CLASS_CONFIG, TestableSourceConnector.class.getSimpleName());
props.put(TASKS_MAX_CONFIG, String.valueOf(1));
props.put(TOPIC_CONFIG, TOPIC);
props.put(KEY_CONVERTER_CLASS_CONFIG, StringConverter.class.getName());
@ -815,7 +817,7 @@ public class AbstractWorkerSourceTaskTest {
workerTask.toSend = Arrays.asList(record1, record2, record3);
// With errors.tolerance to all, the faiiled conversion should simply skip the record, and record successful batch
// With errors.tolerance to all, the failed conversion should simply skip the record, and record successful batch
assertTrue(workerTask.sendRecords());
}
@ -956,10 +958,13 @@ public class AbstractWorkerSourceTaskTest {
private void createWorkerTask(Converter keyConverter, Converter valueConverter, HeaderConverter headerConverter,
RetryWithToleranceOperator<SourceRecord> retryWithToleranceOperator, Supplier<List<ErrorReporter<SourceRecord>>> errorReportersSupplier,
TransformationChain transformationChain) {
TransformationChain<SourceRecord, SourceRecord> transformationChain) {
Plugin<Converter> keyConverterPlugin = metrics.wrap(keyConverter, taskId, true);
Plugin<Converter> valueConverterPlugin = metrics.wrap(valueConverter, taskId, false);
Plugin<HeaderConverter> headerConverterPlugin = metrics.wrap(headerConverter, taskId);
workerTask = new AbstractWorkerSourceTask(
taskId, sourceTask, statusListener, TargetState.STARTED, keyConverter, valueConverter, headerConverter, transformationChain,
sourceTaskContext, producer, admin, TopicCreationGroup.configuredGroups(sourceConfig), offsetReader, offsetWriter, offsetStore,
taskId, sourceTask, statusListener, TargetState.STARTED, configState, keyConverterPlugin, valueConverterPlugin, headerConverterPlugin, transformationChain,
workerTransactionContext, producer, admin, TopicCreationGroup.configuredGroups(sourceConfig), offsetReader, offsetWriter, offsetStore,
config, metrics, errorHandlingMetrics, plugins.delegatingLoader(), Time.SYSTEM, retryWithToleranceOperator,
statusBackingStore, Runnable::run, errorReportersSupplier) {
@Override

View File

@ -18,19 +18,36 @@ package org.apache.kafka.connect.runtime;
import org.apache.kafka.clients.CommonClientConfigs;
import org.apache.kafka.common.MetricName;
import org.apache.kafka.common.config.ConfigDef;
import org.apache.kafka.common.metrics.KafkaMetric;
import org.apache.kafka.common.metrics.Measurable;
import org.apache.kafka.common.metrics.Monitorable;
import org.apache.kafka.common.metrics.PluginMetrics;
import org.apache.kafka.common.metrics.Sensor;
import org.apache.kafka.common.metrics.internals.PluginMetricsImpl;
import org.apache.kafka.common.metrics.stats.Avg;
import org.apache.kafka.common.metrics.stats.Max;
import org.apache.kafka.common.utils.MockTime;
import org.apache.kafka.connect.data.Schema;
import org.apache.kafka.connect.data.SchemaAndValue;
import org.apache.kafka.connect.runtime.ConnectMetrics.MetricGroup;
import org.apache.kafka.connect.runtime.ConnectMetrics.MetricGroupId;
import org.apache.kafka.connect.source.SourceRecord;
import org.apache.kafka.connect.storage.Converter;
import org.apache.kafka.connect.storage.HeaderConverter;
import org.apache.kafka.connect.transforms.Transformation;
import org.apache.kafka.connect.transforms.predicates.Predicate;
import org.apache.kafka.connect.util.ConnectorTaskId;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.ValueSource;
import java.util.Collections;
import java.io.IOException;
import java.util.HashMap;
import java.util.LinkedHashMap;
import java.util.Map;
import static org.junit.jupiter.api.Assertions.assertEquals;
@ -43,12 +60,11 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
public class ConnectMetricsTest {
private static final Map<String, String> DEFAULT_WORKER_CONFIG = new HashMap<>();
static {
DEFAULT_WORKER_CONFIG.put(WorkerConfig.KEY_CONVERTER_CLASS_CONFIG, "org.apache.kafka.connect.json.JsonConverter");
DEFAULT_WORKER_CONFIG.put(WorkerConfig.VALUE_CONVERTER_CLASS_CONFIG, "org.apache.kafka.connect.json.JsonConverter");
}
private static final Map<String, String> DEFAULT_WORKER_CONFIG = Map.of(
WorkerConfig.KEY_CONVERTER_CLASS_CONFIG, "org.apache.kafka.connect.json.JsonConverter",
WorkerConfig.VALUE_CONVERTER_CLASS_CONFIG, "org.apache.kafka.connect.json.JsonConverter");
private static final ConnectorTaskId CONNECTOR_TASK_ID = new ConnectorTaskId("connector", 0);
private static final Map<String, String> TAGS = Map.of("t1", "v1");
private ConnectMetrics metrics;
@ -171,6 +187,221 @@ public class ConnectMetricsTest {
cm.stop();
}
@Test
public void testConnectorPluginMetrics() throws Exception {
try (PluginMetricsImpl pluginMetrics = metrics.connectorPluginMetrics(CONNECTOR_TASK_ID.connector())) {
MetricName metricName = pluginMetrics.metricName("name", "description", TAGS);
Map<String, String> expectedTags = new LinkedHashMap<>();
expectedTags.put("connector", CONNECTOR_TASK_ID.connector());
expectedTags.putAll(TAGS);
assertEquals(expectedTags, metricName.tags());
}
}
@Test
public void testTaskPluginMetrics() throws Exception {
try (PluginMetricsImpl pluginMetrics = metrics.taskPluginMetrics(CONNECTOR_TASK_ID)) {
MetricName metricName = pluginMetrics.metricName("name", "description", TAGS);
Map<String, String> expectedTags = new LinkedHashMap<>();
expectedTags.put("connector", CONNECTOR_TASK_ID.connector());
expectedTags.put("task", String.valueOf(CONNECTOR_TASK_ID.task()));
expectedTags.putAll(TAGS);
assertEquals(expectedTags, metricName.tags());
}
}
static final class MonitorableConverter implements Converter, HeaderConverter, Monitorable {
private int calls = 0;
private PluginMetrics pluginMetrics = null;
private MetricName metricName = null;
@Override
public void withPluginMetrics(PluginMetrics pluginMetrics) {
this.pluginMetrics = pluginMetrics;
metricName = pluginMetrics.metricName("name", "description", TAGS);
pluginMetrics.addMetric(metricName, (Measurable) (config, now) -> calls);
}
@Override
public void configure(Map<String, ?> configs, boolean isKey) { }
@Override
public byte[] fromConnectData(String topic, Schema schema, Object value) {
calls++;
return new byte[0];
}
@Override
public SchemaAndValue toConnectData(String topic, byte[] value) {
calls++;
return null;
}
@Override
public ConfigDef config() {
return Converter.super.config();
}
@Override
public void configure(Map<String, ?> configs) { }
@Override
public SchemaAndValue toConnectHeader(String topic, String headerKey, byte[] value) {
calls++;
return null;
}
@Override
public byte[] fromConnectHeader(String topic, String headerKey, Schema schema, Object value) {
calls++;
return new byte[0];
}
}
@ParameterizedTest
@ValueSource(booleans = {true, false})
public void testWrapConverter(boolean isKey) throws IOException {
try (MonitorableConverter converter = new MonitorableConverter()) {
metrics.wrap(converter, CONNECTOR_TASK_ID, isKey);
assertNotNull(converter.pluginMetrics);
MetricName metricName = converter.metricName;
Map<String, String> expectedTags = new LinkedHashMap<>();
expectedTags.put("connector", CONNECTOR_TASK_ID.connector());
expectedTags.put("task", String.valueOf(CONNECTOR_TASK_ID.task()));
expectedTags.put("converter", isKey ? "key" : "value");
expectedTags.putAll(TAGS);
assertEquals(expectedTags, metricName.tags());
KafkaMetric metric = metrics.metrics().metrics().get(metricName);
assertEquals(0.0, (double) metric.metricValue());
converter.toConnectData("topic", new byte[]{});
assertEquals(1.0, (double) metric.metricValue());
converter.fromConnectData("topic", null, null);
assertEquals(2.0, (double) metric.metricValue());
}
}
@Test
public void testWrapHeaderConverter() throws IOException {
try (MonitorableConverter converter = new MonitorableConverter()) {
metrics.wrap(converter, CONNECTOR_TASK_ID);
assertNotNull(converter.pluginMetrics);
MetricName metricName = converter.metricName;
Map<String, String> expectedTags = new LinkedHashMap<>();
expectedTags.put("connector", CONNECTOR_TASK_ID.connector());
expectedTags.put("task", String.valueOf(CONNECTOR_TASK_ID.task()));
expectedTags.put("converter", "header");
expectedTags.putAll(TAGS);
assertEquals(expectedTags, metricName.tags());
KafkaMetric metric = metrics.metrics().metrics().get(metricName);
assertEquals(0.0, (double) metric.metricValue());
converter.toConnectHeader("topic", "header", new byte[]{});
assertEquals(1.0, (double) metric.metricValue());
converter.fromConnectHeader("topic", "header", null, null);
assertEquals(2.0, (double) metric.metricValue());
}
}
static final class MonitorableTransformation implements Transformation<SourceRecord>, Monitorable {
private int calls = 0;
private PluginMetrics pluginMetrics = null;
private MetricName metricName = null;
@Override
public void withPluginMetrics(PluginMetrics pluginMetrics) {
this.pluginMetrics = pluginMetrics;
metricName = pluginMetrics.metricName("name", "description", TAGS);
pluginMetrics.addMetric(metricName, (Measurable) (config, now) -> calls);
}
@Override
public void configure(Map<String, ?> configs) { }
@Override
public SourceRecord apply(SourceRecord record) {
calls++;
return null;
}
@Override
public ConfigDef config() {
return null;
}
@Override
public void close() { }
}
@Test
public void testWrapTransformation() {
try (MonitorableTransformation transformation = new MonitorableTransformation()) {
metrics.wrap(transformation, CONNECTOR_TASK_ID, "alias");
assertNotNull(transformation.pluginMetrics);
MetricName metricName = transformation.metricName;
Map<String, String> expectedTags = new LinkedHashMap<>();
expectedTags.put("connector", CONNECTOR_TASK_ID.connector());
expectedTags.put("task", String.valueOf(CONNECTOR_TASK_ID.task()));
expectedTags.put("transformation", "alias");
expectedTags.putAll(TAGS);
assertEquals(expectedTags, metricName.tags());
KafkaMetric metric = metrics.metrics().metrics().get(metricName);
assertEquals(0.0, (double) metric.metricValue());
transformation.apply(null);
assertEquals(1.0, (double) metric.metricValue());
}
}
static final class MonitorablePredicate implements Predicate<SourceRecord>, Monitorable {
private int calls = 0;
private PluginMetrics pluginMetrics = null;
private MetricName metricName = null;
@Override
public void withPluginMetrics(PluginMetrics pluginMetrics) {
this.pluginMetrics = pluginMetrics;
metricName = pluginMetrics.metricName("name", "description", TAGS);
pluginMetrics.addMetric(metricName, (Measurable) (config, now) -> calls);
}
@Override
public void configure(Map<String, ?> configs) { }
@Override
public ConfigDef config() {
return null;
}
@Override
public boolean test(SourceRecord record) {
calls++;
return false;
}
@Override
public void close() { }
}
@Test
public void testWrapPredicate() {
try (MonitorablePredicate predicate = new MonitorablePredicate()) {
metrics.wrap(predicate, CONNECTOR_TASK_ID, "alias");
assertNotNull(predicate.pluginMetrics);
MetricName metricName = predicate.metricName;
Map<String, String> expectedTags = new LinkedHashMap<>();
expectedTags.put("connector", CONNECTOR_TASK_ID.connector());
expectedTags.put("task", String.valueOf(CONNECTOR_TASK_ID.task()));
expectedTags.put("predicate", "alias");
expectedTags.putAll(TAGS);
assertEquals(expectedTags, metricName.tags());
KafkaMetric metric = metrics.metrics().metrics().get(metricName);
assertEquals(0.0, (double) metric.metricValue());
predicate.test(null);
assertEquals(1.0, (double) metric.metricValue());
}
}
private Sensor addToGroup(ConnectMetrics connectMetrics, boolean shouldClose) {
ConnectMetricsRegistry registry = connectMetrics.registry();
ConnectMetrics.MetricGroup metricGroup = connectMetrics.group(registry.taskGroupName(),
@ -188,6 +419,6 @@ public class ConnectMetricsTest {
}
static MetricName metricName(String name) {
return new MetricName(name, "test_group", "metrics for testing", Collections.emptyMap());
return new MetricName(name, "test_group", "metrics for testing", Map.of());
}
}

View File

@ -26,6 +26,7 @@ import org.apache.kafka.connect.runtime.isolation.Plugins;
import org.apache.kafka.connect.sink.SinkRecord;
import org.apache.kafka.connect.transforms.Transformation;
import org.apache.kafka.connect.transforms.predicates.Predicate;
import org.apache.kafka.connect.util.ConnectorTaskId;
import org.junit.jupiter.api.Test;
@ -46,6 +47,9 @@ import static org.mockito.Mockito.when;
public class ConnectorConfigTest<R extends ConnectRecord<R>> {
private static final ConnectMetrics METRICS = new MockConnectMetrics();
private static final ConnectorTaskId CONNECTOR_TASK_ID = new ConnectorTaskId("test", 0);
public static final Plugins MOCK_PLUGINS = new Plugins(new HashMap<>()) {
@Override
public Set<PluginDesc<Transformation<?>>> transformations() {
@ -159,7 +163,7 @@ public class ConnectorConfigTest<R extends ConnectRecord<R>> {
props.put("transforms.a.type", SimpleTransformation.class.getName());
props.put("transforms.a.magic.number", "42");
final ConnectorConfig config = new ConnectorConfig(MOCK_PLUGINS, props);
final List<TransformationStage<SinkRecord>> transformationStages = config.transformationStages();
final List<TransformationStage<SinkRecord>> transformationStages = config.transformationStages(CONNECTOR_TASK_ID, METRICS);
assertEquals(1, transformationStages.size());
final TransformationStage<SinkRecord> stage = transformationStages.get(0);
assertEquals(SimpleTransformation.class, stage.transformClass());
@ -188,7 +192,7 @@ public class ConnectorConfigTest<R extends ConnectRecord<R>> {
props.put("transforms.b.type", SimpleTransformation.class.getName());
props.put("transforms.b.magic.number", "84");
final ConnectorConfig config = new ConnectorConfig(MOCK_PLUGINS, props);
final List<TransformationStage<SinkRecord>> transformationStages = config.transformationStages();
final List<TransformationStage<SinkRecord>> transformationStages = config.transformationStages(CONNECTOR_TASK_ID, METRICS);
assertEquals(2, transformationStages.size());
assertEquals(42, transformationStages.get(0).apply(DUMMY_RECORD).kafkaPartition().intValue());
assertEquals(84, transformationStages.get(1).apply(DUMMY_RECORD).kafkaPartition().intValue());
@ -289,7 +293,7 @@ public class ConnectorConfigTest<R extends ConnectRecord<R>> {
private void assertTransformationStageWithPredicate(Map<String, String> props, boolean expectedNegated) {
final ConnectorConfig config = new ConnectorConfig(MOCK_PLUGINS, props);
final List<TransformationStage<SinkRecord>> transformationStages = config.transformationStages();
final List<TransformationStage<SinkRecord>> transformationStages = config.transformationStages(CONNECTOR_TASK_ID, METRICS);
assertEquals(1, transformationStages.size());
TransformationStage<SinkRecord> stage = transformationStages.get(0);

View File

@ -25,6 +25,7 @@ import org.apache.kafka.clients.consumer.OffsetAndMetadata;
import org.apache.kafka.clients.producer.KafkaProducer;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.config.ConfigDef;
import org.apache.kafka.common.internals.Plugin;
import org.apache.kafka.common.utils.MockTime;
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.connect.components.Versioned;
@ -33,7 +34,7 @@ import org.apache.kafka.connect.data.Schema;
import org.apache.kafka.connect.data.SchemaBuilder;
import org.apache.kafka.connect.data.Struct;
import org.apache.kafka.connect.errors.RetriableException;
import org.apache.kafka.connect.integration.MonitorableSourceConnector;
import org.apache.kafka.connect.integration.TestableSourceConnector;
import org.apache.kafka.connect.json.JsonConverter;
import org.apache.kafka.connect.runtime.errors.ErrorHandlingMetrics;
import org.apache.kafka.connect.runtime.errors.ErrorReporter;
@ -83,7 +84,7 @@ import java.util.Set;
import static java.util.Collections.emptyMap;
import static java.util.Collections.singletonList;
import static org.apache.kafka.common.utils.Time.SYSTEM;
import static org.apache.kafka.connect.integration.MonitorableSourceConnector.TOPIC_CONFIG;
import static org.apache.kafka.connect.integration.TestableSourceConnector.TOPIC_CONFIG;
import static org.apache.kafka.connect.runtime.ConnectorConfig.CONNECTOR_CLASS_CONFIG;
import static org.apache.kafka.connect.runtime.ConnectorConfig.KEY_CONVERTER_CLASS_CONFIG;
import static org.apache.kafka.connect.runtime.ConnectorConfig.TASKS_MAX_CONFIG;
@ -195,7 +196,7 @@ public class ErrorHandlingTaskTest {
// setup up props for the source connector
Map<String, String> props = new HashMap<>();
props.put("name", "foo-connector");
props.put(CONNECTOR_CLASS_CONFIG, MonitorableSourceConnector.class.getSimpleName());
props.put(CONNECTOR_CLASS_CONFIG, TestableSourceConnector.class.getSimpleName());
props.put(TASKS_MAX_CONFIG, String.valueOf(1));
props.put(TOPIC_CONFIG, topic);
props.put(KEY_CONVERTER_CLASS_CONFIG, StringConverter.class.getName());
@ -424,13 +425,17 @@ public class ErrorHandlingTaskTest {
oo.put("schemas.enable", "false");
converter.configure(oo);
Plugin<Transformation<SinkRecord>> transformationPlugin = metrics.wrap(new FaultyPassthrough<SinkRecord>(), taskId, "");
TransformationChain<ConsumerRecord<byte[], byte[]>, SinkRecord> sinkTransforms =
new TransformationChain<>(singletonList(new TransformationStage<>(new FaultyPassthrough<SinkRecord>())), retryWithToleranceOperator);
new TransformationChain<>(singletonList(new TransformationStage<>(transformationPlugin)), retryWithToleranceOperator);
Plugin<Converter> keyConverterPlugin = metrics.wrap(converter, taskId, true);
Plugin<Converter> valueConverterPlugin = metrics.wrap(converter, taskId, false);
Plugin<HeaderConverter> headerConverterPlugin = metrics.wrap(headerConverter, taskId);
workerSinkTask = new WorkerSinkTask(
taskId, sinkTask, statusListener, initialState, workerConfig,
ClusterConfigState.EMPTY, metrics, converter, converter, errorHandlingMetrics,
headerConverter, sinkTransforms, consumer, pluginLoader, time,
ClusterConfigState.EMPTY, metrics, keyConverterPlugin, valueConverterPlugin, errorHandlingMetrics,
headerConverterPlugin, sinkTransforms, consumer, pluginLoader, time,
retryWithToleranceOperator, workerErrantRecordReporter,
statusBackingStore, () -> errorReporters);
}
@ -456,12 +461,16 @@ public class ErrorHandlingTaskTest {
private void createSourceTask(TargetState initialState, RetryWithToleranceOperator<SourceRecord> retryWithToleranceOperator,
List<ErrorReporter<SourceRecord>> errorReporters, Converter converter) {
Plugin<Transformation<SourceRecord>> transformationPlugin = metrics.wrap(new FaultyPassthrough<SourceRecord>(), taskId, "");
TransformationChain<SourceRecord, SourceRecord> sourceTransforms = new TransformationChain<>(singletonList(
new TransformationStage<>(new FaultyPassthrough<SourceRecord>())), retryWithToleranceOperator);
new TransformationStage<>(transformationPlugin)), retryWithToleranceOperator);
Plugin<Converter> keyConverterPlugin = metrics.wrap(converter, taskId, true);
Plugin<Converter> valueConverterPlugin = metrics.wrap(converter, taskId, false);
Plugin<HeaderConverter> headerConverterPlugin = metrics.wrap(headerConverter, taskId);
workerSourceTask = spy(new WorkerSourceTask(
taskId, sourceTask, statusListener, initialState, converter,
converter, errorHandlingMetrics, headerConverter,
taskId, sourceTask, statusListener, initialState, keyConverterPlugin,
valueConverterPlugin, errorHandlingMetrics, headerConverterPlugin,
sourceTransforms, producer, admin,
TopicCreationGroup.configuredGroups(sourceConfig),
offsetReader, offsetWriter, offsetStore, workerConfig,

View File

@ -26,11 +26,12 @@ import org.apache.kafka.common.errors.RecordTooLargeException;
import org.apache.kafka.common.header.Header;
import org.apache.kafka.common.header.Headers;
import org.apache.kafka.common.header.internals.RecordHeaders;
import org.apache.kafka.common.internals.Plugin;
import org.apache.kafka.common.utils.MockTime;
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.connect.data.Schema;
import org.apache.kafka.connect.errors.ConnectException;
import org.apache.kafka.connect.integration.MonitorableSourceConnector;
import org.apache.kafka.connect.integration.TestableSourceConnector;
import org.apache.kafka.connect.runtime.ConnectMetrics.MetricGroup;
import org.apache.kafka.connect.runtime.errors.ErrorHandlingMetrics;
import org.apache.kafka.connect.runtime.errors.RetryWithToleranceOperatorTest;
@ -83,7 +84,7 @@ import java.util.function.Consumer;
import java.util.stream.Collectors;
import static java.util.Collections.emptySet;
import static org.apache.kafka.connect.integration.MonitorableSourceConnector.TOPIC_CONFIG;
import static org.apache.kafka.connect.integration.TestableSourceConnector.TOPIC_CONFIG;
import static org.apache.kafka.connect.runtime.ConnectorConfig.CONNECTOR_CLASS_CONFIG;
import static org.apache.kafka.connect.runtime.ConnectorConfig.KEY_CONVERTER_CLASS_CONFIG;
import static org.apache.kafka.connect.runtime.ConnectorConfig.TASKS_MAX_CONFIG;
@ -248,7 +249,7 @@ public class ExactlyOnceWorkerSourceTaskTest {
// setup up props for the source connector
Map<String, String> props = new HashMap<>();
props.put("name", "foo-connector");
props.put(CONNECTOR_CLASS_CONFIG, MonitorableSourceConnector.class.getSimpleName());
props.put(CONNECTOR_CLASS_CONFIG, TestableSourceConnector.class.getSimpleName());
props.put(TASKS_MAX_CONFIG, String.valueOf(1));
props.put(TOPIC_CONFIG, TOPIC);
props.put(KEY_CONVERTER_CLASS_CONFIG, StringConverter.class.getName());
@ -276,7 +277,10 @@ public class ExactlyOnceWorkerSourceTaskTest {
}
private void createWorkerTask(TargetState initialState, Converter keyConverter, Converter valueConverter, HeaderConverter headerConverter) {
workerTask = new ExactlyOnceWorkerSourceTask(taskId, sourceTask, statusListener, initialState, keyConverter, valueConverter, headerConverter,
Plugin<Converter> keyConverterPlugin = metrics.wrap(keyConverter, taskId, true);
Plugin<Converter> valueConverterPlugin = metrics.wrap(valueConverter, taskId, false);
Plugin<HeaderConverter> headerConverterPlugin = metrics.wrap(headerConverter, taskId);
workerTask = new ExactlyOnceWorkerSourceTask(taskId, sourceTask, statusListener, initialState, keyConverterPlugin, valueConverterPlugin, headerConverterPlugin,
transformationChain, producer, admin, TopicCreationGroup.configuredGroups(sourceConfig), offsetReader, offsetWriter, offsetStore,
config, clusterConfigState, metrics, errorHandlingMetrics, plugins.delegatingLoader(), time, RetryWithToleranceOperatorTest.noneOperator(), statusBackingStore,
sourceConfig, Runnable::run, preProducerCheck, postProducerCheck, Collections::emptyList);

View File

@ -16,6 +16,7 @@
*/
package org.apache.kafka.connect.runtime;
import org.apache.kafka.common.internals.Plugin;
import org.apache.kafka.connect.source.SourceRecord;
import org.apache.kafka.connect.transforms.Transformation;
import org.apache.kafka.connect.transforms.predicates.Predicate;
@ -38,33 +39,34 @@ public class TransformationStageTest {
private final SourceRecord transformed = new SourceRecord(singletonMap("transformed", 2), null, null, null, null);
@Test
public void apply() {
public void apply() throws Exception {
applyAndAssert(true, false, transformed);
applyAndAssert(true, true, initial);
applyAndAssert(false, false, initial);
applyAndAssert(false, true, transformed);
}
private void applyAndAssert(boolean predicateResult, boolean negate,
SourceRecord expectedResult) {
@SuppressWarnings("unchecked")
@SuppressWarnings("unchecked")
private void applyAndAssert(boolean predicateResult, boolean negate, SourceRecord expectedResult) throws Exception {
Plugin<Predicate<SourceRecord>> predicatePlugin = mock(Plugin.class);
Predicate<SourceRecord> predicate = mock(Predicate.class);
when(predicate.test(any())).thenReturn(predicateResult);
@SuppressWarnings("unchecked")
when(predicatePlugin.get()).thenReturn(predicate);
Plugin<Transformation<SourceRecord>> transformationPlugin = mock(Plugin.class);
Transformation<SourceRecord> transformation = mock(Transformation.class);
if (expectedResult == transformed) {
when(transformationPlugin.get()).thenReturn(transformation);
when(transformation.apply(any())).thenReturn(transformed);
}
TransformationStage<SourceRecord> stage = new TransformationStage<>(
predicate,
predicatePlugin,
negate,
transformation);
transformationPlugin);
assertEquals(expectedResult, stage.apply(initial));
stage.close();
verify(predicate).close();
verify(transformation).close();
verify(predicatePlugin).close();
verify(transformationPlugin).close();
}
}

View File

@ -16,6 +16,7 @@
*/
package org.apache.kafka.connect.runtime;
import org.apache.kafka.clients.consumer.Consumer;
import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
import org.apache.kafka.clients.consumer.ConsumerRecord;
import org.apache.kafka.clients.consumer.ConsumerRecords;
@ -30,6 +31,7 @@ import org.apache.kafka.common.errors.WakeupException;
import org.apache.kafka.common.header.Header;
import org.apache.kafka.common.header.Headers;
import org.apache.kafka.common.header.internals.RecordHeaders;
import org.apache.kafka.common.internals.Plugin;
import org.apache.kafka.common.record.RecordBatch;
import org.apache.kafka.common.record.TimestampType;
import org.apache.kafka.common.utils.MockTime;
@ -204,11 +206,28 @@ public class WorkerSinkTaskTest {
private void createTask(TargetState initialState, Converter keyConverter, Converter valueConverter, HeaderConverter headerConverter,
RetryWithToleranceOperator<ConsumerRecord<byte[], byte[]>> retryWithToleranceOperator,
Supplier<List<ErrorReporter<ConsumerRecord<byte[], byte[]>>>> errorReportersSupplier,
TransformationChain transformationChain) {
workerTask = new WorkerSinkTask(
taskId, sinkTask, statusListener, initialState, workerConfig, ClusterConfigState.EMPTY, metrics,
TransformationChain<ConsumerRecord<byte[], byte[]>, SinkRecord> transformationChain) {
createTask(taskId, sinkTask, statusListener, initialState, workerConfig, metrics,
keyConverter, valueConverter, errorHandlingMetrics, headerConverter,
transformationChain, consumer, pluginLoader, time,
retryWithToleranceOperator, statusBackingStore, errorReportersSupplier);
}
private void createTask(ConnectorTaskId taskId, SinkTask task, TaskStatus.Listener statusListener, TargetState initialState,
WorkerConfig workerConfig, ConnectMetrics connectMetrics, Converter keyConverter, Converter valueConverter,
ErrorHandlingMetrics errorMetrics, HeaderConverter headerConverter,
TransformationChain<ConsumerRecord<byte[], byte[]>, SinkRecord> transformationChain,
Consumer<byte[], byte[]> consumer, ClassLoader loader, Time time,
RetryWithToleranceOperator<ConsumerRecord<byte[], byte[]>> retryWithToleranceOperator,
StatusBackingStore statusBackingStore,
Supplier<List<ErrorReporter<ConsumerRecord<byte[], byte[]>>>> errorReportersSupplier) {
Plugin<Converter> keyConverterPlugin = connectMetrics.wrap(keyConverter, taskId, true);
Plugin<Converter> valueConverterPlugin = connectMetrics.wrap(valueConverter, taskId, false);
Plugin<HeaderConverter> headerConverterPlugin = connectMetrics.wrap(headerConverter, taskId);
workerTask = new WorkerSinkTask(
taskId, task, statusListener, initialState, workerConfig, ClusterConfigState.EMPTY, connectMetrics,
keyConverterPlugin, valueConverterPlugin, errorMetrics, headerConverterPlugin,
transformationChain, consumer, loader, time,
retryWithToleranceOperator, null, statusBackingStore, errorReportersSupplier);
}
@ -1857,11 +1876,10 @@ public class WorkerSinkTaskTest {
public void testPartitionCountInCaseOfPartitionRevocation() {
MockConsumer<byte[], byte[]> mockConsumer = new MockConsumer<>(AutoOffsetResetStrategy.EARLIEST.name());
// Setting up Worker Sink Task to check metrics
workerTask = new WorkerSinkTask(
taskId, sinkTask, statusListener, TargetState.PAUSED, workerConfig, ClusterConfigState.EMPTY, metrics,
createTask(taskId, sinkTask, statusListener, TargetState.PAUSED, workerConfig, metrics,
keyConverter, valueConverter, errorHandlingMetrics, headerConverter,
transformationChain, mockConsumer, pluginLoader, time,
RetryWithToleranceOperatorTest.noneOperator(), null, statusBackingStore, Collections::emptyList);
RetryWithToleranceOperatorTest.noneOperator(), statusBackingStore, Collections::emptyList);
mockConsumer.updateBeginningOffsets(
new HashMap<>() {{
put(TOPIC_PARTITION, 0L);

View File

@ -24,6 +24,7 @@ import org.apache.kafka.clients.consumer.OffsetAndMetadata;
import org.apache.kafka.clients.consumer.OffsetCommitCallback;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.header.internals.RecordHeaders;
import org.apache.kafka.common.internals.Plugin;
import org.apache.kafka.common.record.TimestampType;
import org.apache.kafka.common.utils.MockTime;
import org.apache.kafka.common.utils.Time;
@ -174,9 +175,12 @@ public class WorkerSinkTaskThreadedTest {
workerProps.put("value.converter", "org.apache.kafka.connect.json.JsonConverter");
workerProps.put("offset.storage.file.filename", "/tmp/connect.offsets");
WorkerConfig workerConfig = new StandaloneConfig(workerProps);
Plugin<Converter> keyConverterPlugin = metrics.wrap(keyConverter, taskId, true);
Plugin<Converter> valueConverterPlugin = metrics.wrap(valueConverter, taskId, false);
Plugin<HeaderConverter> headerConverterPlugin = metrics.wrap(headerConverter, taskId);
workerTask = new WorkerSinkTask(
taskId, sinkTask, statusListener, initialState, workerConfig, ClusterConfigState.EMPTY, metrics, keyConverter,
valueConverter, errorHandlingMetrics, headerConverter, transformationChain,
taskId, sinkTask, statusListener, initialState, workerConfig, ClusterConfigState.EMPTY, metrics, keyConverterPlugin,
valueConverterPlugin, errorHandlingMetrics, headerConverterPlugin, transformationChain,
consumer, pluginLoader, time, RetryWithToleranceOperatorTest.noneOperator(), null, statusBackingStore,
Collections::emptyList);
recordsReturned = 0;

View File

@ -27,11 +27,12 @@ import org.apache.kafka.common.errors.InvalidTopicException;
import org.apache.kafka.common.errors.TopicAuthorizationException;
import org.apache.kafka.common.header.Headers;
import org.apache.kafka.common.header.internals.RecordHeaders;
import org.apache.kafka.common.internals.Plugin;
import org.apache.kafka.common.utils.LogCaptureAppender;
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.connect.data.Schema;
import org.apache.kafka.connect.errors.ConnectException;
import org.apache.kafka.connect.integration.MonitorableSourceConnector;
import org.apache.kafka.connect.integration.TestableSourceConnector;
import org.apache.kafka.connect.runtime.ConnectMetrics.MetricGroup;
import org.apache.kafka.connect.runtime.errors.ErrorHandlingMetrics;
import org.apache.kafka.connect.runtime.errors.RetryWithToleranceOperator;
@ -85,7 +86,7 @@ import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.function.Supplier;
import static org.apache.kafka.connect.integration.MonitorableSourceConnector.TOPIC_CONFIG;
import static org.apache.kafka.connect.integration.TestableSourceConnector.TOPIC_CONFIG;
import static org.apache.kafka.connect.runtime.ConnectorConfig.CONNECTOR_CLASS_CONFIG;
import static org.apache.kafka.connect.runtime.ConnectorConfig.KEY_CONVERTER_CLASS_CONFIG;
import static org.apache.kafka.connect.runtime.ConnectorConfig.TASKS_MAX_CONFIG;
@ -209,7 +210,7 @@ public class WorkerSourceTaskTest {
// setup up props for the source connector
Map<String, String> props = new HashMap<>();
props.put("name", "foo-connector");
props.put(CONNECTOR_CLASS_CONFIG, MonitorableSourceConnector.class.getSimpleName());
props.put(CONNECTOR_CLASS_CONFIG, TestableSourceConnector.class.getSimpleName());
props.put(TASKS_MAX_CONFIG, String.valueOf(1));
props.put(TOPIC_CONFIG, topic);
props.put(KEY_CONVERTER_CLASS_CONFIG, StringConverter.class.getName());
@ -247,7 +248,10 @@ public class WorkerSourceTaskTest {
private void createWorkerTask(TargetState initialState, Converter keyConverter, Converter valueConverter,
HeaderConverter headerConverter, RetryWithToleranceOperator<SourceRecord> retryWithToleranceOperator) {
workerTask = new WorkerSourceTask(taskId, sourceTask, statusListener, initialState, keyConverter, valueConverter, errorHandlingMetrics, headerConverter,
Plugin<Converter> keyConverterPlugin = metrics.wrap(keyConverter, taskId, true);
Plugin<Converter> valueConverterPlugin = metrics.wrap(valueConverter, taskId, false);
Plugin<HeaderConverter> headerConverterPlugin = metrics.wrap(headerConverter, taskId);
workerTask = new WorkerSourceTask(taskId, sourceTask, statusListener, initialState, keyConverterPlugin, valueConverterPlugin, errorHandlingMetrics, headerConverterPlugin,
transformationChain, producer, admin, TopicCreationGroup.configuredGroups(sourceConfig),
offsetReader, offsetWriter, offsetStore, config, clusterConfigState, metrics, plugins.delegatingLoader(), Time.SYSTEM,
retryWithToleranceOperator, statusBackingStore, Runnable::run, Collections::emptyList);

View File

@ -16,6 +16,7 @@
*/
package org.apache.kafka.connect.runtime;
import org.apache.kafka.common.internals.Plugin;
import org.apache.kafka.connect.connector.ConnectRecord;
import org.apache.kafka.connect.runtime.distributed.ExtendedAssignment;
import org.apache.kafka.connect.runtime.errors.RetryWithToleranceOperator;
@ -181,15 +182,20 @@ public class WorkerTestUtils {
return buildTransformationChain(transformation, toleranceOperator);
}
@SuppressWarnings("unchecked")
public static <T, R extends ConnectRecord<R>> TransformationChain<T, R> buildTransformationChain(
Transformation<R> transformation,
RetryWithToleranceOperator<T> toleranceOperator) {
Predicate<R> predicate = mock(Predicate.class);
when(predicate.test(any())).thenReturn(true);
TransformationStage<R> stage = new TransformationStage(
predicate,
Plugin<Predicate<R>> predicatePlugin = mock(Plugin.class);
when(predicatePlugin.get()).thenReturn(predicate);
Plugin<Transformation<R>> transformationPlugin = mock(Plugin.class);
when(transformationPlugin.get()).thenReturn(transformation);
TransformationStage<R> stage = new TransformationStage<>(
predicatePlugin,
false,
transformation);
transformationPlugin);
TransformationChain<T, R> realTransformationChainRetriableException = new TransformationChain(List.of(stage), toleranceOperator);
TransformationChain<T, R> transformationChainRetriableException = Mockito.spy(realTransformationChainRetriableException);
return transformationChainRetriableException;

View File

@ -312,6 +312,7 @@ public class DistributedHerderTest {
public void setUp() throws Exception {
time = new MockTime();
metrics = new MockConnectMetrics(time);
when(worker.metrics()).thenReturn(metrics);
AutoCloseable uponShutdown = shutdownCalled::countDown;
// Default to the old protocol unless specified otherwise

View File

@ -16,11 +16,19 @@
*/
package org.apache.kafka.connect.runtime.rest;
import org.apache.kafka.common.MetricName;
import org.apache.kafka.common.config.AbstractConfig;
import org.apache.kafka.common.metrics.Gauge;
import org.apache.kafka.common.metrics.KafkaMetric;
import org.apache.kafka.common.metrics.Monitorable;
import org.apache.kafka.common.metrics.PluginMetrics;
import org.apache.kafka.common.utils.LogCaptureAppender;
import org.apache.kafka.connect.rest.ConnectRestExtension;
import org.apache.kafka.connect.rest.ConnectRestExtensionContext;
import org.apache.kafka.connect.runtime.Herder;
import org.apache.kafka.connect.runtime.MockConnectMetrics;
import org.apache.kafka.connect.runtime.isolation.Plugins;
import org.apache.kafka.connect.runtime.isolation.PluginsTest;
import org.apache.kafka.connect.runtime.rest.entities.LoggerLevel;
import com.fasterxml.jackson.core.type.TypeReference;
@ -56,18 +64,21 @@ import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import jakarta.ws.rs.core.MediaType;
import static org.junit.jupiter.api.Assertions.assertArrayEquals;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertNotEquals;
import static org.junit.jupiter.api.Assertions.assertNull;
import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.ArgumentMatchers.eq;
import static org.mockito.Mockito.doReturn;
import static org.mockito.Mockito.reset;
@ExtendWith(MockitoExtension.class)
@MockitoSettings(strictness = Strictness.STRICT_STUBS)
@ -85,6 +96,7 @@ public class ConnectRestServerTest {
@BeforeEach
public void setUp() {
httpClient = HttpClients.createMinimal();
doReturn(new MockConnectMetrics()).when(herder).connectMetrics();
}
@AfterEach
@ -118,6 +130,9 @@ public class ConnectRestServerTest {
@Test
public void testAdvertisedUri() {
// Clear stubs not needed by this test
reset(herder);
// Advertised URI from listeners without protocol
Map<String, String> configMap = new HashMap<>(baseServerProps());
configMap.put(RestServerConfig.LISTENERS_CONFIG, "http://localhost:8080,https://localhost:8443");
@ -375,6 +390,44 @@ public class ConnectRestServerTest {
checkCustomizedHttpResponseHeaders(headerConfig, expectedHeaders);
}
static final class MonitorableConnectRestExtension extends PluginsTest.TestConnectRestExtension implements Monitorable {
private boolean called = false;
private static MetricName metricName;
@Override
public void register(ConnectRestExtensionContext restPluginContext) {
called = true;
}
@Override
public void withPluginMetrics(PluginMetrics metrics) {
metricName = metrics.metricName("name", "description", Map.of());
metrics.addMetric(metricName, (Gauge<Boolean>) (config, now) -> called);
}
}
@Test
public void testMonitorableConnectRestExtension() {
Map<String, String> configMap = new HashMap<>(baseServerProps());
configMap.put(RestServerConfig.REST_EXTENSION_CLASSES_CONFIG, MonitorableConnectRestExtension.class.getName());
doReturn(plugins).when(herder).plugins();
doReturn(List.of(new MonitorableConnectRestExtension())).when(plugins).newPlugins(any(), any(), eq(ConnectRestExtension.class));
server = new ConnectRestServer(null, restClient, configMap);
server.initializeServer();
server.initializeResources(herder);
Map<MetricName, KafkaMetric> metrics = herder.connectMetrics().metrics().metrics();
assertTrue(metrics.containsKey(MonitorableConnectRestExtension.metricName));
assertTrue((boolean) metrics.get(MonitorableConnectRestExtension.metricName).metricValue());
server.stop();
metrics = herder.connectMetrics().metrics().metrics();
assertFalse(metrics.containsKey(MonitorableConnectRestExtension.metricName));
}
private void checkCustomizedHttpResponseHeaders(String headerConfig, Map<String, String> expectedHeaders)
throws IOException {
Map<String, String> configMap = baseServerProps();

View File

@ -30,6 +30,7 @@ import org.apache.kafka.connect.runtime.ConnectorConfig;
import org.apache.kafka.connect.runtime.ConnectorStatus;
import org.apache.kafka.connect.runtime.Herder;
import org.apache.kafka.connect.runtime.HerderConnectorContext;
import org.apache.kafka.connect.runtime.MockConnectMetrics;
import org.apache.kafka.connect.runtime.RestartPlan;
import org.apache.kafka.connect.runtime.RestartRequest;
import org.apache.kafka.connect.runtime.SinkConnectorConfig;
@ -149,6 +150,7 @@ public class StandaloneHerderTest {
public void initialize(boolean mockTransform) {
when(worker.getPlugins()).thenReturn(plugins);
when(worker.metrics()).thenReturn(new MockConnectMetrics());
herder = mock(StandaloneHerder.class, withSettings()
.useConstructor(worker, WORKER_ID, KAFKA_CLUSTER_ID, statusBackingStore, new MemoryConfigBackingStore(transformer), noneConnectorClientConfigOverridePolicy, new MockTime())
.defaultAnswer(CALLS_REAL_METHODS));

View File

@ -19,6 +19,8 @@ package org.apache.kafka.connect.util;
import org.apache.kafka.clients.admin.NewTopic;
import org.apache.kafka.connect.data.Schema;
import org.apache.kafka.connect.runtime.ConnectMetrics;
import org.apache.kafka.connect.runtime.MockConnectMetrics;
import org.apache.kafka.connect.runtime.SourceConnectorConfig;
import org.apache.kafka.connect.runtime.TransformationStage;
import org.apache.kafka.connect.runtime.WorkerConfig;
@ -80,6 +82,8 @@ public class TopicCreationTest {
private static final short DEFAULT_REPLICATION_FACTOR = -1;
private static final int DEFAULT_PARTITIONS = -1;
private static final ConnectMetrics METRICS = new MockConnectMetrics();
private static final ConnectorTaskId CONNECTOR_TASK_ID = new ConnectorTaskId("test", 0);
Map<String, String> workerProps;
WorkerConfig workerConfig;
@ -515,7 +519,7 @@ public class TopicCreationTest {
topicCreation.addTopic(FOO_TOPIC);
assertFalse(topicCreation.isTopicCreationRequired(FOO_TOPIC));
List<TransformationStage<SourceRecord>> transformationStages = sourceConfig.transformationStages();
List<TransformationStage<SourceRecord>> transformationStages = sourceConfig.transformationStages(CONNECTOR_TASK_ID, METRICS);
assertEquals(1, transformationStages.size());
TransformationStage<SourceRecord> xform = transformationStages.get(0);
SourceRecord transformed = xform.apply(new SourceRecord(null, null, "topic", 0, null, null, Schema.INT8_SCHEMA, 42));
@ -622,7 +626,7 @@ public class TopicCreationTest {
assertEquals(barPartitions, barTopicSpec.numPartitions());
assertEquals(barTopicProps, barTopicSpec.configs());
List<TransformationStage<SourceRecord>> transformationStages = sourceConfig.transformationStages();
List<TransformationStage<SourceRecord>> transformationStages = sourceConfig.transformationStages(CONNECTOR_TASK_ID, METRICS);
assertEquals(2, transformationStages.size());
TransformationStage<SourceRecord> castXForm = transformationStages.get(0);

View File

@ -18,6 +18,7 @@ package org.apache.kafka.connect.util.clusters;
import org.apache.kafka.connect.cli.ConnectStandalone;
import org.apache.kafka.connect.runtime.Connect;
import org.apache.kafka.connect.runtime.ConnectMetrics;
import org.apache.kafka.connect.runtime.standalone.StandaloneHerder;
import org.apache.kafka.test.TestUtils;
@ -62,6 +63,7 @@ public class EmbeddedConnectStandalone extends EmbeddedConnect {
private final String offsetsFile;
private volatile WorkerHandle connectWorker;
private Connect<StandaloneHerder> connect;
private EmbeddedConnectStandalone(
int numBrokers,
@ -92,7 +94,7 @@ public class EmbeddedConnectStandalone extends EmbeddedConnect {
workerProps.putIfAbsent(PLUGIN_DISCOVERY_CONFIG, "hybrid_fail");
ConnectStandalone cli = new ConnectStandalone();
Connect<StandaloneHerder> connect = cli.startConnect(workerProps);
connect = cli.startConnect(workerProps);
connectWorker = new WorkerHandle("standalone", connect);
cli.processExtraArgs(connect, connectorConfigFiles());
}
@ -137,6 +139,10 @@ public class EmbeddedConnectStandalone extends EmbeddedConnect {
return result;
}
public ConnectMetrics connectMetrics() {
return connect.herder().connectMetrics();
}
public static class Builder extends EmbeddedConnectBuilder<EmbeddedConnectStandalone, Builder> {
private final List<Map<String, String>> connectorConfigs = new ArrayList<>();

View File

@ -17,5 +17,6 @@ org.apache.kafka.connect.integration.BlockingConnectorTest$BlockingSinkConnector
org.apache.kafka.connect.integration.BlockingConnectorTest$TaskInitializeBlockingSinkConnector
org.apache.kafka.connect.integration.ErrantRecordSinkConnector
org.apache.kafka.connect.integration.MonitorableSinkConnector
org.apache.kafka.connect.integration.TestableSinkConnector
org.apache.kafka.connect.runtime.SampleSinkConnector
org.apache.kafka.connect.integration.ConnectWorkerIntegrationTest$EmptyTaskConfigsConnector

View File

@ -21,5 +21,6 @@ org.apache.kafka.connect.integration.BlockingConnectorTest$BlockingSourceConnect
org.apache.kafka.connect.integration.BlockingConnectorTest$TaskInitializeBlockingSourceConnector
org.apache.kafka.connect.integration.ExactlyOnceSourceIntegrationTest$NaughtyConnector
org.apache.kafka.connect.integration.MonitorableSourceConnector
org.apache.kafka.connect.integration.TestableSourceConnector
org.apache.kafka.connect.runtime.SampleSourceConnector
org.apache.kafka.connect.runtime.rest.resources.ConnectorPluginsResourceTest$ConnectorPluginsResourceTestConnector