mirror of https://github.com/apache/kafka.git
KAFKA-15995: Adding KIP-877 support to Connect (#17804)
Reviewers: Greg Harris <gharris1727@gmail.com>
This commit is contained in:
parent
0989a6a389
commit
c13324fc16
|
@ -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"/>
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -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 {
|
||||
|
||||
|
|
|
@ -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 {
|
||||
|
||||
|
|
|
@ -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();
|
||||
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -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
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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 {
|
||||
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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 +
|
||||
'}';
|
||||
}
|
||||
|
|
|
@ -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));
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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{" +
|
||||
|
|
|
@ -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);
|
||||
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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.
|
||||
*
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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");
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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");
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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;
|
||||
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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));
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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));
|
||||
|
|
|
@ -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");
|
||||
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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));
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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;
|
||||
|
|
|
@ -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));
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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));
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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<>();
|
||||
|
|
|
@ -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
|
|
@ -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
|
||||
|
|
Loading…
Reference in New Issue