diff --git a/checkstyle/suppressions.xml b/checkstyle/suppressions.xml index 74e696103cdf6..98e5d3bc52ef7 100644 --- a/checkstyle/suppressions.xml +++ b/checkstyle/suppressions.xml @@ -99,16 +99,13 @@ - + files="(DistributedHerder|Worker).java"/> + + files="(KafkaConfigBackingStore|Values).java"/> - + files="Worker(SinkTask|SourceTask|Coordinator).java"/> @@ -119,40 +116,27 @@ files="JsonConverter.java"/> - - - - + files="(FileStreamSourceTask|DistributedHerder|KafkaConfigBackingStore).java"/> - + files="(ConnectRecord|JsonConverter|Values|ConnectHeader|ConnectHeaders).java"/> - - - - - + files="(KafkaConfigBackingStore|Values).java"/> + files="(DistributedHerder|RestClient|RestServer|JsonConverter|KafkaConfigBackingStore|FileStreamSourceTask|TopicAdmin).java"/> + files="(DistributedHerder|KafkaBasedLog|WorkerSourceTaskWithTopicCreation)Test.java"/> + files="(WorkerSink|WorkerSource|ErrorHandling)Task(|WithTopicCreation)Test.java"/> + + + + diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/SourceConnectorConfig.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/SourceConnectorConfig.java index a0ee46893acaa..5e175a5042eb8 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/SourceConnectorConfig.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/SourceConnectorConfig.java @@ -16,21 +16,167 @@ */ package org.apache.kafka.connect.runtime; +import org.apache.kafka.common.config.AbstractConfig; import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.connect.runtime.isolation.Plugins; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; import java.util.Map; +import java.util.stream.Collectors; + +import static org.apache.kafka.connect.runtime.TopicCreationConfig.DEFAULT_TOPIC_CREATION_GROUP; +import static org.apache.kafka.connect.runtime.TopicCreationConfig.DEFAULT_TOPIC_CREATION_PREFIX; +import static org.apache.kafka.connect.runtime.TopicCreationConfig.EXCLUDE_REGEX_CONFIG; +import static org.apache.kafka.connect.runtime.TopicCreationConfig.INCLUDE_REGEX_CONFIG; +import static org.apache.kafka.connect.runtime.TopicCreationConfig.PARTITIONS_CONFIG; +import static org.apache.kafka.connect.runtime.TopicCreationConfig.REPLICATION_FACTOR_CONFIG; public class SourceConnectorConfig extends ConnectorConfig { - private static ConfigDef config = ConnectorConfig.configDef(); + protected static final String TOPIC_CREATION_GROUP = "Topic Creation"; + + public static final String TOPIC_CREATION_PREFIX = "topic.creation."; + + public static final String TOPIC_CREATION_GROUPS_CONFIG = TOPIC_CREATION_PREFIX + "groups"; + private static final String TOPIC_CREATION_GROUPS_DOC = "Groups of configurations for topics " + + "created by source connectors"; + private static final String TOPIC_CREATION_GROUPS_DISPLAY = "Topic Creation Groups"; + + private static class EnrichedSourceConnectorConfig extends AbstractConfig { + EnrichedSourceConnectorConfig(ConfigDef configDef, Map props) { + super(configDef, props); + } + + @Override + public Object get(String key) { + return super.get(key); + } + } + + private static ConfigDef config = SourceConnectorConfig.configDef(); + private final EnrichedSourceConnectorConfig enrichedSourceConfig; public static ConfigDef configDef() { - return config; + int orderInGroup = 0; + return new ConfigDef(ConnectorConfig.configDef()) + .define(TOPIC_CREATION_GROUPS_CONFIG, ConfigDef.Type.LIST, Collections.emptyList(), + ConfigDef.CompositeValidator.of(new ConfigDef.NonNullValidator(), ConfigDef.LambdaValidator.with( + (name, value) -> { + List groupAliases = (List) value; + if (groupAliases.size() > new HashSet<>(groupAliases).size()) { + throw new ConfigException(name, value, "Duplicate alias provided."); + } + }, + () -> "unique topic creation groups")), + ConfigDef.Importance.LOW, TOPIC_CREATION_GROUPS_DOC, TOPIC_CREATION_GROUP, + ++orderInGroup, ConfigDef.Width.LONG, TOPIC_CREATION_GROUPS_DISPLAY); } - public SourceConnectorConfig(Plugins plugins, Map props) { + public static ConfigDef embedDefaultGroup(ConfigDef baseConfigDef) { + String defaultGroup = "default"; + ConfigDef newDefaultDef = new ConfigDef(baseConfigDef); + newDefaultDef.embed(DEFAULT_TOPIC_CREATION_PREFIX, defaultGroup, 0, TopicCreationConfig.defaultGroupConfigDef()); + return newDefaultDef; + } + + /** + * Returns an enriched {@link ConfigDef} building upon the {@code ConfigDef}, using the current configuration specified in {@code props} as an input. + * + * @param baseConfigDef the base configuration definition to be enriched + * @param props the non parsed configuration properties + * @return the enriched configuration definition + */ + public static ConfigDef enrich(ConfigDef baseConfigDef, Map props, AbstractConfig defaultGroupConfig) { + List topicCreationGroups = new ArrayList<>(); + Object aliases = ConfigDef.parseType(TOPIC_CREATION_GROUPS_CONFIG, props.get(TOPIC_CREATION_GROUPS_CONFIG), ConfigDef.Type.LIST); + if (aliases instanceof List) { + topicCreationGroups.addAll((List) aliases); + } + + ConfigDef newDef = new ConfigDef(baseConfigDef); + String defaultGroupPrefix = TOPIC_CREATION_PREFIX + DEFAULT_TOPIC_CREATION_GROUP + "."; + short defaultGroupReplicationFactor = defaultGroupConfig.getShort(defaultGroupPrefix + REPLICATION_FACTOR_CONFIG); + int defaultGroupPartitions = defaultGroupConfig.getInt(defaultGroupPrefix + PARTITIONS_CONFIG); + topicCreationGroups.stream().distinct().forEach(group -> { + if (!(group instanceof String)) { + throw new ConfigException("Item in " + TOPIC_CREATION_GROUPS_CONFIG + " property is not of type String"); + } + String alias = (String) group; + String prefix = TOPIC_CREATION_PREFIX + alias + "."; + String configGroup = TOPIC_CREATION_GROUP + ": " + alias; + newDef.embed(prefix, configGroup, 0, + TopicCreationConfig.configDef(configGroup, defaultGroupReplicationFactor, defaultGroupPartitions)); + }); + return newDef; + } + + public SourceConnectorConfig(Plugins plugins, Map props, boolean createTopics) { super(plugins, config, props); + if (createTopics && props.entrySet().stream().anyMatch(e -> e.getKey().startsWith(TOPIC_CREATION_PREFIX))) { + ConfigDef defaultConfigDef = embedDefaultGroup(config); + // This config is only used to set default values for partitions and replication + // factor from the default group and otherwise it remains unused + AbstractConfig defaultGroup = new AbstractConfig(defaultConfigDef, props, false); + + // If the user has added regex of include or exclude patterns in the default group, + // they should be ignored. + Map propsWithoutRegexForDefaultGroup = new HashMap<>(props); + propsWithoutRegexForDefaultGroup.entrySet() + .removeIf(e -> e.getKey().equals(DEFAULT_TOPIC_CREATION_PREFIX + INCLUDE_REGEX_CONFIG) + || e.getKey().equals(DEFAULT_TOPIC_CREATION_PREFIX + EXCLUDE_REGEX_CONFIG)); + enrichedSourceConfig = new EnrichedSourceConnectorConfig(enrich(defaultConfigDef, props, + defaultGroup), propsWithoutRegexForDefaultGroup); + } else { + enrichedSourceConfig = null; + } + } + + @Override + public Object get(String key) { + return enrichedSourceConfig != null ? enrichedSourceConfig.get(key) : super.get(key); + } + + /** + * Returns whether this configuration uses topic creation properties. + * + * @return true if the configuration should be validated and used for topic creation; false otherwise + */ + public boolean usesTopicCreation() { + return enrichedSourceConfig != null; + } + + public List topicCreationInclude(String group) { + return getList(TOPIC_CREATION_PREFIX + group + '.' + INCLUDE_REGEX_CONFIG); + } + + public List topicCreationExclude(String group) { + return getList(TOPIC_CREATION_PREFIX + group + '.' + EXCLUDE_REGEX_CONFIG); + } + + public Short topicCreationReplicationFactor(String group) { + return getShort(TOPIC_CREATION_PREFIX + group + '.' + REPLICATION_FACTOR_CONFIG); + } + + public Integer topicCreationPartitions(String group) { + return getInt(TOPIC_CREATION_PREFIX + group + '.' + PARTITIONS_CONFIG); + } + + public Map topicCreationOtherConfigs(String group) { + if (enrichedSourceConfig == null) { + return Collections.emptyMap(); + } + return enrichedSourceConfig.originalsWithPrefix(TOPIC_CREATION_PREFIX + group + '.').entrySet().stream() + .filter(e -> { + String key = e.getKey(); + return !(INCLUDE_REGEX_CONFIG.equals(key) || EXCLUDE_REGEX_CONFIG.equals(key) + || REPLICATION_FACTOR_CONFIG.equals(key) || PARTITIONS_CONFIG.equals(key)); + }) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); } public static void main(String[] args) { diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/TopicCreationConfig.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/TopicCreationConfig.java new file mode 100644 index 0000000000000..e5c5f15704e0b --- /dev/null +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/TopicCreationConfig.java @@ -0,0 +1,142 @@ +/* + * 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.runtime; + +import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.common.config.ConfigException; +import org.apache.kafka.connect.util.TopicAdmin; + +import java.util.Collections; +import java.util.List; +import java.util.regex.Pattern; +import java.util.regex.PatternSyntaxException; + +public class TopicCreationConfig { + + public static final String DEFAULT_TOPIC_CREATION_PREFIX = "topic.creation.default."; + public static final String DEFAULT_TOPIC_CREATION_GROUP = "default"; + + public static final String INCLUDE_REGEX_CONFIG = "include"; + private static final String INCLUDE_REGEX_DOC = "A list of regular expression literals " + + "used to match the topic names used by the source connector. This list is used " + + "to include topics that should be created using the topic settings defined by this group."; + + public static final String EXCLUDE_REGEX_CONFIG = "exclude"; + private static final String EXCLUDE_REGEX_DOC = "A list of regular expression literals " + + "used to match the topic names used by the source connector. This list is used " + + "to exclude topics from being created with the topic settings defined by this group. " + + "Note that exclusion rules have precedent and override any inclusion rules for the topics."; + + public static final String REPLICATION_FACTOR_CONFIG = "replication.factor"; + private static final String REPLICATION_FACTOR_DOC = "The replication factor for new topics " + + "created for this connector using this group. This value may be -1 to use the broker's" + + "default replication factor, or may be a positive number not larger than the number of " + + "brokers in the Kafka cluster. A value larger than the number of brokers in the Kafka cluster " + + "will result in an error when the new topic is created. For the default group this configuration " + + "is required. For any other group defined in topic.creation.groups this config is " + + "optional and if it's missing it gets the value of the default group"; + + public static final String PARTITIONS_CONFIG = "partitions"; + private static final String PARTITIONS_DOC = "The number of partitions new topics created for " + + "this connector. This value may be -1 to use the broker's default number of partitions, " + + "or a positive number representing the desired number of partitions. " + + "For the default group this configuration is required. For any " + + "other group defined in topic.creation.groups this config is optional and if it's " + + "missing it gets the value of the default group"; + + public static final ConfigDef.Validator REPLICATION_FACTOR_VALIDATOR = ConfigDef.LambdaValidator.with( + (name, value) -> validateReplicationFactor(name, (short) value), + () -> "Positive number not larger than the number of brokers in the Kafka cluster, or -1 to use the broker's default" + ); + public static final ConfigDef.Validator PARTITIONS_VALIDATOR = ConfigDef.LambdaValidator.with( + (name, value) -> validatePartitions(name, (int) value), + () -> "Positive number, or -1 to use the broker's default" + ); + @SuppressWarnings("unchecked") + public static final ConfigDef.Validator REGEX_VALIDATOR = ConfigDef.LambdaValidator.with( + (name, value) -> { + try { + ((List) value).forEach(Pattern::compile); + } catch (PatternSyntaxException e) { + throw new ConfigException(name, value, + "Syntax error in regular expression: " + e.getMessage()); + } + }, + () -> "Positive number, or -1 to use the broker's default" + ); + + private static void validatePartitions(String configName, int factor) { + if (factor != TopicAdmin.NO_PARTITIONS && factor < 1) { + throw new ConfigException(configName, factor, + "Number of partitions must be positive, or -1 to use the broker's default"); + } + } + + private static void validateReplicationFactor(String configName, short factor) { + if (factor != TopicAdmin.NO_REPLICATION_FACTOR && factor < 1) { + throw new ConfigException(configName, factor, + "Replication factor must be positive and not larger than the number of brokers in the Kafka cluster, or -1 to use the broker's default"); + } + } + + public static ConfigDef configDef(String group, short defaultReplicationFactor, int defaultParitionCount) { + int orderInGroup = 0; + ConfigDef configDef = new ConfigDef(); + configDef + .define(INCLUDE_REGEX_CONFIG, ConfigDef.Type.LIST, Collections.emptyList(), + REGEX_VALIDATOR, ConfigDef.Importance.LOW, + INCLUDE_REGEX_DOC, group, ++orderInGroup, ConfigDef.Width.LONG, + "Inclusion Topic Pattern for " + group) + .define(EXCLUDE_REGEX_CONFIG, ConfigDef.Type.LIST, Collections.emptyList(), + REGEX_VALIDATOR, ConfigDef.Importance.LOW, + EXCLUDE_REGEX_DOC, group, ++orderInGroup, ConfigDef.Width.LONG, + "Exclusion Topic Pattern for " + group) + .define(REPLICATION_FACTOR_CONFIG, ConfigDef.Type.SHORT, + defaultReplicationFactor, REPLICATION_FACTOR_VALIDATOR, + ConfigDef.Importance.LOW, REPLICATION_FACTOR_DOC, group, ++orderInGroup, + ConfigDef.Width.LONG, "Replication Factor for Topics in " + group) + .define(PARTITIONS_CONFIG, ConfigDef.Type.INT, + defaultParitionCount, PARTITIONS_VALIDATOR, + ConfigDef.Importance.LOW, PARTITIONS_DOC, group, ++orderInGroup, + ConfigDef.Width.LONG, "Partition Count for Topics in " + group); + return configDef; + } + + public static ConfigDef defaultGroupConfigDef() { + int orderInGroup = 0; + ConfigDef configDef = new ConfigDef(); + configDef + .define(INCLUDE_REGEX_CONFIG, ConfigDef.Type.LIST, ".*", + new ConfigDef.NonNullValidator(), ConfigDef.Importance.LOW, + INCLUDE_REGEX_DOC, DEFAULT_TOPIC_CREATION_GROUP, ++orderInGroup, ConfigDef.Width.LONG, + "Inclusion Topic Pattern for " + DEFAULT_TOPIC_CREATION_GROUP) + .define(EXCLUDE_REGEX_CONFIG, ConfigDef.Type.LIST, Collections.emptyList(), + new ConfigDef.NonNullValidator(), ConfigDef.Importance.LOW, + EXCLUDE_REGEX_DOC, DEFAULT_TOPIC_CREATION_GROUP, ++orderInGroup, ConfigDef.Width.LONG, + "Exclusion Topic Pattern for " + DEFAULT_TOPIC_CREATION_GROUP) + .define(REPLICATION_FACTOR_CONFIG, ConfigDef.Type.SHORT, + ConfigDef.NO_DEFAULT_VALUE, REPLICATION_FACTOR_VALIDATOR, + ConfigDef.Importance.LOW, REPLICATION_FACTOR_DOC, DEFAULT_TOPIC_CREATION_GROUP, ++orderInGroup, + ConfigDef.Width.LONG, "Replication Factor for Topics in " + DEFAULT_TOPIC_CREATION_GROUP) + .define(PARTITIONS_CONFIG, ConfigDef.Type.INT, + ConfigDef.NO_DEFAULT_VALUE, PARTITIONS_VALIDATOR, + ConfigDef.Importance.LOW, PARTITIONS_DOC, DEFAULT_TOPIC_CREATION_GROUP, ++orderInGroup, + ConfigDef.Width.LONG, "Partition Count for Topics in " + DEFAULT_TOPIC_CREATION_GROUP); + return configDef; + } + +} diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/TransformationChain.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/TransformationChain.java index a077a01af668a..5027cb534d79f 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/TransformationChain.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/TransformationChain.java @@ -27,7 +27,7 @@ import java.util.Objects; import java.util.StringJoiner; -public class TransformationChain> { +public class TransformationChain> implements AutoCloseable { private static final Logger log = LoggerFactory.getLogger(TransformationChain.class); private final List> transformations; @@ -55,6 +55,7 @@ record = retryWithToleranceOperator.execute(() -> transformation.apply(current), return record; } + @Override public void close() { for (Transformation transformation : transformations) { transformation.close(); diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java index 98e8e78e2358b..d0a04e8a9c55a 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java @@ -60,6 +60,8 @@ import org.apache.kafka.connect.util.ConnectorTaskId; import org.apache.kafka.connect.util.LoggingContext; import org.apache.kafka.connect.util.SinkUtils; +import org.apache.kafka.connect.util.TopicAdmin; +import org.apache.kafka.connect.util.TopicCreationGroup; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -76,7 +78,6 @@ import java.util.concurrent.Executors; import java.util.stream.Collectors; - /** *

* Worker runs a (dynamic) set of tasks in a set of threads, doing the work of actually moving @@ -249,25 +250,18 @@ public boolean startConnector( final WorkerConnector workerConnector; ClassLoader savedLoader = plugins.currentThreadLoader(); try { - final ConnectorConfig connConfig = new ConnectorConfig(plugins, connProps); - final String connClass = connConfig.getString(ConnectorConfig.CONNECTOR_CLASS_CONFIG); - log.info("Creating connector {} of type {}", connName, connClass); - final Connector connector = plugins.newConnector(connClass); - + // By the time we arrive here, CONNECTOR_CLASS_CONFIG has been validated already + // Getting this value from the unparsed map will allow us to instantiate the + // right config (source or sink) + final String connClassProp = connProps.get(ConnectorConfig.CONNECTOR_CLASS_CONFIG); + log.info("Creating connector {} of type {}", connName, connClassProp); + final Connector connector = plugins.newConnector(connClassProp); final OffsetStorageReader offsetReader = new OffsetStorageReaderImpl( - offsetBackingStore, - connName, - internalKeyConverter, - internalValueConverter - ); - workerConnector = new WorkerConnector( - connName, - connector, - ctx, - metrics, - statusListener, - offsetReader - ); + offsetBackingStore, connName, internalKeyConverter, internalValueConverter); + workerConnector = new WorkerConnector(connName, connector, ctx, metrics, statusListener, offsetReader); + final ConnectorConfig connConfig = workerConnector.isSinkConnector() + ? new SinkConnectorConfig(plugins, connProps) + : new SourceConnectorConfig(plugins, connProps, config.topicCreationEnable()); log.info("Instantiated connector {} with version {} of type {}", connName, connector.version(), connector.getClass()); savedLoader = plugins.compareAndSwapLoaders(connector); workerConnector.initialize(connConfig); @@ -526,21 +520,34 @@ private WorkerTask buildWorkerTask(ClusterConfigState configState, // Decide which type of worker task we need based on the type of task. if (task instanceof SourceTask) { - retryWithToleranceOperator.reporters(sourceTaskReporters(id, connConfig, errorHandlingMetrics)); - TransformationChain transformationChain = new TransformationChain<>(connConfig.transformations(), retryWithToleranceOperator); + SourceConnectorConfig sourceConfig = new SourceConnectorConfig(plugins, + connConfig.originalsStrings(), config.topicCreationEnable()); + retryWithToleranceOperator.reporters(sourceTaskReporters(id, sourceConfig, errorHandlingMetrics)); + TransformationChain transformationChain = new TransformationChain<>(sourceConfig.transformations(), retryWithToleranceOperator); log.info("Initializing: {}", transformationChain); CloseableOffsetStorageReader offsetReader = new OffsetStorageReaderImpl(offsetBackingStore, id.connector(), internalKeyConverter, internalValueConverter); OffsetStorageWriter offsetWriter = new OffsetStorageWriter(offsetBackingStore, id.connector(), internalKeyConverter, internalValueConverter); - Map producerProps = producerConfigs(id, "connector-producer-" + id, config, connConfig, connectorClass, + Map producerProps = producerConfigs(id, "connector-producer-" + id, config, sourceConfig, connectorClass, connectorClientConfigOverridePolicy); KafkaProducer producer = new KafkaProducer<>(producerProps); + TopicAdmin admin; + Map topicCreationGroups; + if (config.topicCreationEnable() && sourceConfig.usesTopicCreation()) { + Map adminProps = adminConfigs(id, "connector-adminclient-" + id, config, + sourceConfig, connectorClass, connectorClientConfigOverridePolicy); + admin = new TopicAdmin(adminProps); + topicCreationGroups = TopicCreationGroup.configuredGroups(sourceConfig); + } else { + admin = null; + topicCreationGroups = null; + } // Note we pass the configState as it performs dynamic transformations under the covers return new WorkerSourceTask(id, (SourceTask) task, statusListener, initialState, keyConverter, valueConverter, - headerConverter, transformationChain, producer, offsetReader, offsetWriter, config, configState, metrics, loader, - time, retryWithToleranceOperator, herder.statusBackingStore()); + headerConverter, transformationChain, producer, admin, topicCreationGroups, + offsetReader, offsetWriter, config, configState, metrics, loader, time, retryWithToleranceOperator, herder.statusBackingStore()); } else if (task instanceof SinkTask) { TransformationChain transformationChain = new TransformationChain<>(connConfig.transformations(), retryWithToleranceOperator); log.info("Initializing: {}", transformationChain); @@ -620,6 +627,7 @@ static Map consumerConfigs(ConnectorTaskId id, } static Map adminConfigs(ConnectorTaskId id, + String defaultClientId, WorkerConfig config, ConnectorConfig connConfig, Class connectorClass, @@ -637,6 +645,7 @@ static Map adminConfigs(ConnectorTaskId id, .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); adminProps.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, Utils.join(config.getList(WorkerConfig.BOOTSTRAP_SERVERS_CONFIG), ",")); + adminProps.put(AdminClientConfig.CLIENT_ID_CONFIG, defaultClientId); adminProps.putAll(nonPrefixedWorkerConfigs); // Admin client-specific overrides in the worker config @@ -693,7 +702,7 @@ private List sinkTaskReporters(ConnectorTaskId id, SinkConnectorC if (topic != null && !topic.isEmpty()) { Map producerProps = producerConfigs(id, "connector-dlq-producer-" + id, config, connConfig, connectorClass, connectorClientConfigOverridePolicy); - Map adminProps = adminConfigs(id, config, connConfig, connectorClass, connectorClientConfigOverridePolicy); + Map adminProps = adminConfigs(id, "connector-dlq-adminclient-", config, connConfig, connectorClass, connectorClientConfigOverridePolicy); DeadLetterQueueReporter reporter = DeadLetterQueueReporter.createAndSetup(adminProps, id, connConfig, producerProps, errorHandlingMetrics); reporters.add(reporter); } @@ -817,6 +826,16 @@ public String workerId() { return workerId; } + /** + * Returns whether this worker is configured to allow source connectors to create the topics + * that they use with custom configurations, if these topics don't already exist. + * + * @return true if topic creation by source connectors is allowed; false otherwise + */ + public boolean isTopicCreationEnabled() { + return config.topicCreationEnable(); + } + /** * Get the {@link ConnectMetrics} that uses Kafka Metrics and manages the JMX reporter. * @return the Connect-specific metrics; never null diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConfig.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConfig.java index 7a4f04ec7cd98..02349265b1576 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConfig.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConfig.java @@ -44,6 +44,7 @@ import static org.apache.kafka.common.config.ConfigDef.Range.atLeast; import static org.apache.kafka.common.config.ConfigDef.ValidString.in; +import static org.apache.kafka.connect.runtime.SourceConnectorConfig.TOPIC_CREATION_PREFIX; /** * Common base class providing configuration for Kafka Connect workers, whether standalone or distributed. @@ -250,9 +251,17 @@ public class WorkerConfig extends AbstractConfig { + "user requests to reset the set of active topics per connector."; protected static final boolean TOPIC_TRACKING_ALLOW_RESET_DEFAULT = true; + public static final String TOPIC_CREATION_ENABLE_CONFIG = "topic.creation.enable"; + protected static final String TOPIC_CREATION_ENABLE_DOC = "Whether to allow " + + "automatic creation of topics used by source connectors, when source connectors " + + "are configured with `" + TOPIC_CREATION_PREFIX + "` properties. Each task will use an " + + "admin client to create its topics and will not depend on the Kafka brokers " + + "to create topics automatically."; + protected static final boolean TOPIC_CREATION_ENABLE_DEFAULT = true; + public static final String RESPONSE_HTTP_HEADERS_CONFIG = "response.http.headers.config"; - public static final String RESPONSE_HTTP_HEADERS_DOC = "Rules for REST API HTTP response headers"; - public static final String RESPONSE_HTTP_HEADERS_DEFAULT = ""; + protected static final String RESPONSE_HTTP_HEADERS_DOC = "Rules for REST API HTTP response headers"; + protected static final String RESPONSE_HTTP_HEADERS_DEFAULT = ""; /** * Get a basic ConfigDef for a WorkerConfig. This includes all the common settings. Subclasses can use this to @@ -335,6 +344,8 @@ protected static ConfigDef baseConfigDef() { Importance.LOW, TOPIC_TRACKING_ENABLE_DOC) .define(TOPIC_TRACKING_ALLOW_RESET_CONFIG, Type.BOOLEAN, TOPIC_TRACKING_ALLOW_RESET_DEFAULT, Importance.LOW, TOPIC_TRACKING_ALLOW_RESET_DOC) + .define(TOPIC_CREATION_ENABLE_CONFIG, Type.BOOLEAN, TOPIC_CREATION_ENABLE_DEFAULT, Importance.LOW, + TOPIC_CREATION_ENABLE_DOC) .define(RESPONSE_HTTP_HEADERS_CONFIG, Type.STRING, RESPONSE_HTTP_HEADERS_DEFAULT, new ResponseHttpHeadersValidator(), Importance.LOW, RESPONSE_HTTP_HEADERS_DOC); } @@ -395,6 +406,10 @@ public Integer getRebalanceTimeout() { return null; } + public boolean topicCreationEnable() { + return getBoolean(TOPIC_CREATION_ENABLE_CONFIG); + } + @Override protected Map postProcessParsedConfig(final Map parsedValues) { return CommonClientConfigs.postProcessReconnectBackoffConfigs(this, parsedValues); diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSinkTask.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSinkTask.java index 3a8c8d484021c..3df94511d9c70 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSinkTask.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSinkTask.java @@ -165,18 +165,8 @@ protected void close() { } catch (Throwable t) { log.warn("Could not stop task", t); } - if (consumer != null) { - try { - consumer.close(); - } catch (Throwable t) { - log.warn("Could not close consumer", t); - } - } - try { - transformationChain.close(); - } catch (Throwable t) { - log.warn("Could not close transformation chain", t); - } + Utils.closeQuietly(consumer, "consumer"); + Utils.closeQuietly(transformationChain, "transformation chain"); Utils.closeQuietly(retryWithToleranceOperator, "retry operator"); } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTask.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTask.java index dc7474766a034..e44d93a718bf0 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTask.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTask.java @@ -16,7 +16,8 @@ */ package org.apache.kafka.connect.runtime; -import org.apache.kafka.clients.producer.Callback; +import org.apache.kafka.clients.admin.NewTopic; +import org.apache.kafka.clients.admin.TopicDescription; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.clients.producer.RecordMetadata; @@ -47,6 +48,9 @@ import org.apache.kafka.connect.storage.StatusBackingStore; import org.apache.kafka.connect.util.ConnectUtils; import org.apache.kafka.connect.util.ConnectorTaskId; +import org.apache.kafka.connect.util.TopicAdmin; +import org.apache.kafka.connect.util.TopicCreation; +import org.apache.kafka.connect.util.TopicCreationGroup; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -78,12 +82,14 @@ class WorkerSourceTask extends WorkerTask { private final Converter valueConverter; private final HeaderConverter headerConverter; private final TransformationChain transformationChain; - private KafkaProducer producer; + private final KafkaProducer producer; + private final TopicAdmin admin; private final CloseableOffsetStorageReader offsetReader; private final OffsetStorageWriter offsetWriter; private final SourceTaskMetricsGroup sourceTaskMetricsGroup; private final AtomicReference producerSendException; private final boolean isTopicTrackingEnabled; + private final TopicCreation topicCreation; private List toSend; private boolean lastSendFailed; // Whether the last send failed *synchronously*, i.e. never made it into the producer's RecordAccumulator @@ -109,6 +115,8 @@ public WorkerSourceTask(ConnectorTaskId id, HeaderConverter headerConverter, TransformationChain transformationChain, KafkaProducer producer, + TopicAdmin admin, + Map topicGroups, CloseableOffsetStorageReader offsetReader, OffsetStorageWriter offsetWriter, WorkerConfig workerConfig, @@ -130,6 +138,7 @@ public WorkerSourceTask(ConnectorTaskId id, this.headerConverter = headerConverter; this.transformationChain = transformationChain; this.producer = producer; + this.admin = admin; this.offsetReader = offsetReader; this.offsetWriter = offsetWriter; @@ -142,6 +151,7 @@ public WorkerSourceTask(ConnectorTaskId id, this.sourceTaskMetricsGroup = new SourceTaskMetricsGroup(id, connectMetrics); this.producerSendException = new AtomicReference<>(); this.isTopicTrackingEnabled = workerConfig.getBoolean(TOPIC_TRACKING_ENABLE_CONFIG); + this.topicCreation = TopicCreation.newTopicCreation(workerConfig, topicGroups); } @Override @@ -166,11 +176,14 @@ protected void close() { log.warn("Could not close producer", t); } } - try { - transformationChain.close(); - } catch (Throwable t) { - log.warn("Could not close transformation chain", t); + if (admin != null) { + try { + admin.close(Duration.ofSeconds(30)); + } catch (Throwable t) { + log.warn("Failed to close admin client on time", t); + } } + Utils.closeQuietly(transformationChain, "transformation chain"); Utils.closeQuietly(retryWithToleranceOperator, "retry operator"); } @@ -344,37 +357,41 @@ private boolean sendRecords() { } } try { + maybeCreateTopic(record.topic()); final String topic = producerRecord.topic(); producer.send( - producerRecord, - new Callback() { - @Override - public void onCompletion(RecordMetadata recordMetadata, Exception e) { - if (e != null) { - log.error("{} failed to send record to {}:", WorkerSourceTask.this, topic, e); - log.debug("{} Failed record: {}", WorkerSourceTask.this, preTransformRecord); - producerSendException.compareAndSet(null, e); - } else { - recordSent(producerRecord); - counter.completeRecord(); - log.trace("{} Wrote record successfully: topic {} partition {} offset {}", - WorkerSourceTask.this, - recordMetadata.topic(), recordMetadata.partition(), - recordMetadata.offset()); - commitTaskRecord(preTransformRecord, recordMetadata); - if (isTopicTrackingEnabled) { - recordActiveTopic(producerRecord.topic()); - } - } + producerRecord, + (recordMetadata, e) -> { + if (e != null) { + log.error("{} failed to send record to {}: ", WorkerSourceTask.this, topic, e); + log.debug("{} Failed record: {}", WorkerSourceTask.this, preTransformRecord); + producerSendException.compareAndSet(null, e); + } else { + recordSent(producerRecord); + counter.completeRecord(); + log.trace("{} Wrote record successfully: topic {} partition {} offset {}", + WorkerSourceTask.this, + recordMetadata.topic(), recordMetadata.partition(), + recordMetadata.offset()); + commitTaskRecord(preTransformRecord, recordMetadata); + if (isTopicTrackingEnabled) { + recordActiveTopic(producerRecord.topic()); } - }); + } + }); lastSendFailed = false; - } catch (org.apache.kafka.common.errors.RetriableException e) { - log.warn("{} Failed to send {}, backing off before retrying:", this, producerRecord, e); + } catch (RetriableException | org.apache.kafka.common.errors.RetriableException e) { + log.warn("{} Failed to send record to topic '{}' and partition '{}'. Backing off before retrying: ", + this, producerRecord.topic(), producerRecord.partition(), e); toSend = toSend.subList(processed, toSend.size()); lastSendFailed = true; counter.retryRemaining(); return false; + } catch (ConnectException e) { + log.warn("{} Failed to send record to topic '{}' and partition '{}' due to an unrecoverable exception: ", + this, producerRecord.topic(), producerRecord.partition(), e); + log.warn("{} Failed to send {} with unrecoverable exception: ", this, producerRecord, e); + throw e; } catch (KafkaException e) { throw new ConnectException("Unrecoverable exception trying to send", e); } @@ -384,6 +401,37 @@ public void onCompletion(RecordMetadata recordMetadata, Exception e) { return true; } + // Due to transformations that may change the destination topic of a record (such as + // RegexRouter) topic creation can not be batched for multiple topics + private void maybeCreateTopic(String topic) { + if (!topicCreation.isTopicCreationRequired(topic)) { + return; + } + log.info("The task will send records to topic '{}' for the first time. Checking " + + "whether topic exists", topic); + Map existing = admin.describeTopics(topic); + if (!existing.isEmpty()) { + log.info("Topic '{}' already exists.", topic); + topicCreation.addTopic(topic); + return; + } + + log.info("Creating topic '{}'", topic); + TopicCreationGroup topicGroup = topicCreation.findFirstGroup(topic); + log.debug("Topic '{}' matched topic creation group: {}", topic, topicGroup); + NewTopic newTopic = topicGroup.newTopic(topic); + + if (admin.createTopic(newTopic)) { + topicCreation.addTopic(topic); + log.info("Created topic '{}' using creation group {}", newTopic, topicGroup); + } else { + log.warn("Request to create new topic '{}' failed", topic); + throw new ConnectException("Task failed to create new topic " + topic + ". Ensure " + + "that the task is authorized to create topics or that the topic exists and " + + "restart the task"); + } + } + private RecordHeaders convertHeaderFor(SourceRecord record) { Headers headers = record.headers(); RecordHeaders result = new RecordHeaders(); @@ -479,14 +527,11 @@ public boolean commitOffsets() { } // Now we can actually flush the offsets to user storage. - Future flushFuture = offsetWriter.doFlush(new org.apache.kafka.connect.util.Callback() { - @Override - public void onCompletion(Throwable error, Void result) { - if (error != null) { - log.error("{} Failed to flush offsets to storage: ", WorkerSourceTask.this, error); - } else { - log.trace("{} Finished flushing offsets to storage", WorkerSourceTask.this); - } + Future flushFuture = offsetWriter.doFlush((error, result) -> { + if (error != null) { + log.error("{} Failed to flush offsets to storage: ", WorkerSourceTask.this, error); + } else { + log.trace("{} Finished flushing offsets to storage", WorkerSourceTask.this); } }); // Very rare case: offsets were unserializable and we finished immediately, unable to store diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedConfig.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedConfig.java index c7fc1cff128a6..9af6291e3df83 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedConfig.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedConfig.java @@ -18,13 +18,10 @@ import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.common.config.ConfigDef; -import org.apache.kafka.common.config.ConfigDef.Validator; -import org.apache.kafka.common.config.ConfigDef.LambdaValidator; import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.common.config.TopicConfig; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.connect.runtime.WorkerConfig; -import org.apache.kafka.connect.util.TopicAdmin; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -40,6 +37,8 @@ import static org.apache.kafka.common.config.ConfigDef.Range.atLeast; import static org.apache.kafka.common.config.ConfigDef.Range.between; +import static org.apache.kafka.connect.runtime.TopicCreationConfig.PARTITIONS_VALIDATOR; +import static org.apache.kafka.connect.runtime.TopicCreationConfig.REPLICATION_FACTOR_VALIDATOR; public class DistributedConfig extends WorkerConfig { @@ -193,15 +192,6 @@ public class DistributedConfig extends WorkerConfig { public static final String INTER_WORKER_VERIFICATION_ALGORITHMS_DOC = "A list of permitted algorithms for verifying internal requests"; public static final List INTER_WORKER_VERIFICATION_ALGORITHMS_DEFAULT = Collections.singletonList(INTER_WORKER_SIGNATURE_ALGORITHM_DEFAULT); - private static final Validator REPLICATION_FACTOR_VALIDATOR = LambdaValidator.with( - (name, value) -> validateReplicationFactor(name, (short) value), - () -> "Positive number, or -1 to use the broker's default" - ); - private static final Validator PARTITIONS_VALIDATOR = LambdaValidator.with( - (name, value) -> validatePartitions(name, (int) value), - () -> "Positive number, or -1 to use the broker's default" - ); - @SuppressWarnings("unchecked") private static final ConfigDef CONFIG = baseConfigDef() .define(GROUP_ID_CONFIG, @@ -490,18 +480,4 @@ private static void validateKeyAlgorithm(String configName, String algorithm) { throw new ConfigException(configName, algorithm, e.getMessage()); } } - - private static void validatePartitions(String configName, int factor) { - if (factor != TopicAdmin.NO_PARTITIONS && factor < 1) { - throw new ConfigException(configName, factor, - "Number of partitions must be positive, or -1 to use the broker's default"); - } - } - - private static void validateReplicationFactor(String configName, short factor) { - if (factor != TopicAdmin.NO_REPLICATION_FACTOR && factor < 1) { - throw new ConfigException(configName, factor, - "Replication factor must be positive, or -1 to use the broker's default"); - } - } } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java index 349aa71e54e0c..851b75ff51f2a 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java @@ -1289,7 +1289,7 @@ private void reconfigureConnector(final String connName, final Callback cb if (worker.isSinkConnector(connName)) { connConfig = new SinkConnectorConfig(plugins(), configs); } else { - connConfig = new SourceConnectorConfig(plugins(), configs); + connConfig = new SourceConnectorConfig(plugins(), configs, worker.isTopicCreationEnabled()); } final List> taskProps = worker.connectorTaskConfigs(connName, connConfig); diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerder.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerder.java index 6c5398f8e4573..ade3f9e2feeb2 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerder.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerder.java @@ -300,7 +300,7 @@ private List> recomputeTaskConfigs(String connName) { ConnectorConfig connConfig = worker.isSinkConnector(connName) ? new SinkConnectorConfig(plugins(), config) : - new SourceConnectorConfig(plugins(), config); + new SourceConnectorConfig(plugins(), config, worker.isTopicCreationEnabled()); return worker.connectorTaskConfigs(connName, connConfig); } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/util/TopicAdmin.java b/connect/runtime/src/main/java/org/apache/kafka/connect/util/TopicAdmin.java index 61b4a027c7b8a..4809b490b05c7 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/util/TopicAdmin.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/util/TopicAdmin.java @@ -19,20 +19,26 @@ import org.apache.kafka.clients.admin.Admin; import org.apache.kafka.clients.admin.AdminClientConfig; import org.apache.kafka.clients.admin.CreateTopicsOptions; +import org.apache.kafka.clients.admin.DescribeTopicsOptions; import org.apache.kafka.clients.admin.NewTopic; +import org.apache.kafka.clients.admin.TopicDescription; import org.apache.kafka.common.KafkaFuture; import org.apache.kafka.common.config.TopicConfig; import org.apache.kafka.common.errors.ClusterAuthorizationException; import org.apache.kafka.common.errors.InvalidConfigurationException; -import org.apache.kafka.common.errors.TopicAuthorizationException; import org.apache.kafka.common.errors.TimeoutException; +import org.apache.kafka.common.errors.TopicAuthorizationException; import org.apache.kafka.common.errors.TopicExistsException; +import org.apache.kafka.common.errors.UnknownTopicOrPartitionException; import org.apache.kafka.common.errors.UnsupportedVersionException; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.connect.errors.ConnectException; +import org.apache.kafka.connect.errors.RetriableException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.time.Duration; +import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -58,10 +64,10 @@ public class TopicAdmin implements AutoCloseable { * A builder of {@link NewTopic} instances. */ public static class NewTopicBuilder { - private String name; + private final String name; private int numPartitions = NO_PARTITIONS; private short replicationFactor = NO_REPLICATION_FACTOR; - private Map configs = new HashMap<>(); + private final Map configs = new HashMap<>(); NewTopicBuilder(String name) { this.name = name; @@ -313,11 +319,71 @@ public Set createTopics(NewTopic... topics) { return newlyCreatedTopicNames; } + /** + * Attempt to fetch the descriptions of the given topics + * Apache Kafka added support for describing topics in 0.10.0.0, so this method works as expected with that and later versions. + * With brokers older than 0.10.0.0, this method is unable to describe topics and always returns an empty set. + * + * @param topics the topics to describe + * @return a map of topic names to topic descriptions of the topics that were requested; never null but possibly empty + * @throws RetriableException if a retriable error occurs, the operation takes too long, or the + * thread is interrupted while attempting to perform this operation + * @throws ConnectException if a non retriable error occurs + */ + public Map describeTopics(String... topics) { + if (topics == null) { + return Collections.emptyMap(); + } + String bootstrapServers = bootstrapServers(); + String topicNameList = String.join(", ", topics); + + Map> newResults = + admin.describeTopics(Arrays.asList(topics), new DescribeTopicsOptions()).values(); + + // Iterate over each future so that we can handle individual failures like when some topics don't exist + Map existingTopics = new HashMap<>(); + newResults.forEach((topic, desc) -> { + try { + existingTopics.put(topic, desc.get()); + } catch (ExecutionException e) { + Throwable cause = e.getCause(); + if (cause instanceof UnknownTopicOrPartitionException) { + log.debug("Topic '{}' does not exist on the brokers at {}", topic, bootstrapServers); + return; + } + if (cause instanceof ClusterAuthorizationException || cause instanceof TopicAuthorizationException) { + String msg = String.format("Not authorized to describe topic(s) '%s' on the brokers %s", + topicNameList, bootstrapServers); + throw new ConnectException(msg, cause); + } + if (cause instanceof UnsupportedVersionException) { + String msg = String.format("Unable to describe topic(s) '%s' since the brokers " + + "at %s do not support the DescribeTopics API.", + topicNameList, bootstrapServers); + throw new ConnectException(msg, cause); + } + if (cause instanceof TimeoutException) { + // Timed out waiting for the operation to complete + throw new RetriableException("Timed out while describing topics '" + topicNameList + "'", cause); + } + throw new ConnectException("Error while attempting to describe topics '" + topicNameList + "'", e); + } catch (InterruptedException e) { + Thread.interrupted(); + throw new RetriableException("Interrupted while attempting to describe topics '" + topicNameList + "'", e); + } + }); + return existingTopics; + } + @Override public void close() { admin.close(); } + public void close(Duration timeout) { + admin.close(timeout); + } + private String bootstrapServers() { Object servers = adminConfig.get(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG); return servers != null ? servers.toString() : ""; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/util/TopicCreation.java b/connect/runtime/src/main/java/org/apache/kafka/connect/util/TopicCreation.java new file mode 100644 index 0000000000000..f914ed92dc283 --- /dev/null +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/util/TopicCreation.java @@ -0,0 +1,148 @@ +/* + * 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.util; + +import org.apache.kafka.connect.runtime.WorkerConfig; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Collections; +import java.util.HashSet; +import java.util.LinkedHashMap; +import java.util.Map; +import java.util.Set; + +import static org.apache.kafka.connect.runtime.TopicCreationConfig.DEFAULT_TOPIC_CREATION_GROUP; + +/** + * Utility to be used by worker source tasks in order to create topics, if topic creation is + * enabled for source connectors at the worker and the connector configurations. + */ +public class TopicCreation { + private static final Logger log = LoggerFactory.getLogger(TopicCreation.class); + private static final TopicCreation EMPTY = + new TopicCreation(false, null, Collections.emptyMap(), Collections.emptySet()); + + private final boolean isTopicCreationEnabled; + private final TopicCreationGroup defaultTopicGroup; + private final Map topicGroups; + private final Set topicCache; + + protected TopicCreation(boolean isTopicCreationEnabled, + TopicCreationGroup defaultTopicGroup, + Map topicGroups, + Set topicCache) { + this.isTopicCreationEnabled = isTopicCreationEnabled; + this.defaultTopicGroup = defaultTopicGroup; + this.topicGroups = topicGroups; + this.topicCache = topicCache; + } + + public static TopicCreation newTopicCreation(WorkerConfig workerConfig, + Map topicGroups) { + if (!workerConfig.topicCreationEnable() || topicGroups == null) { + return EMPTY; + } + Map groups = new LinkedHashMap<>(topicGroups); + groups.remove(DEFAULT_TOPIC_CREATION_GROUP); + return new TopicCreation(true, topicGroups.get(DEFAULT_TOPIC_CREATION_GROUP), groups, new HashSet<>()); + } + + /** + * Return an instance of this utility that represents what the state of the internal data + * structures should be when topic creation is disabled. + * + * @return the utility when topic creation is disabled + */ + public static TopicCreation empty() { + return EMPTY; + } + + /** + * Check whether topic creation is enabled for this utility instance. This is state is set at + * instantiation time and remains unchanged for the lifetime of every {@link TopicCreation} + * object. + * + * @return true if topic creation is enabled; false otherwise + */ + public boolean isTopicCreationEnabled() { + return isTopicCreationEnabled; + } + + /** + * Check whether topic creation may be required for a specific topic name. + * + * @return true if topic creation is enabled and the topic name is not in the topic cache; + * false otherwise + */ + public boolean isTopicCreationRequired(String topic) { + return isTopicCreationEnabled && !topicCache.contains(topic); + } + + /** + * Return the default topic creation group. This group is always defined when topic creation is + * enabled but is {@code null} if topic creation is disabled. + * + * @return the default topic creation group if topic creation is enabled; {@code null} otherwise + */ + public TopicCreationGroup defaultTopicGroup() { + return defaultTopicGroup; + } + + /** + * Return the topic creation groups defined for a source connector as a map of topic creation + * group name to topic creation group instance. This map maintains all the optionally defined + * groups besides the default group which is defined for any connector when topic creation is + * enabled. + * + * @return the map of all the topic creation groups besides the default group; may be empty + * but not {@code null} + */ + public Map topicGroups() { + return topicGroups; + } + + /** + * Inform this utility instance that a topic has been created and its creation will no + * longer be required. After {@link #addTopic(String)} is called for a give {@param topic} + * any subsequent calls to {@link #isTopicCreationRequired} will return {@code false} for the + * same topic. + * + * @param topic the topic name to mark as created + */ + public void addTopic(String topic) { + if (isTopicCreationEnabled) { + topicCache.add(topic); + } + } + + /** + * Get the first topic creation group that is configured to match the given {@param topic} + * name. If topic creation is enabled, any topic should match at least the default topic + * creation group. + * + * @param topic the topic name to match against group configurations + * + * @return the first group that matches the given topic + */ + public TopicCreationGroup findFirstGroup(String topic) { + return topicGroups.values().stream() + .filter(group -> group.matches(topic)) + .findFirst() + .orElse(defaultTopicGroup); + } +} diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/util/TopicCreationGroup.java b/connect/runtime/src/main/java/org/apache/kafka/connect/util/TopicCreationGroup.java new file mode 100644 index 0000000000000..11970339ab845 --- /dev/null +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/util/TopicCreationGroup.java @@ -0,0 +1,151 @@ +/* + * 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.util; + +import org.apache.kafka.clients.admin.Admin; +import org.apache.kafka.clients.admin.NewTopic; +import org.apache.kafka.connect.runtime.SourceConnectorConfig; + +import java.util.Collections; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.regex.Pattern; + +import static org.apache.kafka.connect.runtime.SourceConnectorConfig.TOPIC_CREATION_GROUPS_CONFIG; +import static org.apache.kafka.connect.runtime.TopicCreationConfig.DEFAULT_TOPIC_CREATION_GROUP; + +/** + * Utility to simplify creating and managing topics via the {@link Admin}. + */ +public class TopicCreationGroup { + private final String name; + private final Pattern inclusionPattern; + private final Pattern exclusionPattern; + private final int numPartitions; + private final short replicationFactor; + private final Map otherConfigs; + + protected TopicCreationGroup(String group, SourceConnectorConfig config) { + this.name = group; + this.inclusionPattern = Pattern.compile(String.join("|", config.topicCreationInclude(group))); + this.exclusionPattern = Pattern.compile(String.join("|", config.topicCreationExclude(group))); + this.numPartitions = config.topicCreationPartitions(group); + this.replicationFactor = config.topicCreationReplicationFactor(group); + this.otherConfigs = config.topicCreationOtherConfigs(group); + } + + /** + * Parses the configuration of a source connector and returns the topic creation groups + * defined in the given configuration as a map of group names to {@link TopicCreation} objects. + * + * @param config the source connector configuration + * + * @return the map of topic creation groups; may be empty but not {@code null} + */ + public static Map configuredGroups(SourceConnectorConfig config) { + if (!config.usesTopicCreation()) { + return Collections.emptyMap(); + } + List groupNames = config.getList(TOPIC_CREATION_GROUPS_CONFIG); + Map groups = new LinkedHashMap<>(); + for (String group : groupNames) { + groups.put(group, new TopicCreationGroup(group, config)); + } + // Even if there was a group called 'default' in the config, it will be overridden here. + // Order matters for all the topic groups besides the default, since it will be + // removed from this collection by the Worker + groups.put(DEFAULT_TOPIC_CREATION_GROUP, new TopicCreationGroup(DEFAULT_TOPIC_CREATION_GROUP, config)); + return groups; + } + + /** + * Return the name of the topic creation group. + * + * @return the name of the topic creation group + */ + public String name() { + return name; + } + + /** + * Answer whether this topic creation group is configured to allow the creation of the given + * {@param topic} name. + * + * @param topic the topic name to check against the groups configuration + * + * @return true if the topic name matches the inclusion regex and does + * not match the exclusion regex of this group's configuration; false otherwise + */ + public boolean matches(String topic) { + return !exclusionPattern.matcher(topic).matches() && inclusionPattern.matcher(topic) + .matches(); + } + + /** + * Return the description for a new topic with the given {@param topic} name with the topic + * settings defined for this topic creation group. + * + * @param topic the name of the topic to be created + * + * @return the topic description of the given topic with settings of this topic creation group + */ + public NewTopic newTopic(String topic) { + TopicAdmin.NewTopicBuilder builder = new TopicAdmin.NewTopicBuilder(topic); + return builder.partitions(numPartitions) + .replicationFactor(replicationFactor) + .config(otherConfigs) + .build(); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof TopicCreationGroup)) { + return false; + } + TopicCreationGroup that = (TopicCreationGroup) o; + return Objects.equals(name, that.name) + && numPartitions == that.numPartitions + && replicationFactor == that.replicationFactor + && Objects.equals(inclusionPattern.pattern(), that.inclusionPattern.pattern()) + && Objects.equals(exclusionPattern.pattern(), that.exclusionPattern.pattern()) + && Objects.equals(otherConfigs, that.otherConfigs); + } + + @Override + public int hashCode() { + return Objects.hash(name, numPartitions, replicationFactor, inclusionPattern.pattern(), + exclusionPattern.pattern(), otherConfigs + ); + } + + @Override + public String toString() { + return "TopicCreationGroup{" + + "name='" + name + '\'' + + ", inclusionPattern=" + inclusionPattern + + ", exclusionPattern=" + exclusionPattern + + ", numPartitions=" + numPartitions + + ", replicationFactor=" + replicationFactor + + ", otherConfigs=" + otherConfigs + + '}'; + } +} diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectWorkerIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectWorkerIntegrationTest.java index b3f3020205528..076e462394a3d 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectWorkerIntegrationTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectWorkerIntegrationTest.java @@ -37,11 +37,15 @@ import java.util.concurrent.TimeUnit; import static org.apache.kafka.clients.CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG; +import static org.apache.kafka.connect.integration.MonitorableSourceConnector.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.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.TopicCreationConfig.DEFAULT_TOPIC_CREATION_PREFIX; +import static org.apache.kafka.connect.runtime.TopicCreationConfig.PARTITIONS_CONFIG; +import static org.apache.kafka.connect.runtime.TopicCreationConfig.REPLICATION_FACTOR_CONFIG; import static org.apache.kafka.connect.runtime.WorkerConfig.CONNECTOR_CLIENT_POLICY_CLASS_CONFIG; import static org.apache.kafka.connect.runtime.WorkerConfig.OFFSET_COMMIT_INTERVAL_MS_CONFIG; import static org.junit.Assert.assertFalse; @@ -57,7 +61,9 @@ public class ConnectWorkerIntegrationTest { private static final int NUM_TOPIC_PARTITIONS = 3; private static final long OFFSET_COMMIT_INTERVAL_MS = TimeUnit.SECONDS.toMillis(30); private static final int NUM_WORKERS = 3; + private static final int NUM_TASKS = 4; private static final String CONNECTOR_NAME = "simple-source"; + private static final String TOPIC_NAME = "test-topic"; private EmbeddedConnectCluster.Builder connectBuilder; private EmbeddedConnectCluster connect; @@ -98,18 +104,11 @@ public void testAddAndRemoveWorker() throws Exception { // start the clusters connect.start(); - int numTasks = 4; // create test topic - connect.kafka().createTopic("test-topic", NUM_TOPIC_PARTITIONS); + connect.kafka().createTopic(TOPIC_NAME, NUM_TOPIC_PARTITIONS); // set up props for the source connector - Map props = new HashMap<>(); - props.put(CONNECTOR_CLASS_CONFIG, MonitorableSourceConnector.class.getSimpleName()); - props.put(TASKS_MAX_CONFIG, String.valueOf(numTasks)); - props.put("throughput", String.valueOf(1)); - props.put("messages.per.poll", String.valueOf(10)); - props.put(KEY_CONVERTER_CLASS_CONFIG, StringConverter.class.getName()); - props.put(VALUE_CONVERTER_CLASS_CONFIG, StringConverter.class.getName()); + Map props = defaultSourceConnectorProps(TOPIC_NAME); connect.assertions().assertAtLeastNumWorkersAreUp(NUM_WORKERS, "Initial group of workers did not start in time."); @@ -117,7 +116,7 @@ public void testAddAndRemoveWorker() throws Exception { // start a source connector connect.configureConnector(CONNECTOR_NAME, props); - connect.assertions().assertConnectorAndAtLeastNumTasksAreRunning(CONNECTOR_NAME, numTasks, + connect.assertions().assertConnectorAndAtLeastNumTasksAreRunning(CONNECTOR_NAME, NUM_TASKS, "Connector tasks did not start in time."); WorkerHandle extraWorker = connect.addWorker(); @@ -125,7 +124,7 @@ public void testAddAndRemoveWorker() throws Exception { connect.assertions().assertAtLeastNumWorkersAreUp(NUM_WORKERS + 1, "Expanded group of workers did not start in time."); - connect.assertions().assertConnectorAndAtLeastNumTasksAreRunning(CONNECTOR_NAME, numTasks, + connect.assertions().assertConnectorAndAtLeastNumTasksAreRunning(CONNECTOR_NAME, NUM_TASKS, "Connector tasks are not all in running state."); Set workers = connect.activeWorkers(); @@ -151,24 +150,24 @@ public void testRestartFailedTask() throws Exception { int numTasks = 1; + // setup up props for the source connector + Map props = defaultSourceConnectorProps(TOPIC_NAME); // Properties for the source connector. The task should fail at startup due to the bad broker address. - Map connectorProps = new HashMap<>(); - connectorProps.put(CONNECTOR_CLASS_CONFIG, MonitorableSourceConnector.class.getName()); - connectorProps.put(TASKS_MAX_CONFIG, Objects.toString(numTasks)); - connectorProps.put(CONNECTOR_CLIENT_PRODUCER_OVERRIDES_PREFIX + BOOTSTRAP_SERVERS_CONFIG, "nobrokerrunningatthisaddress"); + props.put(TASKS_MAX_CONFIG, Objects.toString(numTasks)); + props.put(CONNECTOR_CLIENT_PRODUCER_OVERRIDES_PREFIX + BOOTSTRAP_SERVERS_CONFIG, "nobrokerrunningatthisaddress"); connect.assertions().assertExactlyNumWorkersAreUp(NUM_WORKERS, "Initial group of workers did not start in time."); // Try to start the connector and its single task. - connect.configureConnector(CONNECTOR_NAME, connectorProps); + connect.configureConnector(CONNECTOR_NAME, props); connect.assertions().assertConnectorIsRunningAndTasksHaveFailed(CONNECTOR_NAME, numTasks, "Connector tasks did not fail in time"); // Reconfigure the connector without the bad broker address. - connectorProps.remove(CONNECTOR_CLIENT_PRODUCER_OVERRIDES_PREFIX + BOOTSTRAP_SERVERS_CONFIG); - connect.configureConnector(CONNECTOR_NAME, connectorProps); + props.remove(CONNECTOR_CLIENT_PRODUCER_OVERRIDES_PREFIX + BOOTSTRAP_SERVERS_CONFIG); + connect.configureConnector(CONNECTOR_NAME, props); // Restart the failed task String taskRestartEndpoint = connect.endpointForResource( @@ -191,17 +190,10 @@ public void testBrokerCoordinator() throws Exception { connect.start(); int numTasks = 4; // create test topic - connect.kafka().createTopic("test-topic", NUM_TOPIC_PARTITIONS); + connect.kafka().createTopic(TOPIC_NAME, NUM_TOPIC_PARTITIONS); // set up props for the source connector - Map props = new HashMap<>(); - props.put(CONNECTOR_CLASS_CONFIG, MonitorableSourceConnector.class.getSimpleName()); - props.put(TASKS_MAX_CONFIG, String.valueOf(numTasks)); - props.put("topic", "test-topic"); - props.put("throughput", String.valueOf(1)); - props.put("messages.per.poll", String.valueOf(10)); - props.put(KEY_CONVERTER_CLASS_CONFIG, StringConverter.class.getName()); - props.put(VALUE_CONVERTER_CLASS_CONFIG, StringConverter.class.getName()); + Map props = defaultSourceConnectorProps(TOPIC_NAME); connect.assertions().assertAtLeastNumWorkersAreUp(NUM_WORKERS, "Initial group of workers did not start in time."); @@ -247,28 +239,46 @@ public void testTaskStatuses() throws Exception { connect.start(); connect.assertions().assertAtLeastNumWorkersAreUp(NUM_WORKERS, - "Initial group of workers did not start in time."); + "Initial group of workers did not start in time."); // base connector props - Map connectorProps = new HashMap<>(); - connectorProps.put(CONNECTOR_CLASS_CONFIG, MonitorableSourceConnector.class.getSimpleName()); + Map props = defaultSourceConnectorProps(TOPIC_NAME); + props.put(CONNECTOR_CLASS_CONFIG, MonitorableSourceConnector.class.getSimpleName()); // start the connector with only one task - final int initialNumTasks = 1; - connectorProps.put(TASKS_MAX_CONFIG, String.valueOf(initialNumTasks)); - connect.configureConnector(CONNECTOR_NAME, connectorProps); - connect.assertions().assertConnectorAndExactlyNumTasksAreRunning(CONNECTOR_NAME, initialNumTasks, "Connector tasks did not start in time"); + int initialNumTasks = 1; + props.put(TASKS_MAX_CONFIG, String.valueOf(initialNumTasks)); + connect.configureConnector(CONNECTOR_NAME, props); + connect.assertions().assertConnectorAndExactlyNumTasksAreRunning(CONNECTOR_NAME, + initialNumTasks, "Connector tasks did not start in time"); // then reconfigure it to use more tasks - final int increasedNumTasks = 5; - connectorProps.put(TASKS_MAX_CONFIG, String.valueOf(increasedNumTasks)); - connect.configureConnector(CONNECTOR_NAME, connectorProps); - connect.assertions().assertConnectorAndExactlyNumTasksAreRunning(CONNECTOR_NAME, increasedNumTasks, "Connector task statuses did not update in time."); + int increasedNumTasks = 5; + props.put(TASKS_MAX_CONFIG, String.valueOf(increasedNumTasks)); + connect.configureConnector(CONNECTOR_NAME, props); + connect.assertions().assertConnectorAndExactlyNumTasksAreRunning(CONNECTOR_NAME, + increasedNumTasks, "Connector task statuses did not update in time."); // then reconfigure it to use fewer tasks - final int decreasedNumTasks = 3; - connectorProps.put(TASKS_MAX_CONFIG, String.valueOf(decreasedNumTasks)); - connect.configureConnector(CONNECTOR_NAME, connectorProps); - connect.assertions().assertConnectorAndExactlyNumTasksAreRunning(CONNECTOR_NAME, decreasedNumTasks, "Connector task statuses did not update in time."); + int decreasedNumTasks = 3; + props.put(TASKS_MAX_CONFIG, String.valueOf(decreasedNumTasks)); + connect.configureConnector(CONNECTOR_NAME, props); + connect.assertions().assertConnectorAndExactlyNumTasksAreRunning(CONNECTOR_NAME, + decreasedNumTasks, "Connector task statuses did not update in time."); + } + + private Map defaultSourceConnectorProps(String topic) { + // setup up props for the source connector + Map props = new HashMap<>(); + props.put(CONNECTOR_CLASS_CONFIG, MonitorableSourceConnector.class.getSimpleName()); + props.put(TASKS_MAX_CONFIG, String.valueOf(NUM_TASKS)); + props.put(TOPIC_CONFIG, topic); + props.put("throughput", String.valueOf(10)); + props.put("messages.per.poll", String.valueOf(10)); + props.put(KEY_CONVERTER_CLASS_CONFIG, StringConverter.class.getName()); + props.put(VALUE_CONVERTER_CLASS_CONFIG, StringConverter.class.getName()); + props.put(DEFAULT_TOPIC_CREATION_PREFIX + REPLICATION_FACTOR_CONFIG, String.valueOf(1)); + props.put(DEFAULT_TOPIC_CREATION_PREFIX + PARTITIONS_CONFIG, String.valueOf(1)); + return props; } } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectorTopicsIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectorTopicsIntegrationTest.java index acfc896705858..75374a94819ac 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectorTopicsIntegrationTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectorTopicsIntegrationTest.java @@ -51,6 +51,9 @@ 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.TopicCreationConfig.DEFAULT_TOPIC_CREATION_PREFIX; +import static org.apache.kafka.connect.runtime.TopicCreationConfig.PARTITIONS_CONFIG; +import static org.apache.kafka.connect.runtime.TopicCreationConfig.REPLICATION_FACTOR_CONFIG; import static org.apache.kafka.connect.runtime.WorkerConfig.CONNECTOR_CLIENT_POLICY_CLASS_CONFIG; import static org.apache.kafka.connect.runtime.WorkerConfig.TOPIC_TRACKING_ALLOW_RESET_CONFIG; import static org.apache.kafka.connect.runtime.WorkerConfig.TOPIC_TRACKING_ENABLE_CONFIG; @@ -304,6 +307,8 @@ private Map defaultSourceConnectorProps(String topic) { props.put("messages.per.poll", String.valueOf(10)); props.put(KEY_CONVERTER_CLASS_CONFIG, StringConverter.class.getName()); props.put(VALUE_CONVERTER_CLASS_CONFIG, StringConverter.class.getName()); + props.put(DEFAULT_TOPIC_CREATION_PREFIX + REPLICATION_FACTOR_CONFIG, String.valueOf(1)); + props.put(DEFAULT_TOPIC_CREATION_PREFIX + PARTITIONS_CONFIG, String.valueOf(1)); return props; } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ExampleConnectIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ExampleConnectIntegrationTest.java index 0773fd97b4ac4..8538fb401dd39 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ExampleConnectIntegrationTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ExampleConnectIntegrationTest.java @@ -37,6 +37,9 @@ 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.connect.runtime.TopicCreationConfig.DEFAULT_TOPIC_CREATION_PREFIX; +import static org.apache.kafka.connect.runtime.TopicCreationConfig.PARTITIONS_CONFIG; +import static org.apache.kafka.connect.runtime.TopicCreationConfig.REPLICATION_FACTOR_CONFIG; import static org.apache.kafka.connect.runtime.WorkerConfig.OFFSET_COMMIT_INTERVAL_MS_CONFIG; import static org.apache.kafka.test.TestUtils.waitForCondition; import static org.junit.Assert.assertEquals; @@ -170,7 +173,7 @@ public void testSourceConnector() throws Exception { // create test topic connect.kafka().createTopic("test-topic", NUM_TOPIC_PARTITIONS); - // setup up props for the sink connector + // setup up props for the source connector Map props = new HashMap<>(); props.put(CONNECTOR_CLASS_CONFIG, SOURCE_CONNECTOR_CLASS_NAME); props.put(TASKS_MAX_CONFIG, String.valueOf(NUM_TASKS)); @@ -178,6 +181,8 @@ public void testSourceConnector() throws Exception { props.put("throughput", String.valueOf(500)); props.put(KEY_CONVERTER_CLASS_CONFIG, StringConverter.class.getName()); props.put(VALUE_CONVERTER_CLASS_CONFIG, StringConverter.class.getName()); + props.put(DEFAULT_TOPIC_CREATION_PREFIX + REPLICATION_FACTOR_CONFIG, String.valueOf(1)); + props.put(DEFAULT_TOPIC_CREATION_PREFIX + PARTITIONS_CONFIG, String.valueOf(1)); // expect all records to be produced by the connector connectorHandle.expectedRecords(NUM_RECORDS_PRODUCED); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/RebalanceSourceConnectorsIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/RebalanceSourceConnectorsIntegrationTest.java index 19e786317089b..d626378d22aa0 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/RebalanceSourceConnectorsIntegrationTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/RebalanceSourceConnectorsIntegrationTest.java @@ -42,6 +42,9 @@ 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.TopicCreationConfig.DEFAULT_TOPIC_CREATION_PREFIX; +import static org.apache.kafka.connect.runtime.TopicCreationConfig.PARTITIONS_CONFIG; +import static org.apache.kafka.connect.runtime.TopicCreationConfig.REPLICATION_FACTOR_CONFIG; import static org.apache.kafka.connect.runtime.WorkerConfig.OFFSET_COMMIT_INTERVAL_MS_CONFIG; import static org.apache.kafka.connect.runtime.distributed.ConnectProtocolCompatibility.COMPATIBLE; import static org.apache.kafka.connect.runtime.distributed.DistributedConfig.CONNECT_PROTOCOL_CONFIG; @@ -105,14 +108,7 @@ public void testStartTwoConnectors() throws Exception { connect.kafka().createTopic(TOPIC_NAME, NUM_TOPIC_PARTITIONS); // setup up props for the source connector - Map props = new HashMap<>(); - props.put(CONNECTOR_CLASS_CONFIG, MonitorableSourceConnector.class.getSimpleName()); - props.put(TASKS_MAX_CONFIG, String.valueOf(NUM_TASKS)); - props.put("throughput", String.valueOf(1)); - props.put("messages.per.poll", String.valueOf(10)); - props.put(TOPIC_CONFIG, TOPIC_NAME); - props.put(KEY_CONVERTER_CLASS_CONFIG, StringConverter.class.getName()); - props.put(VALUE_CONVERTER_CLASS_CONFIG, StringConverter.class.getName()); + Map props = defaultSourceConnectorProps(TOPIC_NAME); connect.assertions().assertAtLeastNumWorkersAreUp(NUM_WORKERS, "Connect workers did not start in time."); @@ -144,14 +140,7 @@ public void testReconfigConnector() throws Exception { connect.kafka().createTopic(anotherTopic, NUM_TOPIC_PARTITIONS); // setup up props for the source connector - Map props = new HashMap<>(); - props.put(CONNECTOR_CLASS_CONFIG, MonitorableSourceConnector.class.getSimpleName()); - props.put(TASKS_MAX_CONFIG, String.valueOf(NUM_TASKS)); - props.put("throughput", String.valueOf(1)); - props.put("messages.per.poll", String.valueOf(10)); - props.put(TOPIC_CONFIG, TOPIC_NAME); - props.put(KEY_CONVERTER_CLASS_CONFIG, StringConverter.class.getName()); - props.put(VALUE_CONVERTER_CLASS_CONFIG, StringConverter.class.getName()); + Map props = defaultSourceConnectorProps(TOPIC_NAME); connect.assertions().assertAtLeastNumWorkersAreUp(NUM_WORKERS, "Connect workers did not start in time."); @@ -199,14 +188,7 @@ public void testDeleteConnector() throws Exception { connect.kafka().createTopic(TOPIC_NAME, NUM_TOPIC_PARTITIONS); // setup up props for the source connector - Map props = new HashMap<>(); - props.put(CONNECTOR_CLASS_CONFIG, MonitorableSourceConnector.class.getSimpleName()); - props.put(TASKS_MAX_CONFIG, String.valueOf(NUM_TASKS)); - props.put("throughput", String.valueOf(1)); - props.put("messages.per.poll", String.valueOf(10)); - props.put(TOPIC_CONFIG, TOPIC_NAME); - props.put(KEY_CONVERTER_CLASS_CONFIG, StringConverter.class.getName()); - props.put(VALUE_CONVERTER_CLASS_CONFIG, StringConverter.class.getName()); + Map props = defaultSourceConnectorProps(TOPIC_NAME); connect.assertions().assertAtLeastNumWorkersAreUp(NUM_WORKERS, "Connect workers did not start in time."); @@ -233,14 +215,7 @@ public void testAddingWorker() throws Exception { connect.kafka().createTopic(TOPIC_NAME, NUM_TOPIC_PARTITIONS); // setup up props for the source connector - Map props = new HashMap<>(); - props.put(CONNECTOR_CLASS_CONFIG, MonitorableSourceConnector.class.getSimpleName()); - props.put(TASKS_MAX_CONFIG, String.valueOf(NUM_TASKS)); - props.put("throughput", String.valueOf(1)); - props.put("messages.per.poll", String.valueOf(10)); - props.put(TOPIC_CONFIG, TOPIC_NAME); - props.put(KEY_CONVERTER_CLASS_CONFIG, StringConverter.class.getName()); - props.put(VALUE_CONVERTER_CLASS_CONFIG, StringConverter.class.getName()); + Map props = defaultSourceConnectorProps(TOPIC_NAME); connect.assertions().assertAtLeastNumWorkersAreUp(NUM_WORKERS, "Connect workers did not start in time."); @@ -269,14 +244,7 @@ public void testRemovingWorker() throws Exception { connect.kafka().createTopic(TOPIC_NAME, NUM_TOPIC_PARTITIONS); // setup up props for the source connector - Map props = new HashMap<>(); - props.put(CONNECTOR_CLASS_CONFIG, MonitorableSourceConnector.class.getSimpleName()); - props.put(TASKS_MAX_CONFIG, String.valueOf(NUM_TASKS)); - props.put("throughput", String.valueOf(1)); - props.put("messages.per.poll", String.valueOf(10)); - props.put(TOPIC_CONFIG, TOPIC_NAME); - props.put(KEY_CONVERTER_CLASS_CONFIG, StringConverter.class.getName()); - props.put(VALUE_CONVERTER_CLASS_CONFIG, StringConverter.class.getName()); + Map props = defaultSourceConnectorProps(TOPIC_NAME); connect.assertions().assertExactlyNumWorkersAreUp(NUM_WORKERS, "Connect workers did not start in time."); @@ -296,6 +264,21 @@ public void testRemovingWorker() throws Exception { WORKER_SETUP_DURATION_MS, "Connect and tasks are imbalanced between the workers."); } + private Map defaultSourceConnectorProps(String topic) { + // setup up props for the source connector + Map props = new HashMap<>(); + props.put(CONNECTOR_CLASS_CONFIG, MonitorableSourceConnector.class.getSimpleName()); + props.put(TASKS_MAX_CONFIG, String.valueOf(NUM_TASKS)); + props.put(TOPIC_CONFIG, topic); + props.put("throughput", String.valueOf(10)); + props.put("messages.per.poll", String.valueOf(10)); + props.put(KEY_CONVERTER_CLASS_CONFIG, StringConverter.class.getName()); + props.put(VALUE_CONVERTER_CLASS_CONFIG, StringConverter.class.getName()); + props.put(DEFAULT_TOPIC_CREATION_PREFIX + REPLICATION_FACTOR_CONFIG, String.valueOf(1)); + props.put(DEFAULT_TOPIC_CREATION_PREFIX + PARTITIONS_CONFIG, String.valueOf(1)); + return props; + } + private boolean assertConnectorAndTasksAreUnique() { try { Map> connectors = new HashMap<>(); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/SourceConnectorsIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/SourceConnectorsIntegrationTest.java new file mode 100644 index 0000000000000..e6811f5c3bba2 --- /dev/null +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/SourceConnectorsIntegrationTest.java @@ -0,0 +1,233 @@ +/* + * 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.connect.runtime.SourceConnectorConfig; +import org.apache.kafka.connect.storage.StringConverter; +import org.apache.kafka.connect.util.clusters.EmbeddedConnectCluster; +import org.apache.kafka.test.IntegrationTest; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import java.util.Collections; +import java.util.HashMap; +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.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; +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.SourceConnectorConfig.TOPIC_CREATION_GROUPS_CONFIG; +import static org.apache.kafka.connect.runtime.TopicCreationConfig.DEFAULT_TOPIC_CREATION_PREFIX; +import static org.apache.kafka.connect.runtime.TopicCreationConfig.EXCLUDE_REGEX_CONFIG; +import static org.apache.kafka.connect.runtime.TopicCreationConfig.INCLUDE_REGEX_CONFIG; +import static org.apache.kafka.connect.runtime.TopicCreationConfig.PARTITIONS_CONFIG; +import static org.apache.kafka.connect.runtime.TopicCreationConfig.REPLICATION_FACTOR_CONFIG; +import static org.apache.kafka.connect.runtime.WorkerConfig.CONNECTOR_CLIENT_POLICY_CLASS_CONFIG; +import static org.apache.kafka.connect.runtime.WorkerConfig.TOPIC_CREATION_ENABLE_CONFIG; +import static org.apache.kafka.connect.util.clusters.EmbeddedConnectCluster.DEFAULT_NUM_BROKERS; + +/** + * Integration test for source connectors with a focus on topic creation with custom properties by + * the connector tasks. + */ +@Category(IntegrationTest.class) +public class SourceConnectorsIntegrationTest { + + private static final int NUM_WORKERS = 3; + private static final int NUM_TASKS = 1; + private static final String FOO_TOPIC = "foo-topic"; + private static final String FOO_CONNECTOR = "foo-source"; + private static final String BAR_TOPIC = "bar-topic"; + private static final String BAR_CONNECTOR = "bar-source"; + private static final String FOO_GROUP = "foo"; + private static final String BAR_GROUP = "bar"; + private static final int DEFAULT_REPLICATION_FACTOR = DEFAULT_NUM_BROKERS; + private static final int DEFAULT_PARTITIONS = 1; + private static final int FOO_GROUP_REPLICATION_FACTOR = DEFAULT_NUM_BROKERS; + private static final int FOO_GROUP_PARTITIONS = 9; + + private EmbeddedConnectCluster.Builder connectBuilder; + private EmbeddedConnectCluster connect; + Map workerProps = new HashMap<>(); + Properties brokerProps = new Properties(); + + @Before + public void setup() { + // setup Connect worker properties + workerProps.put(CONNECTOR_CLIENT_POLICY_CLASS_CONFIG, "All"); + + // setup Kafka broker properties + brokerProps.put("auto.create.topics.enable", String.valueOf(false)); + + // build a Connect cluster backed by Kafka and Zk + connectBuilder = new EmbeddedConnectCluster.Builder() + .name("connect-cluster") + .numWorkers(NUM_WORKERS) + .workerProps(workerProps) + .brokerProps(brokerProps) + .maskExitProcedures(true); // true is the default, setting here as example + } + + @After + public void close() { + // stop all Connect, Kafka and Zk threads. + connect.stop(); + } + + @Test + public void testTopicsAreCreatedWhenAutoCreateTopicsIsEnabledAtTheBroker() throws InterruptedException { + brokerProps.put("auto.create.topics.enable", String.valueOf(true)); + workerProps.put(TOPIC_CREATION_ENABLE_CONFIG, String.valueOf(false)); + connect = connectBuilder.brokerProps(brokerProps).workerProps(workerProps).build(); + // start the clusters + connect.start(); + + connect.assertions().assertAtLeastNumWorkersAreUp(NUM_WORKERS, "Initial group of workers did not start in time."); + + Map fooProps = sourceConnectorPropsWithGroups(FOO_TOPIC); + + // start a source connector + connect.configureConnector(FOO_CONNECTOR, fooProps); + fooProps.put(NAME_CONFIG, FOO_CONNECTOR); + + connect.assertions().assertExactlyNumErrorsOnConnectorConfigValidation(fooProps.get(CONNECTOR_CLASS_CONFIG), fooProps, 0, + "Validating connector configuration produced an unexpected number or errors."); + + connect.assertions().assertConnectorAndAtLeastNumTasksAreRunning(FOO_CONNECTOR, NUM_TASKS, + "Connector tasks did not start in time."); + + connect.assertions().assertTopicsExist(FOO_TOPIC); + connect.assertions().assertTopicSettings(FOO_TOPIC, DEFAULT_REPLICATION_FACTOR, DEFAULT_PARTITIONS); + } + + @Test + public void testTopicsAreCreatedWhenTopicCreationIsEnabled() throws InterruptedException { + connect = connectBuilder.build(); + // start the clusters + connect.start(); + + connect.assertions().assertAtLeastNumWorkersAreUp(NUM_WORKERS, "Initial group of workers did not start in time."); + + Map fooProps = sourceConnectorPropsWithGroups(FOO_TOPIC); + + // start a source connector + connect.configureConnector(FOO_CONNECTOR, fooProps); + fooProps.put(NAME_CONFIG, FOO_CONNECTOR); + + connect.assertions().assertExactlyNumErrorsOnConnectorConfigValidation(fooProps.get(CONNECTOR_CLASS_CONFIG), fooProps, 0, + "Validating connector configuration produced an unexpected number or errors."); + + connect.assertions().assertConnectorAndAtLeastNumTasksAreRunning(FOO_CONNECTOR, NUM_TASKS, + "Connector tasks did not start in time."); + + connect.assertions().assertTopicsExist(FOO_TOPIC); + connect.assertions().assertTopicSettings(FOO_TOPIC, FOO_GROUP_REPLICATION_FACTOR, FOO_GROUP_PARTITIONS); + } + + @Test + public void testSwitchingToTopicCreationEnabled() throws InterruptedException { + workerProps.put(TOPIC_CREATION_ENABLE_CONFIG, String.valueOf(false)); + connect = connectBuilder.build(); + // start the clusters + connect.start(); + + connect.kafka().createTopic(BAR_TOPIC, DEFAULT_PARTITIONS, DEFAULT_REPLICATION_FACTOR, Collections.emptyMap()); + + connect.assertions().assertTopicsExist(BAR_TOPIC); + connect.assertions().assertTopicSettings(BAR_TOPIC, DEFAULT_REPLICATION_FACTOR, DEFAULT_PARTITIONS); + + connect.assertions().assertAtLeastNumWorkersAreUp(NUM_WORKERS, "Initial group of workers did not start in time."); + + Map barProps = defaultSourceConnectorProps(BAR_TOPIC); + // start a source connector with topic creation properties + connect.configureConnector(BAR_CONNECTOR, barProps); + barProps.put(NAME_CONFIG, BAR_CONNECTOR); + + Map fooProps = sourceConnectorPropsWithGroups(FOO_TOPIC); + // start a source connector without topic creation properties + connect.configureConnector(FOO_CONNECTOR, fooProps); + fooProps.put(NAME_CONFIG, FOO_CONNECTOR); + + connect.assertions().assertExactlyNumErrorsOnConnectorConfigValidation(fooProps.get(CONNECTOR_CLASS_CONFIG), fooProps, 0, + "Validating connector configuration produced an unexpected number or errors."); + + connect.assertions().assertExactlyNumErrorsOnConnectorConfigValidation(barProps.get(CONNECTOR_CLASS_CONFIG), barProps, 0, + "Validating connector configuration produced an unexpected number or errors."); + + connect.assertions().assertConnectorAndAtLeastNumTasksAreRunning(FOO_CONNECTOR, NUM_TASKS, + "Connector tasks did not start in time."); + + connect.assertions().assertConnectorAndAtLeastNumTasksAreRunning(BAR_CONNECTOR, NUM_TASKS, + "Connector tasks did not start in time."); + + connect.assertions().assertTopicsExist(BAR_TOPIC); + connect.assertions().assertTopicSettings(BAR_TOPIC, DEFAULT_REPLICATION_FACTOR, DEFAULT_PARTITIONS); + + connect.assertions().assertTopicsDoNotExist(FOO_TOPIC); + + connect.activeWorkers().forEach(w -> connect.removeWorker(w)); + + workerProps.put(TOPIC_CREATION_ENABLE_CONFIG, String.valueOf(true)); + + IntStream.range(0, 3).forEach(i -> connect.addWorker()); + connect.assertions().assertAtLeastNumWorkersAreUp(NUM_WORKERS, "Initial group of workers did not start in time."); + + connect.assertions().assertConnectorAndAtLeastNumTasksAreRunning(FOO_CONNECTOR, NUM_TASKS, + "Connector tasks did not start in time."); + + connect.assertions().assertConnectorAndAtLeastNumTasksAreRunning(BAR_CONNECTOR, NUM_TASKS, + "Connector tasks did not start in time."); + + connect.assertions().assertTopicsExist(FOO_TOPIC); + connect.assertions().assertTopicSettings(FOO_TOPIC, FOO_GROUP_REPLICATION_FACTOR, FOO_GROUP_PARTITIONS); + connect.assertions().assertTopicsExist(BAR_TOPIC); + connect.assertions().assertTopicSettings(BAR_TOPIC, DEFAULT_REPLICATION_FACTOR, DEFAULT_PARTITIONS); + } + + private Map defaultSourceConnectorProps(String topic) { + // setup up props for the source connector + Map props = new HashMap<>(); + props.put(CONNECTOR_CLASS_CONFIG, MonitorableSourceConnector.class.getSimpleName()); + props.put(TASKS_MAX_CONFIG, String.valueOf(NUM_TASKS)); + props.put(TOPIC_CONFIG, topic); + props.put("throughput", String.valueOf(10)); + props.put("messages.per.poll", String.valueOf(10)); + props.put(KEY_CONVERTER_CLASS_CONFIG, StringConverter.class.getName()); + props.put(VALUE_CONVERTER_CLASS_CONFIG, StringConverter.class.getName()); + return props; + } + + private Map sourceConnectorPropsWithGroups(String topic) { + // setup up props for the source connector + Map props = defaultSourceConnectorProps(topic); + props.put(TOPIC_CREATION_GROUPS_CONFIG, String.join(",", FOO_GROUP, BAR_GROUP)); + props.put(DEFAULT_TOPIC_CREATION_PREFIX + REPLICATION_FACTOR_CONFIG, String.valueOf(DEFAULT_REPLICATION_FACTOR)); + props.put(DEFAULT_TOPIC_CREATION_PREFIX + PARTITIONS_CONFIG, String.valueOf(DEFAULT_PARTITIONS)); + props.put(SourceConnectorConfig.TOPIC_CREATION_PREFIX + FOO_GROUP + "." + INCLUDE_REGEX_CONFIG, FOO_TOPIC); + props.put(SourceConnectorConfig.TOPIC_CREATION_PREFIX + FOO_GROUP + "." + EXCLUDE_REGEX_CONFIG, BAR_TOPIC); + props.put(SourceConnectorConfig.TOPIC_CREATION_PREFIX + FOO_GROUP + "." + PARTITIONS_CONFIG, + String.valueOf(FOO_GROUP_PARTITIONS)); + return props; + } +} diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/AbstractHerderTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/AbstractHerderTest.java index 149877aedf170..97bba9468639b 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/AbstractHerderTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/AbstractHerderTest.java @@ -61,6 +61,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.function.Function; import java.util.stream.Collectors; import static org.apache.kafka.connect.runtime.AbstractHerder.keysWithVariableValues; @@ -289,16 +290,19 @@ public void testConfigValidationMissingName() { // We expect there to be errors due to the missing name and .... Note that these assertions depend heavily on // the config fields for SourceConnectorConfig, but we expect these to change rarely. assertEquals(TestSourceConnector.class.getName(), result.name()); - assertEquals(Arrays.asList(ConnectorConfig.COMMON_GROUP, ConnectorConfig.TRANSFORMS_GROUP, ConnectorConfig.ERROR_GROUP), result.groups()); + assertEquals(Arrays.asList(ConnectorConfig.COMMON_GROUP, ConnectorConfig.TRANSFORMS_GROUP, + ConnectorConfig.ERROR_GROUP, SourceConnectorConfig.TOPIC_CREATION_GROUP), result.groups()); assertEquals(2, result.errorCount()); - // Base connector config has 13 fields, connector's configs add 2 - assertEquals(15, result.values().size()); + Map infos = result.values().stream() + .collect(Collectors.toMap(info -> info.configKey().name(), Function.identity())); + assertEquals(16, infos.size()); // Missing name should generate an error - assertEquals(ConnectorConfig.NAME_CONFIG, result.values().get(0).configValue().name()); - assertEquals(1, result.values().get(0).configValue().errors().size()); + assertEquals(ConnectorConfig.NAME_CONFIG, + infos.get(ConnectorConfig.NAME_CONFIG).configValue().name()); + assertEquals(1, infos.get(ConnectorConfig.NAME_CONFIG).configValue().errors().size()); // "required" config from connector should generate an error - assertEquals("required", result.values().get(13).configValue().name()); - assertEquals(1, result.values().get(13).configValue().errors().size()); + assertEquals("required", infos.get("required").configValue().name()); + assertEquals(1, infos.get("required").configValue().errors().size()); verifyAll(); } @@ -348,20 +352,23 @@ public void testConfigValidationTransformsExtendResults() { ConnectorConfig.COMMON_GROUP, ConnectorConfig.TRANSFORMS_GROUP, ConnectorConfig.ERROR_GROUP, + SourceConnectorConfig.TOPIC_CREATION_GROUP, "Transforms: xformA", "Transforms: xformB" ); assertEquals(expectedGroups, result.groups()); assertEquals(2, result.errorCount()); - // Base connector config has 13 fields, connector's configs add 2, 2 type fields from the transforms, and - // 1 from the valid transformation's config - assertEquals(18, result.values().size()); + Map infos = result.values().stream() + .collect(Collectors.toMap(info -> info.configKey().name(), Function.identity())); + assertEquals(19, infos.size()); // Should get 2 type fields from the transforms, first adds its own config since it has a valid class - assertEquals("transforms.xformA.type", result.values().get(13).configValue().name()); - assertTrue(result.values().get(13).configValue().errors().isEmpty()); - assertEquals("transforms.xformA.subconfig", result.values().get(14).configValue().name()); - assertEquals("transforms.xformB.type", result.values().get(15).configValue().name()); - assertFalse(result.values().get(15).configValue().errors().isEmpty()); + assertEquals("transforms.xformA.type", + infos.get("transforms.xformA.type").configValue().name()); + assertTrue(infos.get("transforms.xformA.type").configValue().errors().isEmpty()); + assertEquals("transforms.xformA.subconfig", + infos.get("transforms.xformA.subconfig").configValue().name()); + assertEquals("transforms.xformB.type", infos.get("transforms.xformB.type").configValue().name()); + assertFalse(infos.get("transforms.xformB.type").configValue().errors().isEmpty()); verifyAll(); } @@ -390,12 +397,13 @@ public void testConfigValidationPrincipalOnlyOverride() { List expectedGroups = Arrays.asList( ConnectorConfig.COMMON_GROUP, ConnectorConfig.TRANSFORMS_GROUP, - ConnectorConfig.ERROR_GROUP + ConnectorConfig.ERROR_GROUP, + SourceConnectorConfig.TOPIC_CREATION_GROUP ); assertEquals(expectedGroups, result.groups()); assertEquals(1, result.errorCount()); // Base connector config has 13 fields, connector's configs add 2, and 2 producer overrides - assertEquals(17, result.values().size()); + assertEquals(18, result.values().size()); assertTrue(result.values().stream().anyMatch( configInfo -> ackConfigKey.equals(configInfo.configValue().name()) && !configInfo.configValue().errors().isEmpty())); assertTrue(result.values().stream().anyMatch( diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ConnectorConfigTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ConnectorConfigTest.java index f674a8e22f244..48d494767228e 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ConnectorConfigTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ConnectorConfigTest.java @@ -37,7 +37,7 @@ public class ConnectorConfigTest> { - private static final Plugins MOCK_PLUGINS = new Plugins(new HashMap()) { + public static final Plugins MOCK_PLUGINS = new Plugins(new HashMap()) { @Override public Set> transformations() { return Collections.emptySet(); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ErrorHandlingTaskTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ErrorHandlingTaskTest.java index f3681593ef63f..443490488f8d2 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ErrorHandlingTaskTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ErrorHandlingTaskTest.java @@ -16,7 +16,7 @@ */ package org.apache.kafka.connect.runtime; -import java.util.Arrays; +import org.apache.kafka.clients.admin.NewTopic; import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; @@ -31,6 +31,7 @@ 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.json.JsonConverter; import org.apache.kafka.connect.runtime.distributed.ClusterConfigState; import org.apache.kafka.connect.runtime.errors.ErrorHandlingMetrics; @@ -51,9 +52,11 @@ import org.apache.kafka.connect.storage.OffsetStorageReaderImpl; import org.apache.kafka.connect.storage.OffsetStorageWriter; import org.apache.kafka.connect.storage.StatusBackingStore; +import org.apache.kafka.connect.storage.StringConverter; import org.apache.kafka.connect.transforms.Transformation; import org.apache.kafka.connect.transforms.util.SimpleConfig; import org.apache.kafka.connect.util.ConnectorTaskId; +import org.apache.kafka.connect.util.TopicAdmin; import org.easymock.Capture; import org.easymock.EasyMock; import org.easymock.IExpectationSetters; @@ -70,6 +73,7 @@ import org.slf4j.LoggerFactory; import java.time.Duration; +import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.Map; @@ -77,6 +81,12 @@ 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.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.WorkerConfig.TOPIC_CREATION_ENABLE_CONFIG; import static org.junit.Assert.assertEquals; @RunWith(PowerMockRunner.class) @@ -116,6 +126,7 @@ public class ErrorHandlingTaskTest { private SourceTask sourceTask; private Capture sinkTaskContext = EasyMock.newCapture(); private WorkerConfig workerConfig; + private SourceConnectorConfig sourceConfig; @Mock private PluginClassLoader pluginLoader; @SuppressWarnings("unused") @@ -129,6 +140,8 @@ public class ErrorHandlingTaskTest { @SuppressWarnings("unused") @Mock private KafkaProducer producer; + @SuppressWarnings("unused") + @Mock private TopicAdmin admin; @Mock OffsetStorageReaderImpl offsetReader; @@ -139,10 +152,14 @@ public class ErrorHandlingTaskTest { @SuppressWarnings("unused") @Mock private TaskStatus.Listener statusListener; + @SuppressWarnings("unused") @Mock private StatusBackingStore statusBackingStore; private ErrorHandlingMetrics errorHandlingMetrics; + // when this test becomes parameterized, this variable will be a test parameter + public boolean enableTopicCreation = false; + @Before public void setup() { time = new MockTime(0, 0, 0); @@ -155,11 +172,25 @@ public void setup() { workerProps.put("internal.key.converter.schemas.enable", "false"); workerProps.put("internal.value.converter.schemas.enable", "false"); workerProps.put("offset.storage.file.filename", "/tmp/connect.offsets"); + workerProps.put(TOPIC_CREATION_ENABLE_CONFIG, String.valueOf(enableTopicCreation)); pluginLoader = PowerMock.createMock(PluginClassLoader.class); workerConfig = new StandaloneConfig(workerProps); + sourceConfig = new SourceConnectorConfig(plugins, sourceConnectorProps(TOPIC), true); errorHandlingMetrics = new ErrorHandlingMetrics(taskId, metrics); } + private Map sourceConnectorProps(String topic) { + // setup up props for the source connector + Map props = new HashMap<>(); + props.put("name", "foo-connector"); + props.put(CONNECTOR_CLASS_CONFIG, MonitorableSourceConnector.class.getSimpleName()); + props.put(TASKS_MAX_CONFIG, String.valueOf(1)); + props.put(TOPIC_CONFIG, topic); + props.put(KEY_CONVERTER_CLASS_CONFIG, StringConverter.class.getName()); + props.put(VALUE_CONVERTER_CLASS_CONFIG, StringConverter.class.getName()); + return props; + } + @After public void tearDown() { if (metrics != null) { @@ -208,8 +239,7 @@ public void testSourceTasksCloseErrorReporters() { sourceTask.stop(); PowerMock.expectLastCall(); - producer.close(EasyMock.anyObject()); - PowerMock.expectLastCall(); + expectClose(); reporter.close(); EasyMock.expectLastCall(); @@ -236,8 +266,7 @@ public void testCloseErrorReportersExceptionPropagation() { sourceTask.stop(); PowerMock.expectLastCall(); - producer.close(EasyMock.anyObject()); - PowerMock.expectLastCall(); + expectClose(); // Even though the reporters throw exceptions, they should both still be closed. reporterA.close(); @@ -341,6 +370,7 @@ public void testErrorHandlingInSourceTasks() throws Exception { EasyMock.expect(sourceTask.poll()).andReturn(singletonList(record1)); EasyMock.expect(sourceTask.poll()).andReturn(singletonList(record2)); + expectTopicCreation(TOPIC); EasyMock.expect(producer.send(EasyMock.anyObject(), EasyMock.anyObject())).andReturn(null).times(2); PowerMock.replayAll(); @@ -405,6 +435,7 @@ public void testErrorHandlingInSourceTasksWthBadConverter() throws Exception { EasyMock.expect(sourceTask.poll()).andReturn(singletonList(record1)); EasyMock.expect(sourceTask.poll()).andReturn(singletonList(record2)); + expectTopicCreation(TOPIC); EasyMock.expect(producer.send(EasyMock.anyObject(), EasyMock.anyObject())).andReturn(null).times(2); PowerMock.replayAll(); @@ -477,6 +508,23 @@ private void expectTaskGetTopic(boolean anyTimes) { } } + private void expectClose() { + producer.close(EasyMock.anyObject(Duration.class)); + EasyMock.expectLastCall(); + + admin.close(EasyMock.anyObject(Duration.class)); + EasyMock.expectLastCall(); + } + + private void expectTopicCreation(String topic) { + if (workerConfig.topicCreationEnable()) { + EasyMock.expect(admin.describeTopics(topic)).andReturn(Collections.emptyMap()); + + Capture newTopicCapture = EasyMock.newCapture(); + EasyMock.expect(admin.createTopic(EasyMock.capture(newTopicCapture))).andReturn(true); + } + } + private void createSinkTask(TargetState initialState, RetryWithToleranceOperator retryWithToleranceOperator) { JsonConverter converter = new JsonConverter(); Map oo = workerConfig.originalsWithPrefix("value.converter."); @@ -518,7 +566,8 @@ private void createSourceTask(TargetState initialState, RetryWithToleranceOperat workerSourceTask = PowerMock.createPartialMock( WorkerSourceTask.class, new String[]{"commitOffsets", "isStopping"}, taskId, sourceTask, statusListener, initialState, converter, converter, headerConverter, sourceTransforms, - producer, offsetReader, offsetWriter, workerConfig, + producer, admin, null, + offsetReader, offsetWriter, workerConfig, ClusterConfigState.EMPTY, metrics, pluginLoader, time, retryWithToleranceOperator, statusBackingStore); } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ErrorHandlingTaskWithTopicCreationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ErrorHandlingTaskWithTopicCreationTest.java new file mode 100644 index 0000000000000..909ec4ab994d2 --- /dev/null +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/ErrorHandlingTaskWithTopicCreationTest.java @@ -0,0 +1,655 @@ +/* + * 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.runtime; + +import org.apache.kafka.clients.admin.NewTopic; +import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.KafkaConsumer; +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.utils.MockTime; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.connect.connector.ConnectRecord; +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.json.JsonConverter; +import org.apache.kafka.connect.runtime.distributed.ClusterConfigState; +import org.apache.kafka.connect.runtime.errors.ErrorHandlingMetrics; +import org.apache.kafka.connect.runtime.errors.ErrorReporter; +import org.apache.kafka.connect.runtime.errors.LogReporter; +import org.apache.kafka.connect.runtime.errors.RetryWithToleranceOperator; +import org.apache.kafka.connect.runtime.errors.ToleranceType; +import org.apache.kafka.connect.runtime.isolation.PluginClassLoader; +import org.apache.kafka.connect.runtime.isolation.Plugins; +import org.apache.kafka.connect.runtime.standalone.StandaloneConfig; +import org.apache.kafka.connect.sink.SinkConnector; +import org.apache.kafka.connect.sink.SinkRecord; +import org.apache.kafka.connect.sink.SinkTask; +import org.apache.kafka.connect.source.SourceRecord; +import org.apache.kafka.connect.source.SourceTask; +import org.apache.kafka.connect.storage.Converter; +import org.apache.kafka.connect.storage.HeaderConverter; +import org.apache.kafka.connect.storage.OffsetStorageReaderImpl; +import org.apache.kafka.connect.storage.OffsetStorageWriter; +import org.apache.kafka.connect.storage.StatusBackingStore; +import org.apache.kafka.connect.storage.StringConverter; +import org.apache.kafka.connect.transforms.Transformation; +import org.apache.kafka.connect.transforms.util.SimpleConfig; +import org.apache.kafka.connect.util.ConnectorTaskId; +import org.apache.kafka.connect.util.TopicAdmin; +import org.apache.kafka.connect.util.TopicCreationGroup; +import org.easymock.Capture; +import org.easymock.EasyMock; +import org.easymock.IExpectationSetters; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.powermock.api.easymock.PowerMock; +import org.powermock.api.easymock.annotation.Mock; +import org.powermock.core.classloader.annotations.PowerMockIgnore; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.time.Duration; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +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.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.SourceConnectorConfig.TOPIC_CREATION_GROUPS_CONFIG; +import static org.apache.kafka.connect.runtime.TopicCreationConfig.DEFAULT_TOPIC_CREATION_PREFIX; +import static org.apache.kafka.connect.runtime.TopicCreationConfig.INCLUDE_REGEX_CONFIG; +import static org.apache.kafka.connect.runtime.TopicCreationConfig.PARTITIONS_CONFIG; +import static org.apache.kafka.connect.runtime.TopicCreationConfig.REPLICATION_FACTOR_CONFIG; +import static org.apache.kafka.connect.runtime.WorkerConfig.TOPIC_CREATION_ENABLE_CONFIG; +import static org.junit.Assert.assertEquals; + +@RunWith(PowerMockRunner.class) +@PrepareForTest({WorkerSinkTask.class, WorkerSourceTask.class}) +@PowerMockIgnore("javax.management.*") +public class ErrorHandlingTaskWithTopicCreationTest { + + private static final String TOPIC = "test"; + private static final int PARTITION1 = 12; + private static final int PARTITION2 = 13; + private static final long FIRST_OFFSET = 45; + + @Mock Plugins plugins; + + private static final Map TASK_PROPS = new HashMap<>(); + + static { + TASK_PROPS.put(SinkConnector.TOPICS_CONFIG, TOPIC); + TASK_PROPS.put(TaskConfig.TASK_CLASS_CONFIG, TestSinkTask.class.getName()); + } + + public static final long OPERATOR_RETRY_TIMEOUT_MILLIS = 60000; + public static final long OPERATOR_RETRY_MAX_DELAY_MILLIS = 5000; + public static final ToleranceType OPERATOR_TOLERANCE_TYPE = ToleranceType.ALL; + + private static final TaskConfig TASK_CONFIG = new TaskConfig(TASK_PROPS); + + private ConnectorTaskId taskId = new ConnectorTaskId("job", 0); + private TargetState initialState = TargetState.STARTED; + private Time time; + private MockConnectMetrics metrics; + @SuppressWarnings("unused") + @Mock + private SinkTask sinkTask; + @SuppressWarnings("unused") + @Mock + private SourceTask sourceTask; + private Capture sinkTaskContext = EasyMock.newCapture(); + private WorkerConfig workerConfig; + private SourceConnectorConfig sourceConfig; + @Mock + private PluginClassLoader pluginLoader; + @SuppressWarnings("unused") + @Mock + private HeaderConverter headerConverter; + private WorkerSinkTask workerSinkTask; + private WorkerSourceTask workerSourceTask; + @SuppressWarnings("unused") + @Mock + private KafkaConsumer consumer; + @SuppressWarnings("unused") + @Mock + private KafkaProducer producer; + @SuppressWarnings("unused") + @Mock private TopicAdmin admin; + + @Mock + OffsetStorageReaderImpl offsetReader; + @Mock + OffsetStorageWriter offsetWriter; + + private Capture rebalanceListener = EasyMock.newCapture(); + @SuppressWarnings("unused") + @Mock + private TaskStatus.Listener statusListener; + @SuppressWarnings("unused") + @Mock private StatusBackingStore statusBackingStore; + + private ErrorHandlingMetrics errorHandlingMetrics; + + // when this test becomes parameterized, this variable will be a test parameter + public boolean enableTopicCreation = true; + + @Before + public void setup() { + time = new MockTime(0, 0, 0); + metrics = new MockConnectMetrics(); + Map workerProps = new HashMap<>(); + workerProps.put("key.converter", "org.apache.kafka.connect.json.JsonConverter"); + workerProps.put("value.converter", "org.apache.kafka.connect.json.JsonConverter"); + workerProps.put("internal.key.converter", "org.apache.kafka.connect.json.JsonConverter"); + workerProps.put("internal.value.converter", "org.apache.kafka.connect.json.JsonConverter"); + workerProps.put("internal.key.converter.schemas.enable", "false"); + workerProps.put("internal.value.converter.schemas.enable", "false"); + workerProps.put("offset.storage.file.filename", "/tmp/connect.offsets"); + workerProps.put(TOPIC_CREATION_ENABLE_CONFIG, String.valueOf(enableTopicCreation)); + pluginLoader = PowerMock.createMock(PluginClassLoader.class); + workerConfig = new StandaloneConfig(workerProps); + sourceConfig = new SourceConnectorConfig(plugins, sourceConnectorPropsWithGroups(TOPIC), true); + errorHandlingMetrics = new ErrorHandlingMetrics(taskId, metrics); + } + + private Map sourceConnectorPropsWithGroups(String topic) { + // setup up props for the source connector + Map props = new HashMap<>(); + props.put("name", "foo-connector"); + props.put(CONNECTOR_CLASS_CONFIG, MonitorableSourceConnector.class.getSimpleName()); + props.put(TASKS_MAX_CONFIG, String.valueOf(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(TOPIC_CREATION_GROUPS_CONFIG, String.join(",", "foo", "bar")); + props.put(DEFAULT_TOPIC_CREATION_PREFIX + REPLICATION_FACTOR_CONFIG, String.valueOf(1)); + props.put(DEFAULT_TOPIC_CREATION_PREFIX + PARTITIONS_CONFIG, String.valueOf(1)); + props.put(SourceConnectorConfig.TOPIC_CREATION_PREFIX + "foo" + "." + INCLUDE_REGEX_CONFIG, topic); + return props; + } + + @After + public void tearDown() { + if (metrics != null) { + metrics.stop(); + } + } + + @Test + public void testSinkTasksCloseErrorReporters() throws Exception { + ErrorReporter reporter = EasyMock.mock(ErrorReporter.class); + + RetryWithToleranceOperator retryWithToleranceOperator = operator(); + retryWithToleranceOperator.metrics(errorHandlingMetrics); + retryWithToleranceOperator.reporters(singletonList(reporter)); + + createSinkTask(initialState, retryWithToleranceOperator); + + expectInitializeTask(); + reporter.close(); + EasyMock.expectLastCall(); + sinkTask.stop(); + EasyMock.expectLastCall(); + + consumer.close(); + EasyMock.expectLastCall(); + + PowerMock.replayAll(); + + workerSinkTask.initialize(TASK_CONFIG); + workerSinkTask.initializeAndStart(); + workerSinkTask.close(); + + PowerMock.verifyAll(); + } + + @Test + public void testSourceTasksCloseErrorReporters() { + ErrorReporter reporter = EasyMock.mock(ErrorReporter.class); + + RetryWithToleranceOperator retryWithToleranceOperator = operator(); + retryWithToleranceOperator.metrics(errorHandlingMetrics); + retryWithToleranceOperator.reporters(singletonList(reporter)); + + createSourceTask(initialState, retryWithToleranceOperator); + + sourceTask.stop(); + PowerMock.expectLastCall(); + + expectClose(); + + reporter.close(); + EasyMock.expectLastCall(); + + PowerMock.replayAll(); + + workerSourceTask.initialize(TASK_CONFIG); + workerSourceTask.close(); + + PowerMock.verifyAll(); + } + + @Test + public void testCloseErrorReportersExceptionPropagation() { + ErrorReporter reporterA = EasyMock.mock(ErrorReporter.class); + ErrorReporter reporterB = EasyMock.mock(ErrorReporter.class); + + RetryWithToleranceOperator retryWithToleranceOperator = operator(); + retryWithToleranceOperator.metrics(errorHandlingMetrics); + retryWithToleranceOperator.reporters(Arrays.asList(reporterA, reporterB)); + + createSourceTask(initialState, retryWithToleranceOperator); + + sourceTask.stop(); + PowerMock.expectLastCall(); + + expectClose(); + + // Even though the reporters throw exceptions, they should both still be closed. + reporterA.close(); + EasyMock.expectLastCall().andThrow(new RuntimeException()); + + reporterB.close(); + EasyMock.expectLastCall().andThrow(new RuntimeException()); + + PowerMock.replayAll(); + + workerSourceTask.initialize(TASK_CONFIG); + workerSourceTask.close(); + + PowerMock.verifyAll(); + } + + @Test + public void testErrorHandlingInSinkTasks() throws Exception { + Map reportProps = new HashMap<>(); + reportProps.put(ConnectorConfig.ERRORS_LOG_ENABLE_CONFIG, "true"); + reportProps.put(ConnectorConfig.ERRORS_LOG_INCLUDE_MESSAGES_CONFIG, "true"); + LogReporter reporter = new LogReporter(taskId, connConfig(reportProps), errorHandlingMetrics); + + RetryWithToleranceOperator retryWithToleranceOperator = operator(); + retryWithToleranceOperator.metrics(errorHandlingMetrics); + retryWithToleranceOperator.reporters(singletonList(reporter)); + createSinkTask(initialState, retryWithToleranceOperator); + + expectInitializeTask(); + expectTaskGetTopic(true); + + // valid json + ConsumerRecord record1 = new ConsumerRecord<>(TOPIC, PARTITION1, FIRST_OFFSET, null, "{\"a\": 10}".getBytes()); + // bad json + ConsumerRecord record2 = new ConsumerRecord<>(TOPIC, PARTITION2, FIRST_OFFSET, null, "{\"a\" 10}".getBytes()); + + EasyMock.expect(consumer.poll(Duration.ofMillis(EasyMock.anyLong()))).andReturn(records(record1)); + EasyMock.expect(consumer.poll(Duration.ofMillis(EasyMock.anyLong()))).andReturn(records(record2)); + + sinkTask.put(EasyMock.anyObject()); + EasyMock.expectLastCall().times(2); + + PowerMock.replayAll(); + + workerSinkTask.initialize(TASK_CONFIG); + workerSinkTask.initializeAndStart(); + workerSinkTask.iteration(); + + workerSinkTask.iteration(); + + // two records were consumed from Kafka + assertSinkMetricValue("sink-record-read-total", 2.0); + // only one was written to the task + assertSinkMetricValue("sink-record-send-total", 1.0); + // one record completely failed (converter issues) + assertErrorHandlingMetricValue("total-record-errors", 1.0); + // 2 failures in the transformation, and 1 in the converter + assertErrorHandlingMetricValue("total-record-failures", 3.0); + // one record completely failed (converter issues), and thus was skipped + assertErrorHandlingMetricValue("total-records-skipped", 1.0); + + PowerMock.verifyAll(); + } + + private RetryWithToleranceOperator operator() { + return new RetryWithToleranceOperator(OPERATOR_RETRY_TIMEOUT_MILLIS, OPERATOR_RETRY_MAX_DELAY_MILLIS, OPERATOR_TOLERANCE_TYPE, SYSTEM); + } + + @Test + public void testErrorHandlingInSourceTasks() throws Exception { + Map reportProps = new HashMap<>(); + reportProps.put(ConnectorConfig.ERRORS_LOG_ENABLE_CONFIG, "true"); + reportProps.put(ConnectorConfig.ERRORS_LOG_INCLUDE_MESSAGES_CONFIG, "true"); + LogReporter reporter = new LogReporter(taskId, connConfig(reportProps), errorHandlingMetrics); + + RetryWithToleranceOperator retryWithToleranceOperator = operator(); + retryWithToleranceOperator.metrics(errorHandlingMetrics); + retryWithToleranceOperator.reporters(singletonList(reporter)); + createSourceTask(initialState, retryWithToleranceOperator); + + // valid json + Schema valSchema = SchemaBuilder.struct().field("val", Schema.INT32_SCHEMA).build(); + Struct struct1 = new Struct(valSchema).put("val", 1234); + SourceRecord record1 = new SourceRecord(emptyMap(), emptyMap(), TOPIC, PARTITION1, valSchema, struct1); + Struct struct2 = new Struct(valSchema).put("val", 6789); + SourceRecord record2 = new SourceRecord(emptyMap(), emptyMap(), TOPIC, PARTITION1, valSchema, struct2); + + EasyMock.expect(workerSourceTask.isStopping()).andReturn(false); + EasyMock.expect(workerSourceTask.isStopping()).andReturn(false); + EasyMock.expect(workerSourceTask.isStopping()).andReturn(true); + + EasyMock.expect(workerSourceTask.commitOffsets()).andReturn(true); + + offsetWriter.offset(EasyMock.anyObject(), EasyMock.anyObject()); + EasyMock.expectLastCall().times(2); + sourceTask.initialize(EasyMock.anyObject()); + EasyMock.expectLastCall(); + + sourceTask.start(EasyMock.anyObject()); + EasyMock.expectLastCall(); + + EasyMock.expect(sourceTask.poll()).andReturn(singletonList(record1)); + EasyMock.expect(sourceTask.poll()).andReturn(singletonList(record2)); + expectTopicDoesNotExist(TOPIC); + EasyMock.expect(producer.send(EasyMock.anyObject(), EasyMock.anyObject())).andReturn(null).times(2); + + PowerMock.replayAll(); + + workerSourceTask.initialize(TASK_CONFIG); + workerSourceTask.execute(); + + // two records were consumed from Kafka + assertSourceMetricValue("source-record-poll-total", 2.0); + // only one was written to the task + assertSourceMetricValue("source-record-write-total", 0.0); + // one record completely failed (converter issues) + assertErrorHandlingMetricValue("total-record-errors", 0.0); + // 2 failures in the transformation, and 1 in the converter + assertErrorHandlingMetricValue("total-record-failures", 4.0); + // one record completely failed (converter issues), and thus was skipped + assertErrorHandlingMetricValue("total-records-skipped", 0.0); + + PowerMock.verifyAll(); + } + + private ConnectorConfig connConfig(Map connProps) { + Map props = new HashMap<>(); + props.put(ConnectorConfig.NAME_CONFIG, "test"); + props.put(ConnectorConfig.CONNECTOR_CLASS_CONFIG, SinkTask.class.getName()); + props.putAll(connProps); + return new ConnectorConfig(plugins, props); + } + + @Test + public void testErrorHandlingInSourceTasksWthBadConverter() throws Exception { + Map reportProps = new HashMap<>(); + reportProps.put(ConnectorConfig.ERRORS_LOG_ENABLE_CONFIG, "true"); + reportProps.put(ConnectorConfig.ERRORS_LOG_INCLUDE_MESSAGES_CONFIG, "true"); + LogReporter reporter = new LogReporter(taskId, connConfig(reportProps), errorHandlingMetrics); + + RetryWithToleranceOperator retryWithToleranceOperator = operator(); + retryWithToleranceOperator.metrics(errorHandlingMetrics); + retryWithToleranceOperator.reporters(singletonList(reporter)); + createSourceTask(initialState, retryWithToleranceOperator, badConverter()); + + // valid json + Schema valSchema = SchemaBuilder.struct().field("val", Schema.INT32_SCHEMA).build(); + Struct struct1 = new Struct(valSchema).put("val", 1234); + SourceRecord record1 = new SourceRecord(emptyMap(), emptyMap(), TOPIC, PARTITION1, valSchema, struct1); + Struct struct2 = new Struct(valSchema).put("val", 6789); + SourceRecord record2 = new SourceRecord(emptyMap(), emptyMap(), TOPIC, PARTITION1, valSchema, struct2); + + EasyMock.expect(workerSourceTask.isStopping()).andReturn(false); + EasyMock.expect(workerSourceTask.isStopping()).andReturn(false); + EasyMock.expect(workerSourceTask.isStopping()).andReturn(true); + + EasyMock.expect(workerSourceTask.commitOffsets()).andReturn(true); + + offsetWriter.offset(EasyMock.anyObject(), EasyMock.anyObject()); + EasyMock.expectLastCall().times(2); + sourceTask.initialize(EasyMock.anyObject()); + EasyMock.expectLastCall(); + + sourceTask.start(EasyMock.anyObject()); + EasyMock.expectLastCall(); + + EasyMock.expect(sourceTask.poll()).andReturn(singletonList(record1)); + EasyMock.expect(sourceTask.poll()).andReturn(singletonList(record2)); + expectTopicDoesNotExist(TOPIC); + EasyMock.expect(producer.send(EasyMock.anyObject(), EasyMock.anyObject())).andReturn(null).times(2); + + PowerMock.replayAll(); + + workerSourceTask.initialize(TASK_CONFIG); + workerSourceTask.execute(); + + // two records were consumed from Kafka + assertSourceMetricValue("source-record-poll-total", 2.0); + // only one was written to the task + assertSourceMetricValue("source-record-write-total", 0.0); + // one record completely failed (converter issues) + assertErrorHandlingMetricValue("total-record-errors", 0.0); + // 2 failures in the transformation, and 1 in the converter + assertErrorHandlingMetricValue("total-record-failures", 8.0); + // one record completely failed (converter issues), and thus was skipped + assertErrorHandlingMetricValue("total-records-skipped", 0.0); + + PowerMock.verifyAll(); + } + + private void assertSinkMetricValue(String name, double expected) { + ConnectMetrics.MetricGroup sinkTaskGroup = workerSinkTask.sinkTaskMetricsGroup().metricGroup(); + double measured = metrics.currentMetricValueAsDouble(sinkTaskGroup, name); + assertEquals(expected, measured, 0.001d); + } + + private void assertSourceMetricValue(String name, double expected) { + ConnectMetrics.MetricGroup sinkTaskGroup = workerSourceTask.sourceTaskMetricsGroup().metricGroup(); + double measured = metrics.currentMetricValueAsDouble(sinkTaskGroup, name); + assertEquals(expected, measured, 0.001d); + } + + private void assertErrorHandlingMetricValue(String name, double expected) { + ConnectMetrics.MetricGroup sinkTaskGroup = errorHandlingMetrics.metricGroup(); + double measured = metrics.currentMetricValueAsDouble(sinkTaskGroup, name); + assertEquals(expected, measured, 0.001d); + } + + private void expectInitializeTask() throws Exception { + consumer.subscribe(EasyMock.eq(singletonList(TOPIC)), EasyMock.capture(rebalanceListener)); + PowerMock.expectLastCall(); + + sinkTask.initialize(EasyMock.capture(sinkTaskContext)); + PowerMock.expectLastCall(); + sinkTask.start(TASK_PROPS); + PowerMock.expectLastCall(); + } + + private void expectTaskGetTopic(boolean anyTimes) { + final Capture connectorCapture = EasyMock.newCapture(); + final Capture topicCapture = EasyMock.newCapture(); + IExpectationSetters expect = EasyMock.expect(statusBackingStore.getTopic( + EasyMock.capture(connectorCapture), + EasyMock.capture(topicCapture))); + if (anyTimes) { + expect.andStubAnswer(() -> new TopicStatus( + topicCapture.getValue(), + new ConnectorTaskId(connectorCapture.getValue(), 0), + Time.SYSTEM.milliseconds())); + } else { + expect.andAnswer(() -> new TopicStatus( + topicCapture.getValue(), + new ConnectorTaskId(connectorCapture.getValue(), 0), + Time.SYSTEM.milliseconds())); + } + if (connectorCapture.hasCaptured() && topicCapture.hasCaptured()) { + assertEquals("job", connectorCapture.getValue()); + assertEquals(TOPIC, topicCapture.getValue()); + } + } + + private void expectClose() { + producer.close(EasyMock.anyObject(Duration.class)); + EasyMock.expectLastCall(); + + admin.close(EasyMock.anyObject(Duration.class)); + EasyMock.expectLastCall(); + } + + private void expectTopicDoesNotExist(String topic) { + if (workerConfig.topicCreationEnable()) { + EasyMock.expect(admin.describeTopics(topic)).andReturn(Collections.emptyMap()); + + Capture newTopicCapture = EasyMock.newCapture(); + EasyMock.expect(admin.createTopic(EasyMock.capture(newTopicCapture))).andReturn(true); + } + } + + private void createSinkTask(TargetState initialState, RetryWithToleranceOperator retryWithToleranceOperator) { + JsonConverter converter = new JsonConverter(); + Map oo = workerConfig.originalsWithPrefix("value.converter."); + oo.put("converter.type", "value"); + oo.put("schemas.enable", "false"); + converter.configure(oo); + + TransformationChain sinkTransforms = new TransformationChain<>(singletonList(new FaultyPassthrough()), retryWithToleranceOperator); + + workerSinkTask = new WorkerSinkTask( + taskId, sinkTask, statusListener, initialState, workerConfig, + ClusterConfigState.EMPTY, metrics, converter, converter, + headerConverter, sinkTransforms, consumer, pluginLoader, time, + retryWithToleranceOperator, statusBackingStore); + } + + private void createSourceTask(TargetState initialState, RetryWithToleranceOperator retryWithToleranceOperator) { + JsonConverter converter = new JsonConverter(); + Map oo = workerConfig.originalsWithPrefix("value.converter."); + oo.put("converter.type", "value"); + oo.put("schemas.enable", "false"); + converter.configure(oo); + + createSourceTask(initialState, retryWithToleranceOperator, converter); + } + + private Converter badConverter() { + FaultyConverter converter = new FaultyConverter(); + Map oo = workerConfig.originalsWithPrefix("value.converter."); + oo.put("converter.type", "value"); + oo.put("schemas.enable", "false"); + converter.configure(oo); + return converter; + } + + private void createSourceTask(TargetState initialState, RetryWithToleranceOperator retryWithToleranceOperator, Converter converter) { + TransformationChain sourceTransforms = new TransformationChain<>(singletonList(new FaultyPassthrough()), retryWithToleranceOperator); + + workerSourceTask = PowerMock.createPartialMock( + WorkerSourceTask.class, new String[]{"commitOffsets", "isStopping"}, + taskId, sourceTask, statusListener, initialState, converter, converter, headerConverter, sourceTransforms, + producer, admin, TopicCreationGroup.configuredGroups(sourceConfig), + offsetReader, offsetWriter, workerConfig, + ClusterConfigState.EMPTY, metrics, pluginLoader, time, retryWithToleranceOperator, + statusBackingStore); + } + + private ConsumerRecords records(ConsumerRecord record) { + return new ConsumerRecords<>(Collections.singletonMap( + new TopicPartition(record.topic(), record.partition()), singletonList(record))); + } + + private abstract static class TestSinkTask extends SinkTask { + } + + static class FaultyConverter extends JsonConverter { + private static final Logger log = LoggerFactory.getLogger(FaultyConverter.class); + private int invocations = 0; + + public byte[] fromConnectData(String topic, Schema schema, Object value) { + if (value == null) { + return super.fromConnectData(topic, schema, null); + } + invocations++; + if (invocations % 3 == 0) { + log.debug("Succeeding record: {} where invocations={}", value, invocations); + return super.fromConnectData(topic, schema, value); + } else { + log.debug("Failing record: {} at invocations={}", value, invocations); + throw new RetriableException("Bad invocations " + invocations + " for mod 3"); + } + } + } + + static class FaultyPassthrough> implements Transformation { + + private static final Logger log = LoggerFactory.getLogger(FaultyPassthrough.class); + + private static final String MOD_CONFIG = "mod"; + private static final int MOD_CONFIG_DEFAULT = 3; + + public static final ConfigDef CONFIG_DEF = new ConfigDef() + .define(MOD_CONFIG, ConfigDef.Type.INT, MOD_CONFIG_DEFAULT, ConfigDef.Importance.MEDIUM, "Pass records without failure only if timestamp % mod == 0"); + + private int mod = MOD_CONFIG_DEFAULT; + + private int invocations = 0; + + @Override + public R apply(R record) { + invocations++; + if (invocations % mod == 0) { + log.debug("Succeeding record: {} where invocations={}", record, invocations); + return record; + } else { + log.debug("Failing record: {} at invocations={}", record, invocations); + throw new RetriableException("Bad invocations " + invocations + " for mod " + mod); + } + } + + @Override + public ConfigDef config() { + return CONFIG_DEF; + } + + @Override + public void close() { + log.info("Shutting down transform"); + } + + @Override + public void configure(Map configs) { + final SimpleConfig config = new SimpleConfig(CONFIG_DEF, configs); + mod = Math.max(config.getInt(MOD_CONFIG), 2); + log.info("Configuring {}. Setting mod to {}", this.getClass(), mod); + } + } +} diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/SourceConnectorConfigTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/SourceConnectorConfigTest.java new file mode 100644 index 0000000000000..1972b62e81113 --- /dev/null +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/SourceConnectorConfigTest.java @@ -0,0 +1,150 @@ +/* + * 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.runtime; + +import org.apache.kafka.common.config.ConfigException; +import org.junit.Test; + +import java.util.HashMap; +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; + +import static org.apache.kafka.common.config.TopicConfig.CLEANUP_POLICY_COMPACT; +import static org.apache.kafka.common.config.TopicConfig.CLEANUP_POLICY_CONFIG; +import static org.apache.kafka.common.config.TopicConfig.COMPRESSION_TYPE_CONFIG; +import static org.apache.kafka.common.config.TopicConfig.RETENTION_MS_CONFIG; +import static org.apache.kafka.connect.runtime.ConnectorConfig.CONNECTOR_CLASS_CONFIG; +import static org.apache.kafka.connect.runtime.ConnectorConfig.NAME_CONFIG; +import static org.apache.kafka.connect.runtime.ConnectorConfigTest.MOCK_PLUGINS; +import static org.apache.kafka.connect.runtime.TopicCreationConfig.DEFAULT_TOPIC_CREATION_GROUP; +import static org.apache.kafka.connect.runtime.TopicCreationConfig.DEFAULT_TOPIC_CREATION_PREFIX; +import static org.apache.kafka.connect.runtime.TopicCreationConfig.PARTITIONS_CONFIG; +import static org.apache.kafka.connect.runtime.TopicCreationConfig.REPLICATION_FACTOR_CONFIG; +import static org.hamcrest.CoreMatchers.containsString; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertThrows; +import static org.junit.Assert.assertTrue; + +public class SourceConnectorConfigTest { + + private static final String FOO_CONNECTOR = "foo-source"; + private static final short DEFAULT_REPLICATION_FACTOR = -1; + private static final int DEFAULT_PARTITIONS = -1; + + public Map defaultConnectorProps() { + Map props = new HashMap<>(); + props.put(NAME_CONFIG, FOO_CONNECTOR); + props.put(CONNECTOR_CLASS_CONFIG, ConnectorConfigTest.TestConnector.class.getName()); + return props; + } + + public Map defaultConnectorPropsWithTopicCreation() { + Map props = defaultConnectorProps(); + props.put(DEFAULT_TOPIC_CREATION_PREFIX + REPLICATION_FACTOR_CONFIG, String.valueOf(DEFAULT_REPLICATION_FACTOR)); + props.put(DEFAULT_TOPIC_CREATION_PREFIX + PARTITIONS_CONFIG, String.valueOf(DEFAULT_PARTITIONS)); + return props; + } + + @Test + public void noTopicCreation() { + Map props = defaultConnectorProps(); + SourceConnectorConfig config = new SourceConnectorConfig(MOCK_PLUGINS, props, false); + assertFalse(config.usesTopicCreation()); + } + + @Test + public void shouldNotAllowZeroPartitionsOrReplicationFactor() { + Map props = defaultConnectorPropsWithTopicCreation(); + props.put(DEFAULT_TOPIC_CREATION_PREFIX + PARTITIONS_CONFIG, String.valueOf(0)); + Exception e = assertThrows(ConfigException.class, () -> new SourceConnectorConfig(MOCK_PLUGINS, props, true)); + assertThat(e.getMessage(), containsString("Number of partitions must be positive, or -1")); + + props.put(DEFAULT_TOPIC_CREATION_PREFIX + PARTITIONS_CONFIG, String.valueOf(DEFAULT_PARTITIONS)); + props.put(DEFAULT_TOPIC_CREATION_PREFIX + REPLICATION_FACTOR_CONFIG, String.valueOf(0)); + + e = assertThrows(ConfigException.class, () -> new SourceConnectorConfig(MOCK_PLUGINS, props, true)); + assertThat(e.getMessage(), containsString("Replication factor must be positive and not " + + "larger than the number of brokers in the Kafka cluster, or -1 to use the " + + "broker's default")); + } + + @Test + public void shouldNotAllowPartitionsOrReplicationFactorLessThanNegativeOne() { + Map props = defaultConnectorPropsWithTopicCreation(); + for (int i = -2; i > -100; --i) { + props.put(DEFAULT_TOPIC_CREATION_PREFIX + PARTITIONS_CONFIG, String.valueOf(i)); + props.put(DEFAULT_TOPIC_CREATION_PREFIX + REPLICATION_FACTOR_CONFIG, String.valueOf(DEFAULT_REPLICATION_FACTOR)); + Exception e = assertThrows(ConfigException.class, () -> new SourceConnectorConfig(MOCK_PLUGINS, props, true)); + assertThat(e.getMessage(), containsString("Number of partitions must be positive, or -1")); + + props.put(DEFAULT_TOPIC_CREATION_PREFIX + PARTITIONS_CONFIG, String.valueOf(DEFAULT_PARTITIONS)); + props.put(DEFAULT_TOPIC_CREATION_PREFIX + REPLICATION_FACTOR_CONFIG, String.valueOf(i)); + e = assertThrows(ConfigException.class, () -> new SourceConnectorConfig(MOCK_PLUGINS, props, true)); + assertThat(e.getMessage(), containsString("Replication factor must be positive and not " + + "larger than the number of brokers in the Kafka cluster, or -1 to use the " + + "broker's default")); + } + } + + @Test + public void shouldAllowNegativeOneAndPositiveForReplicationFactor() { + Map props = defaultConnectorPropsWithTopicCreation(); + SourceConnectorConfig config = new SourceConnectorConfig(MOCK_PLUGINS, props, true); + assertTrue(config.usesTopicCreation()); + + for (int i = 1; i <= 100; ++i) { + props.put(DEFAULT_TOPIC_CREATION_PREFIX + PARTITIONS_CONFIG, String.valueOf(i)); + props.put(DEFAULT_TOPIC_CREATION_PREFIX + REPLICATION_FACTOR_CONFIG, String.valueOf(DEFAULT_REPLICATION_FACTOR)); + config = new SourceConnectorConfig(MOCK_PLUGINS, props, true); + assertTrue(config.usesTopicCreation()); + + props.put(DEFAULT_TOPIC_CREATION_PREFIX + PARTITIONS_CONFIG, String.valueOf(DEFAULT_PARTITIONS)); + props.put(DEFAULT_TOPIC_CREATION_PREFIX + REPLICATION_FACTOR_CONFIG, String.valueOf(i)); + config = new SourceConnectorConfig(MOCK_PLUGINS, props, true); + assertTrue(config.usesTopicCreation()); + } + } + + @Test + public void shouldAllowSettingTopicProperties() { + Map topicProps = new HashMap<>(); + topicProps.put(CLEANUP_POLICY_CONFIG, CLEANUP_POLICY_COMPACT); + topicProps.put(COMPRESSION_TYPE_CONFIG, "lz4"); + topicProps.put(RETENTION_MS_CONFIG, String.valueOf(TimeUnit.DAYS.toMillis(30))); + + Map props = defaultConnectorPropsWithTopicCreation(); + topicProps.forEach((k, v) -> props.put(DEFAULT_TOPIC_CREATION_PREFIX + k, v)); + + SourceConnectorConfig config = new SourceConnectorConfig(MOCK_PLUGINS, props, true); + assertEquals(topicProps, + convertToStringValues(config.topicCreationOtherConfigs(DEFAULT_TOPIC_CREATION_GROUP))); + } + + private static Map convertToStringValues(Map config) { + // null values are not allowed + return config.entrySet().stream() + .collect(Collectors.toMap(Map.Entry::getKey, e -> { + Objects.requireNonNull(e.getValue()); + return e.getValue().toString(); + })); + } +} diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java index 68d58a7d2fc34..ff61ea3dc7cff 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java @@ -16,22 +16,23 @@ */ package org.apache.kafka.connect.runtime; -import java.nio.ByteBuffer; +import org.apache.kafka.clients.admin.NewTopic; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.clients.producer.RecordMetadata; +import org.apache.kafka.common.InvalidRecordException; import org.apache.kafka.common.MetricName; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.InvalidRecordException; +import org.apache.kafka.common.errors.TopicAuthorizationException; 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.errors.TopicAuthorizationException; 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.header.ConnectHeaders; import org.apache.kafka.connect.errors.ConnectException; +import org.apache.kafka.connect.header.ConnectHeaders; +import org.apache.kafka.connect.integration.MonitorableSourceConnector; import org.apache.kafka.connect.runtime.ConnectMetrics.MetricGroup; import org.apache.kafka.connect.runtime.WorkerSourceTask.SourceTaskMetricsGroup; import org.apache.kafka.connect.runtime.distributed.ClusterConfigState; @@ -50,6 +51,7 @@ import org.apache.kafka.connect.util.Callback; import org.apache.kafka.connect.util.ConnectorTaskId; import org.apache.kafka.connect.util.ThreadedTest; +import org.apache.kafka.connect.util.TopicAdmin; import org.easymock.Capture; import org.easymock.EasyMock; import org.easymock.IAnswer; @@ -64,6 +66,7 @@ import org.powermock.modules.junit4.PowerMockRunner; import org.powermock.reflect.Whitebox; +import java.nio.ByteBuffer; import java.time.Duration; import java.util.ArrayList; import java.util.Arrays; @@ -79,6 +82,12 @@ import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicInteger; +import static org.apache.kafka.connect.integration.MonitorableSourceConnector.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.apache.kafka.connect.runtime.WorkerConfig.TOPIC_CREATION_ENABLE_CONFIG; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNull; @@ -106,6 +115,7 @@ public class WorkerSourceTaskTest extends ThreadedTest { private ConnectorTaskId taskId = new ConnectorTaskId("job", 0); private ConnectorTaskId taskId1 = new ConnectorTaskId("job", 1); private WorkerConfig config; + private SourceConnectorConfig sourceConfig; private Plugins plugins; private MockConnectMetrics metrics; @Mock private SourceTask sourceTask; @@ -114,6 +124,7 @@ public class WorkerSourceTaskTest extends ThreadedTest { @Mock private HeaderConverter headerConverter; @Mock private TransformationChain transformationChain; @Mock private KafkaProducer producer; + @Mock private TopicAdmin admin; @Mock private CloseableOffsetStorageReader offsetReader; @Mock private OffsetStorageWriter offsetWriter; @Mock private ClusterConfigState clusterConfigState; @@ -134,6 +145,9 @@ public class WorkerSourceTaskTest extends ThreadedTest { new SourceRecord(PARTITION, OFFSET, "topic", null, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD) ); + // when this test becomes parameterized, this variable will be a test parameter + public boolean enableTopicCreation = false; + @Override public void setup() { super.setup(); @@ -145,12 +159,26 @@ public void setup() { workerProps.put("internal.key.converter.schemas.enable", "false"); workerProps.put("internal.value.converter.schemas.enable", "false"); workerProps.put("offset.storage.file.filename", "/tmp/connect.offsets"); + workerProps.put(TOPIC_CREATION_ENABLE_CONFIG, String.valueOf(enableTopicCreation)); plugins = new Plugins(workerProps); config = new StandaloneConfig(workerProps); + sourceConfig = new SourceConnectorConfig(plugins, sourceConnectorProps(TOPIC), true); producerCallbacks = EasyMock.newCapture(); metrics = new MockConnectMetrics(); } + private Map sourceConnectorProps(String topic) { + // setup up props for the source connector + Map props = new HashMap<>(); + props.put("name", "foo-connector"); + props.put(CONNECTOR_CLASS_CONFIG, MonitorableSourceConnector.class.getSimpleName()); + props.put(TASKS_MAX_CONFIG, String.valueOf(1)); + props.put(TOPIC_CONFIG, topic); + props.put(KEY_CONVERTER_CLASS_CONFIG, StringConverter.class.getName()); + props.put(VALUE_CONVERTER_CLASS_CONFIG, StringConverter.class.getName()); + return props; + } + @After public void tearDown() { if (metrics != null) metrics.stop(); @@ -166,7 +194,8 @@ private void createWorkerTask(TargetState initialState) { private void createWorkerTask(TargetState initialState, Converter keyConverter, Converter valueConverter, HeaderConverter headerConverter) { workerTask = new WorkerSourceTask(taskId, sourceTask, statusListener, initialState, keyConverter, valueConverter, headerConverter, - transformationChain, producer, offsetReader, offsetWriter, config, clusterConfigState, metrics, plugins.delegatingLoader(), Time.SYSTEM, + transformationChain, producer, admin, null, + offsetReader, offsetWriter, config, clusterConfigState, metrics, plugins.delegatingLoader(), Time.SYSTEM, RetryWithToleranceOperatorTest.NOOP_OPERATOR, statusBackingStore); } @@ -185,11 +214,7 @@ public Void answer() throws Throwable { } }); - producer.close(EasyMock.anyObject(Duration.class)); - EasyMock.expectLastCall(); - - transformationChain.close(); - EasyMock.expectLastCall(); + expectClose(); statusListener.onShutdown(taskId); EasyMock.expectLastCall(); @@ -223,6 +248,8 @@ public void testPause() throws Exception { CountDownLatch pollLatch = expectPolls(10, count); // In this test, we don't flush, so nothing goes any further than the offset writer + expectTopicCreation(TOPIC); + statusListener.onPause(taskId); EasyMock.expectLastCall(); @@ -233,11 +260,7 @@ public void testPause() throws Exception { statusListener.onShutdown(taskId); EasyMock.expectLastCall(); - producer.close(EasyMock.anyObject(Duration.class)); - EasyMock.expectLastCall(); - - transformationChain.close(); - EasyMock.expectLastCall(); + expectClose(); PowerMock.replayAll(); @@ -275,6 +298,8 @@ public void testPollsInBackground() throws Exception { final CountDownLatch pollLatch = expectPolls(10); // In this test, we don't flush, so nothing goes any further than the offset writer + expectTopicCreation(TOPIC); + sourceTask.stop(); EasyMock.expectLastCall(); expectOffsetFlush(true); @@ -282,11 +307,7 @@ public void testPollsInBackground() throws Exception { statusListener.onShutdown(taskId); EasyMock.expectLastCall(); - producer.close(EasyMock.anyObject(Duration.class)); - EasyMock.expectLastCall(); - - transformationChain.close(); - EasyMock.expectLastCall(); + expectClose(); PowerMock.replayAll(); @@ -331,11 +352,7 @@ public List answer() throws Throwable { EasyMock.expectLastCall(); expectOffsetFlush(true); - producer.close(EasyMock.anyObject(Duration.class)); - EasyMock.expectLastCall(); - - transformationChain.close(); - EasyMock.expectLastCall(); + expectClose(); PowerMock.replayAll(); @@ -375,11 +392,7 @@ public void testPollReturnsNoRecords() throws Exception { statusListener.onShutdown(taskId); EasyMock.expectLastCall(); - producer.close(EasyMock.anyObject(Duration.class)); - EasyMock.expectLastCall(); - - transformationChain.close(); - EasyMock.expectLastCall(); + expectClose(); PowerMock.replayAll(); @@ -413,6 +426,8 @@ public void testCommit() throws Exception { final CountDownLatch pollLatch = expectPolls(1); expectOffsetFlush(true); + expectTopicCreation(TOPIC); + sourceTask.stop(); EasyMock.expectLastCall(); expectOffsetFlush(true); @@ -420,11 +435,7 @@ public void testCommit() throws Exception { statusListener.onShutdown(taskId); EasyMock.expectLastCall(); - producer.close(EasyMock.anyObject(Duration.class)); - EasyMock.expectLastCall(); - - transformationChain.close(); - EasyMock.expectLastCall(); + expectClose(); PowerMock.replayAll(); @@ -458,6 +469,8 @@ public void testCommitFailure() throws Exception { final CountDownLatch pollLatch = expectPolls(1); expectOffsetFlush(true); + expectTopicCreation(TOPIC); + sourceTask.stop(); EasyMock.expectLastCall(); expectOffsetFlush(false); @@ -465,11 +478,7 @@ public void testCommitFailure() throws Exception { statusListener.onShutdown(taskId); EasyMock.expectLastCall(); - producer.close(EasyMock.anyObject(Duration.class)); - EasyMock.expectLastCall(); - - transformationChain.close(); - EasyMock.expectLastCall(); + expectClose(); PowerMock.replayAll(); @@ -497,6 +506,8 @@ public void testSendRecordsConvertsData() throws Exception { Capture> sent = expectSendRecordAnyTimes(); + expectTopicCreation(TOPIC); + PowerMock.replayAll(); Whitebox.setInternalState(workerTask, "toSend", records); @@ -519,6 +530,8 @@ public void testSendRecordsPropagatesTimestamp() throws Exception { Capture> sent = expectSendRecordAnyTimes(); + expectTopicCreation(TOPIC); + PowerMock.replayAll(); Whitebox.setInternalState(workerTask, "toSend", records); @@ -559,6 +572,8 @@ public void testSendRecordsNoTimestamp() throws Exception { Capture> sent = expectSendRecordAnyTimes(); + expectTopicCreation(TOPIC); + PowerMock.replayAll(); Whitebox.setInternalState(workerTask, "toSend", records); @@ -577,6 +592,8 @@ public void testSendRecordsRetries() throws Exception { SourceRecord record2 = new SourceRecord(PARTITION, OFFSET, "topic", 2, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); SourceRecord record3 = new SourceRecord(PARTITION, OFFSET, "topic", 3, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + expectTopicCreation(TOPIC); + // First round expectSendRecordOnce(false); // Any Producer retriable exception should work here @@ -609,6 +626,8 @@ public void testSendRecordsProducerCallbackFail() throws Exception { SourceRecord record1 = new SourceRecord(PARTITION, OFFSET, "topic", 1, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); SourceRecord record2 = new SourceRecord(PARTITION, OFFSET, "topic", 2, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + expectTopicCreation(TOPIC); + expectSendRecordProducerCallbackFail(); PowerMock.replayAll(); @@ -626,6 +645,8 @@ public void testSendRecordsTaskCommitRecordFail() throws Exception { SourceRecord record2 = new SourceRecord(PARTITION, OFFSET, "topic", 2, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); SourceRecord record3 = new SourceRecord(PARTITION, OFFSET, "topic", 3, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + expectTopicCreation(TOPIC); + // Source task commit record failure will not cause the task to abort expectSendRecordOnce(false); expectSendRecordTaskCommitRecordFail(false, false); @@ -670,11 +691,7 @@ public Object answer() throws Throwable { statusListener.onShutdown(taskId); EasyMock.expectLastCall(); - producer.close(EasyMock.anyObject(Duration.class)); - EasyMock.expectLastCall(); - - transformationChain.close(); - EasyMock.expectLastCall(); + expectClose(); PowerMock.replayAll(); @@ -767,6 +784,8 @@ public void testHeaders() throws Exception { List records = new ArrayList<>(); records.add(new SourceRecord(PARTITION, OFFSET, "topic", null, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD, null, connectHeaders)); + expectTopicCreation(TOPIC); + Capture> sent = expectSendRecord(true, false, true, true, true, headers); PowerMock.replayAll(); @@ -803,6 +822,8 @@ public void testHeadersWithCustomConverter() throws Exception { records.add(new SourceRecord(PARTITION, OFFSET, "topic", null, Schema.STRING_SCHEMA, "b", Schema.STRING_SCHEMA, stringB, null, headersB)); + expectTopicCreation(TOPIC); + Capture> sentRecordA = expectSendRecord(false, false, true, true, false, null); Capture> sentRecordB = expectSendRecord(false, false, true, true, false, null); @@ -908,7 +929,6 @@ private Capture> expectSendRecord(boolean anyTime return expectSendRecord(anyTimes, isRetry, succeed, true, true, emptyHeaders()); } - @SuppressWarnings("unchecked") private Capture> expectSendRecord( boolean anyTimes, boolean isRetry, @@ -1118,4 +1138,23 @@ private RecordHeaders emptyHeaders() { private abstract static class TestSourceTask extends SourceTask { } + private void expectClose() { + producer.close(EasyMock.anyObject(Duration.class)); + EasyMock.expectLastCall(); + + admin.close(EasyMock.anyObject(Duration.class)); + EasyMock.expectLastCall(); + + transformationChain.close(); + EasyMock.expectLastCall(); + } + + private void expectTopicCreation(String topic) { + if (config.topicCreationEnable()) { + EasyMock.expect(admin.describeTopics(topic)).andReturn(Collections.emptyMap()); + + Capture newTopicCapture = EasyMock.newCapture(); + EasyMock.expect(admin.createTopic(EasyMock.capture(newTopicCapture))).andReturn(true); + } + } } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskWithTopicCreationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskWithTopicCreationTest.java new file mode 100644 index 0000000000000..06862d13f01b0 --- /dev/null +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskWithTopicCreationTest.java @@ -0,0 +1,1437 @@ +/* + * 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.runtime; + +import org.apache.kafka.clients.admin.NewTopic; +import org.apache.kafka.clients.admin.TopicDescription; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.clients.producer.RecordMetadata; +import org.apache.kafka.common.InvalidRecordException; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.MetricName; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.TopicPartitionInfo; +import org.apache.kafka.common.errors.InvalidTopicException; +import org.apache.kafka.common.errors.TopicAuthorizationException; +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.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.runtime.ConnectMetrics.MetricGroup; +import org.apache.kafka.connect.runtime.WorkerSourceTask.SourceTaskMetricsGroup; +import org.apache.kafka.connect.runtime.distributed.ClusterConfigState; +import org.apache.kafka.connect.runtime.errors.RetryWithToleranceOperatorTest; +import org.apache.kafka.connect.runtime.isolation.Plugins; +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.source.SourceTaskContext; +import org.apache.kafka.connect.storage.CloseableOffsetStorageReader; +import org.apache.kafka.connect.storage.Converter; +import org.apache.kafka.connect.storage.HeaderConverter; +import org.apache.kafka.connect.storage.OffsetStorageWriter; +import org.apache.kafka.connect.storage.StatusBackingStore; +import org.apache.kafka.connect.storage.StringConverter; +import org.apache.kafka.connect.util.Callback; +import org.apache.kafka.connect.util.ConnectorTaskId; +import org.apache.kafka.connect.util.ThreadedTest; +import org.apache.kafka.connect.util.TopicAdmin; +import org.apache.kafka.connect.util.TopicCreationGroup; +import org.easymock.Capture; +import org.easymock.EasyMock; +import org.easymock.IAnswer; +import org.easymock.IExpectationSetters; +import org.junit.After; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.powermock.api.easymock.PowerMock; +import org.powermock.api.easymock.annotation.Mock; +import org.powermock.api.easymock.annotation.MockStrict; +import org.powermock.core.classloader.annotations.PowerMockIgnore; +import org.powermock.modules.junit4.PowerMockRunner; +import org.powermock.reflect.Whitebox; + +import java.nio.ByteBuffer; +import java.time.Duration; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicInteger; + +import static org.apache.kafka.connect.integration.MonitorableSourceConnector.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.apache.kafka.connect.runtime.SourceConnectorConfig.TOPIC_CREATION_GROUPS_CONFIG; +import static org.apache.kafka.connect.runtime.TopicCreationConfig.DEFAULT_TOPIC_CREATION_PREFIX; +import static org.apache.kafka.connect.runtime.TopicCreationConfig.EXCLUDE_REGEX_CONFIG; +import static org.apache.kafka.connect.runtime.TopicCreationConfig.INCLUDE_REGEX_CONFIG; +import static org.apache.kafka.connect.runtime.TopicCreationConfig.PARTITIONS_CONFIG; +import static org.apache.kafka.connect.runtime.TopicCreationConfig.REPLICATION_FACTOR_CONFIG; +import static org.apache.kafka.connect.runtime.WorkerConfig.TOPIC_CREATION_ENABLE_CONFIG; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; + +@PowerMockIgnore({"javax.management.*", + "org.apache.log4j.*"}) +@RunWith(PowerMockRunner.class) +public class WorkerSourceTaskWithTopicCreationTest extends ThreadedTest { + private static final String TOPIC = "topic"; + private static final String OTHER_TOPIC = "other-topic"; + private static final Map PARTITION = Collections.singletonMap("key", "partition".getBytes()); + private static final Map OFFSET = Collections.singletonMap("key", 12); + + // Connect-format data + private static final Schema KEY_SCHEMA = Schema.INT32_SCHEMA; + private static final Integer KEY = -1; + private static final Schema RECORD_SCHEMA = Schema.INT64_SCHEMA; + private static final Long RECORD = 12L; + // Serialized data. The actual format of this data doesn't matter -- we just want to see that the right version + // is used in the right place. + private static final byte[] SERIALIZED_KEY = "converted-key".getBytes(); + private static final byte[] SERIALIZED_RECORD = "converted-record".getBytes(); + + private ExecutorService executor = Executors.newSingleThreadExecutor(); + private ConnectorTaskId taskId = new ConnectorTaskId("job", 0); + private ConnectorTaskId taskId1 = new ConnectorTaskId("job", 1); + private WorkerConfig config; + private SourceConnectorConfig sourceConfig; + private Plugins plugins; + private MockConnectMetrics metrics; + @Mock private SourceTask sourceTask; + @Mock private Converter keyConverter; + @Mock private Converter valueConverter; + @Mock private HeaderConverter headerConverter; + @Mock private TransformationChain transformationChain; + @Mock private KafkaProducer producer; + @Mock private TopicAdmin admin; + @Mock private CloseableOffsetStorageReader offsetReader; + @Mock private OffsetStorageWriter offsetWriter; + @Mock private ClusterConfigState clusterConfigState; + private WorkerSourceTask workerTask; + @Mock private Future sendFuture; + @MockStrict private TaskStatus.Listener statusListener; + @Mock private StatusBackingStore statusBackingStore; + + private Capture producerCallbacks; + + private static final Map TASK_PROPS = new HashMap<>(); + static { + TASK_PROPS.put(TaskConfig.TASK_CLASS_CONFIG, TestSourceTask.class.getName()); + } + private static final TaskConfig TASK_CONFIG = new TaskConfig(TASK_PROPS); + + private static final List RECORDS = Arrays.asList( + new SourceRecord(PARTITION, OFFSET, TOPIC, null, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD) + ); + + // when this test becomes parameterized, this variable will be a test parameter + public boolean enableTopicCreation = true; + + @Override + public void setup() { + super.setup(); + Map workerProps = workerProps(); + plugins = new Plugins(workerProps); + config = new StandaloneConfig(workerProps); + sourceConfig = new SourceConnectorConfig(plugins, sourceConnectorPropsWithGroups(TOPIC), true); + producerCallbacks = EasyMock.newCapture(); + metrics = new MockConnectMetrics(); + } + + private Map workerProps() { + Map props = new HashMap<>(); + props.put("key.converter", "org.apache.kafka.connect.json.JsonConverter"); + props.put("value.converter", "org.apache.kafka.connect.json.JsonConverter"); + props.put("internal.key.converter", "org.apache.kafka.connect.json.JsonConverter"); + props.put("internal.value.converter", "org.apache.kafka.connect.json.JsonConverter"); + props.put("internal.key.converter.schemas.enable", "false"); + props.put("internal.value.converter.schemas.enable", "false"); + props.put("offset.storage.file.filename", "/tmp/connect.offsets"); + props.put(TOPIC_CREATION_ENABLE_CONFIG, String.valueOf(enableTopicCreation)); + return props; + } + + private Map sourceConnectorPropsWithGroups(String topic) { + // setup up props for the source connector + Map props = new HashMap<>(); + props.put("name", "foo-connector"); + props.put(CONNECTOR_CLASS_CONFIG, MonitorableSourceConnector.class.getSimpleName()); + props.put(TASKS_MAX_CONFIG, String.valueOf(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(TOPIC_CREATION_GROUPS_CONFIG, String.join(",", "foo", "bar")); + props.put(DEFAULT_TOPIC_CREATION_PREFIX + REPLICATION_FACTOR_CONFIG, String.valueOf(1)); + props.put(DEFAULT_TOPIC_CREATION_PREFIX + PARTITIONS_CONFIG, String.valueOf(1)); + props.put(SourceConnectorConfig.TOPIC_CREATION_PREFIX + "foo" + "." + INCLUDE_REGEX_CONFIG, topic); + props.put(SourceConnectorConfig.TOPIC_CREATION_PREFIX + "bar" + "." + INCLUDE_REGEX_CONFIG, ".*"); + props.put(SourceConnectorConfig.TOPIC_CREATION_PREFIX + "bar" + "." + EXCLUDE_REGEX_CONFIG, topic); + return props; + } + + @After + public void tearDown() { + if (metrics != null) metrics.stop(); + } + + private void createWorkerTask() { + createWorkerTask(TargetState.STARTED); + } + + private void createWorkerTask(TargetState initialState) { + createWorkerTask(initialState, keyConverter, valueConverter, headerConverter); + } + + private void createWorkerTask(TargetState initialState, Converter keyConverter, Converter valueConverter, HeaderConverter headerConverter) { + workerTask = new WorkerSourceTask(taskId, sourceTask, statusListener, initialState, keyConverter, valueConverter, headerConverter, + transformationChain, producer, admin, TopicCreationGroup.configuredGroups(sourceConfig), + offsetReader, offsetWriter, config, clusterConfigState, metrics, plugins.delegatingLoader(), Time.SYSTEM, + RetryWithToleranceOperatorTest.NOOP_OPERATOR, statusBackingStore); + } + + @Test + public void testStartPaused() throws Exception { + final CountDownLatch pauseLatch = new CountDownLatch(1); + + createWorkerTask(TargetState.PAUSED); + + statusListener.onPause(taskId); + EasyMock.expectLastCall().andAnswer(new IAnswer() { + @Override + public Void answer() throws Throwable { + pauseLatch.countDown(); + return null; + } + }); + + expectClose(); + + statusListener.onShutdown(taskId); + EasyMock.expectLastCall(); + + PowerMock.replayAll(); + + workerTask.initialize(TASK_CONFIG); + Future taskFuture = executor.submit(workerTask); + + assertTrue(pauseLatch.await(5, TimeUnit.SECONDS)); + workerTask.stop(); + assertTrue(workerTask.awaitStop(1000)); + + taskFuture.get(); + + PowerMock.verifyAll(); + } + + @Test + public void testPause() throws Exception { + createWorkerTask(); + + sourceTask.initialize(EasyMock.anyObject(SourceTaskContext.class)); + EasyMock.expectLastCall(); + sourceTask.start(TASK_PROPS); + EasyMock.expectLastCall(); + statusListener.onStartup(taskId); + EasyMock.expectLastCall(); + + AtomicInteger count = new AtomicInteger(0); + CountDownLatch pollLatch = expectPolls(10, count); + // In this test, we don't flush, so nothing goes any further than the offset writer + + expectTopicCreation(TOPIC); + + statusListener.onPause(taskId); + EasyMock.expectLastCall(); + + sourceTask.stop(); + EasyMock.expectLastCall(); + expectOffsetFlush(true); + + statusListener.onShutdown(taskId); + EasyMock.expectLastCall(); + + expectClose(); + + PowerMock.replayAll(); + + workerTask.initialize(TASK_CONFIG); + Future taskFuture = executor.submit(workerTask); + assertTrue(awaitLatch(pollLatch)); + + workerTask.transitionTo(TargetState.PAUSED); + + int priorCount = count.get(); + Thread.sleep(100); + + // since the transition is observed asynchronously, the count could be off by one loop iteration + assertTrue(count.get() - priorCount <= 1); + + workerTask.stop(); + assertTrue(workerTask.awaitStop(1000)); + + taskFuture.get(); + + PowerMock.verifyAll(); + } + + @Test + public void testPollsInBackground() throws Exception { + createWorkerTask(); + + sourceTask.initialize(EasyMock.anyObject(SourceTaskContext.class)); + EasyMock.expectLastCall(); + sourceTask.start(TASK_PROPS); + EasyMock.expectLastCall(); + statusListener.onStartup(taskId); + EasyMock.expectLastCall(); + + final CountDownLatch pollLatch = expectPolls(10); + // In this test, we don't flush, so nothing goes any further than the offset writer + + expectTopicCreation(TOPIC); + + sourceTask.stop(); + EasyMock.expectLastCall(); + expectOffsetFlush(true); + + statusListener.onShutdown(taskId); + EasyMock.expectLastCall(); + + expectClose(); + + PowerMock.replayAll(); + + workerTask.initialize(TASK_CONFIG); + Future taskFuture = executor.submit(workerTask); + + assertTrue(awaitLatch(pollLatch)); + workerTask.stop(); + assertTrue(workerTask.awaitStop(1000)); + + taskFuture.get(); + assertPollMetrics(10); + + PowerMock.verifyAll(); + } + + @Test + public void testFailureInPoll() throws Exception { + createWorkerTask(); + + sourceTask.initialize(EasyMock.anyObject(SourceTaskContext.class)); + EasyMock.expectLastCall(); + sourceTask.start(TASK_PROPS); + EasyMock.expectLastCall(); + statusListener.onStartup(taskId); + EasyMock.expectLastCall(); + + final CountDownLatch pollLatch = new CountDownLatch(1); + final RuntimeException exception = new RuntimeException(); + EasyMock.expect(sourceTask.poll()).andAnswer(new IAnswer>() { + @Override + public List answer() throws Throwable { + pollLatch.countDown(); + throw exception; + } + }); + + statusListener.onFailure(taskId, exception); + EasyMock.expectLastCall(); + + sourceTask.stop(); + EasyMock.expectLastCall(); + expectOffsetFlush(true); + + expectClose(); + + PowerMock.replayAll(); + + workerTask.initialize(TASK_CONFIG); + Future taskFuture = executor.submit(workerTask); + + assertTrue(awaitLatch(pollLatch)); + workerTask.stop(); + assertTrue(workerTask.awaitStop(1000)); + + taskFuture.get(); + assertPollMetrics(0); + + PowerMock.verifyAll(); + } + + @Test + public void testPollReturnsNoRecords() throws Exception { + // Test that the task handles an empty list of records + createWorkerTask(); + + sourceTask.initialize(EasyMock.anyObject(SourceTaskContext.class)); + EasyMock.expectLastCall(); + sourceTask.start(TASK_PROPS); + EasyMock.expectLastCall(); + statusListener.onStartup(taskId); + EasyMock.expectLastCall(); + + // We'll wait for some data, then trigger a flush + final CountDownLatch pollLatch = expectEmptyPolls(1, new AtomicInteger()); + expectOffsetFlush(true); + + sourceTask.stop(); + EasyMock.expectLastCall(); + expectOffsetFlush(true); + + statusListener.onShutdown(taskId); + EasyMock.expectLastCall(); + + expectClose(); + + PowerMock.replayAll(); + + workerTask.initialize(TASK_CONFIG); + Future taskFuture = executor.submit(workerTask); + + assertTrue(awaitLatch(pollLatch)); + assertTrue(workerTask.commitOffsets()); + workerTask.stop(); + assertTrue(workerTask.awaitStop(1000)); + + taskFuture.get(); + assertPollMetrics(0); + + PowerMock.verifyAll(); + } + + @Test + public void testCommit() throws Exception { + // Test that the task commits properly when prompted + createWorkerTask(); + + sourceTask.initialize(EasyMock.anyObject(SourceTaskContext.class)); + EasyMock.expectLastCall(); + sourceTask.start(TASK_PROPS); + EasyMock.expectLastCall(); + statusListener.onStartup(taskId); + EasyMock.expectLastCall(); + + // We'll wait for some data, then trigger a flush + final CountDownLatch pollLatch = expectPolls(1); + expectOffsetFlush(true); + + expectTopicCreation(TOPIC); + + sourceTask.stop(); + EasyMock.expectLastCall(); + expectOffsetFlush(true); + + statusListener.onShutdown(taskId); + EasyMock.expectLastCall(); + + expectClose(); + + PowerMock.replayAll(); + + workerTask.initialize(TASK_CONFIG); + Future taskFuture = executor.submit(workerTask); + + assertTrue(awaitLatch(pollLatch)); + assertTrue(workerTask.commitOffsets()); + workerTask.stop(); + assertTrue(workerTask.awaitStop(1000)); + + taskFuture.get(); + assertPollMetrics(1); + + PowerMock.verifyAll(); + } + + @Test + public void testCommitFailure() throws Exception { + // Test that the task commits properly when prompted + createWorkerTask(); + + sourceTask.initialize(EasyMock.anyObject(SourceTaskContext.class)); + EasyMock.expectLastCall(); + sourceTask.start(TASK_PROPS); + EasyMock.expectLastCall(); + statusListener.onStartup(taskId); + EasyMock.expectLastCall(); + + // We'll wait for some data, then trigger a flush + final CountDownLatch pollLatch = expectPolls(1); + expectOffsetFlush(true); + + expectTopicCreation(TOPIC); + + sourceTask.stop(); + EasyMock.expectLastCall(); + expectOffsetFlush(false); + + statusListener.onShutdown(taskId); + EasyMock.expectLastCall(); + + expectClose(); + + PowerMock.replayAll(); + + workerTask.initialize(TASK_CONFIG); + Future taskFuture = executor.submit(workerTask); + + assertTrue(awaitLatch(pollLatch)); + assertTrue(workerTask.commitOffsets()); + workerTask.stop(); + assertTrue(workerTask.awaitStop(1000)); + + taskFuture.get(); + assertPollMetrics(1); + + PowerMock.verifyAll(); + } + + @Test + public void testSendRecordsConvertsData() throws Exception { + createWorkerTask(); + + List records = new ArrayList<>(); + // Can just use the same record for key and value + records.add(new SourceRecord(PARTITION, OFFSET, TOPIC, null, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD)); + + Capture> sent = expectSendRecordAnyTimes(); + + expectTopicCreation(TOPIC); + + PowerMock.replayAll(); + + Whitebox.setInternalState(workerTask, "toSend", records); + Whitebox.invokeMethod(workerTask, "sendRecords"); + assertEquals(SERIALIZED_KEY, sent.getValue().key()); + assertEquals(SERIALIZED_RECORD, sent.getValue().value()); + + PowerMock.verifyAll(); + } + + @Test + public void testSendRecordsPropagatesTimestamp() throws Exception { + final Long timestamp = System.currentTimeMillis(); + + createWorkerTask(); + + List records = Collections.singletonList( + new SourceRecord(PARTITION, OFFSET, TOPIC, null, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD, timestamp) + ); + + Capture> sent = expectSendRecordAnyTimes(); + + expectTopicCreation(TOPIC); + + PowerMock.replayAll(); + + Whitebox.setInternalState(workerTask, "toSend", records); + Whitebox.invokeMethod(workerTask, "sendRecords"); + assertEquals(timestamp, sent.getValue().timestamp()); + + PowerMock.verifyAll(); + } + + @Test(expected = InvalidRecordException.class) + public void testSendRecordsCorruptTimestamp() throws Exception { + final Long timestamp = -3L; + createWorkerTask(); + + List records = Collections.singletonList( + new SourceRecord(PARTITION, OFFSET, TOPIC, null, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD, timestamp) + ); + + Capture> sent = expectSendRecordAnyTimes(); + + PowerMock.replayAll(); + + Whitebox.setInternalState(workerTask, "toSend", records); + Whitebox.invokeMethod(workerTask, "sendRecords"); + assertEquals(null, sent.getValue().timestamp()); + + PowerMock.verifyAll(); + } + + @Test + public void testSendRecordsNoTimestamp() throws Exception { + final Long timestamp = -1L; + createWorkerTask(); + + List records = Collections.singletonList( + new SourceRecord(PARTITION, OFFSET, TOPIC, null, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD, timestamp) + ); + + Capture> sent = expectSendRecordAnyTimes(); + + expectTopicCreation(TOPIC); + + PowerMock.replayAll(); + + Whitebox.setInternalState(workerTask, "toSend", records); + Whitebox.invokeMethod(workerTask, "sendRecords"); + assertEquals(null, sent.getValue().timestamp()); + + PowerMock.verifyAll(); + } + + @Test + public void testSendRecordsRetries() throws Exception { + createWorkerTask(); + + // Differentiate only by Kafka partition so we can reuse conversion expectations + SourceRecord record1 = new SourceRecord(PARTITION, OFFSET, TOPIC, 1, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + SourceRecord record2 = new SourceRecord(PARTITION, OFFSET, TOPIC, 2, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + SourceRecord record3 = new SourceRecord(PARTITION, OFFSET, TOPIC, 3, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + + expectTopicCreation(TOPIC); + + // First round + expectSendRecordOnce(false); + // Any Producer retriable exception should work here + expectSendRecordSyncFailure(new org.apache.kafka.common.errors.TimeoutException("retriable sync failure")); + + // Second round + expectSendRecordOnce(true); + expectSendRecordOnce(false); + + PowerMock.replayAll(); + + // Try to send 3, make first pass, second fail. Should save last two + Whitebox.setInternalState(workerTask, "toSend", Arrays.asList(record1, record2, record3)); + Whitebox.invokeMethod(workerTask, "sendRecords"); + assertEquals(true, Whitebox.getInternalState(workerTask, "lastSendFailed")); + assertEquals(Arrays.asList(record2, record3), Whitebox.getInternalState(workerTask, "toSend")); + + // Next they all succeed + Whitebox.invokeMethod(workerTask, "sendRecords"); + assertEquals(false, Whitebox.getInternalState(workerTask, "lastSendFailed")); + assertNull(Whitebox.getInternalState(workerTask, "toSend")); + + PowerMock.verifyAll(); + } + + @Test(expected = ConnectException.class) + public void testSendRecordsProducerCallbackFail() throws Exception { + createWorkerTask(); + + SourceRecord record1 = new SourceRecord(PARTITION, OFFSET, TOPIC, 1, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + SourceRecord record2 = new SourceRecord(PARTITION, OFFSET, TOPIC, 2, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + + expectTopicCreation(TOPIC); + + expectSendRecordProducerCallbackFail(); + + PowerMock.replayAll(); + + Whitebox.setInternalState(workerTask, "toSend", Arrays.asList(record1, record2)); + Whitebox.invokeMethod(workerTask, "sendRecords"); + } + + @Test(expected = ConnectException.class) + public void testSendRecordsProducerSendFailsImmediately() throws Exception { + createWorkerTask(); + + SourceRecord record1 = new SourceRecord(PARTITION, OFFSET, TOPIC, 1, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + SourceRecord record2 = new SourceRecord(PARTITION, OFFSET, TOPIC, 2, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + + expectPreliminaryCalls(); + expectTopicCreation(TOPIC); + + EasyMock.expect(producer.send(EasyMock.anyObject(), EasyMock.anyObject())) + .andThrow(new KafkaException("Producer closed while send in progress", new InvalidTopicException(TOPIC))); + + PowerMock.replayAll(); + + Whitebox.setInternalState(workerTask, "toSend", Arrays.asList(record1, record2)); + Whitebox.invokeMethod(workerTask, "sendRecords"); + } + + @Test + public void testSendRecordsTaskCommitRecordFail() throws Exception { + createWorkerTask(); + + // Differentiate only by Kafka partition so we can reuse conversion expectations + SourceRecord record1 = new SourceRecord(PARTITION, OFFSET, TOPIC, 1, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + SourceRecord record2 = new SourceRecord(PARTITION, OFFSET, TOPIC, 2, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + SourceRecord record3 = new SourceRecord(PARTITION, OFFSET, TOPIC, 3, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + + expectTopicCreation(TOPIC); + + // Source task commit record failure will not cause the task to abort + expectSendRecordOnce(false); + expectSendRecordTaskCommitRecordFail(false, false); + expectSendRecordOnce(false); + + PowerMock.replayAll(); + + Whitebox.setInternalState(workerTask, "toSend", Arrays.asList(record1, record2, record3)); + Whitebox.invokeMethod(workerTask, "sendRecords"); + assertEquals(false, Whitebox.getInternalState(workerTask, "lastSendFailed")); + assertNull(Whitebox.getInternalState(workerTask, "toSend")); + + PowerMock.verifyAll(); + } + + @Test + public void testSlowTaskStart() throws Exception { + final CountDownLatch startupLatch = new CountDownLatch(1); + final CountDownLatch finishStartupLatch = new CountDownLatch(1); + + createWorkerTask(); + + sourceTask.initialize(EasyMock.anyObject(SourceTaskContext.class)); + EasyMock.expectLastCall(); + sourceTask.start(TASK_PROPS); + EasyMock.expectLastCall().andAnswer(new IAnswer() { + @Override + public Object answer() throws Throwable { + startupLatch.countDown(); + assertTrue(awaitLatch(finishStartupLatch)); + return null; + } + }); + + statusListener.onStartup(taskId); + EasyMock.expectLastCall(); + + sourceTask.stop(); + EasyMock.expectLastCall(); + expectOffsetFlush(true); + + statusListener.onShutdown(taskId); + EasyMock.expectLastCall(); + + expectClose(); + + PowerMock.replayAll(); + + workerTask.initialize(TASK_CONFIG); + Future workerTaskFuture = executor.submit(workerTask); + + // Stopping immediately while the other thread has work to do should result in no polling, no offset commits, + // exiting the work thread immediately, and the stop() method will be invoked in the background thread since it + // cannot be invoked immediately in the thread trying to stop the task. + assertTrue(awaitLatch(startupLatch)); + workerTask.stop(); + finishStartupLatch.countDown(); + assertTrue(workerTask.awaitStop(1000)); + + workerTaskFuture.get(); + + PowerMock.verifyAll(); + } + + @Test + public void testCancel() { + createWorkerTask(); + + offsetReader.close(); + PowerMock.expectLastCall(); + + PowerMock.replayAll(); + + workerTask.cancel(); + + PowerMock.verifyAll(); + } + + @Test + public void testMetricsGroup() { + SourceTaskMetricsGroup group = new SourceTaskMetricsGroup(taskId, metrics); + SourceTaskMetricsGroup group1 = new SourceTaskMetricsGroup(taskId1, metrics); + for (int i = 0; i != 10; ++i) { + group.recordPoll(100, 1000 + i * 100); + group.recordWrite(10); + } + for (int i = 0; i != 20; ++i) { + group1.recordPoll(100, 1000 + i * 100); + group1.recordWrite(10); + } + assertEquals(1900.0, metrics.currentMetricValueAsDouble(group.metricGroup(), "poll-batch-max-time-ms"), 0.001d); + assertEquals(1450.0, metrics.currentMetricValueAsDouble(group.metricGroup(), "poll-batch-avg-time-ms"), 0.001d); + assertEquals(33.333, metrics.currentMetricValueAsDouble(group.metricGroup(), "source-record-poll-rate"), 0.001d); + assertEquals(1000, metrics.currentMetricValueAsDouble(group.metricGroup(), "source-record-poll-total"), 0.001d); + assertEquals(3.3333, metrics.currentMetricValueAsDouble(group.metricGroup(), "source-record-write-rate"), 0.001d); + assertEquals(100, metrics.currentMetricValueAsDouble(group.metricGroup(), "source-record-write-total"), 0.001d); + assertEquals(900.0, metrics.currentMetricValueAsDouble(group.metricGroup(), "source-record-active-count"), 0.001d); + + // Close the group + group.close(); + + for (MetricName metricName : group.metricGroup().metrics().metrics().keySet()) { + // Metrics for this group should no longer exist + assertFalse(group.metricGroup().groupId().includes(metricName)); + } + // Sensors for this group should no longer exist + assertNull(group.metricGroup().metrics().getSensor("sink-record-read")); + assertNull(group.metricGroup().metrics().getSensor("sink-record-send")); + assertNull(group.metricGroup().metrics().getSensor("sink-record-active-count")); + assertNull(group.metricGroup().metrics().getSensor("partition-count")); + assertNull(group.metricGroup().metrics().getSensor("offset-seq-number")); + assertNull(group.metricGroup().metrics().getSensor("offset-commit-completion")); + assertNull(group.metricGroup().metrics().getSensor("offset-commit-completion-skip")); + assertNull(group.metricGroup().metrics().getSensor("put-batch-time")); + + assertEquals(2900.0, metrics.currentMetricValueAsDouble(group1.metricGroup(), "poll-batch-max-time-ms"), 0.001d); + assertEquals(1950.0, metrics.currentMetricValueAsDouble(group1.metricGroup(), "poll-batch-avg-time-ms"), 0.001d); + assertEquals(66.667, metrics.currentMetricValueAsDouble(group1.metricGroup(), "source-record-poll-rate"), 0.001d); + assertEquals(2000, metrics.currentMetricValueAsDouble(group1.metricGroup(), "source-record-poll-total"), 0.001d); + assertEquals(6.667, metrics.currentMetricValueAsDouble(group1.metricGroup(), "source-record-write-rate"), 0.001d); + assertEquals(200, metrics.currentMetricValueAsDouble(group1.metricGroup(), "source-record-write-total"), 0.001d); + assertEquals(1800.0, metrics.currentMetricValueAsDouble(group1.metricGroup(), "source-record-active-count"), 0.001d); + } + + @Test + public void testHeaders() throws Exception { + Headers headers = new RecordHeaders(); + headers.add("header_key", "header_value".getBytes()); + + org.apache.kafka.connect.header.Headers connectHeaders = new ConnectHeaders(); + connectHeaders.add("header_key", new SchemaAndValue(Schema.STRING_SCHEMA, "header_value")); + + createWorkerTask(); + + List records = new ArrayList<>(); + records.add(new SourceRecord(PARTITION, OFFSET, TOPIC, null, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD, null, connectHeaders)); + + expectTopicCreation(TOPIC); + + Capture> sent = expectSendRecord(TOPIC, true, false, true, true, true, headers); + + PowerMock.replayAll(); + + Whitebox.setInternalState(workerTask, "toSend", records); + Whitebox.invokeMethod(workerTask, "sendRecords"); + assertEquals(SERIALIZED_KEY, sent.getValue().key()); + assertEquals(SERIALIZED_RECORD, sent.getValue().value()); + assertEquals(headers, sent.getValue().headers()); + + PowerMock.verifyAll(); + } + + @Test + public void testHeadersWithCustomConverter() throws Exception { + StringConverter stringConverter = new StringConverter(); + TestConverterWithHeaders testConverter = new TestConverterWithHeaders(); + + createWorkerTask(TargetState.STARTED, stringConverter, testConverter, stringConverter); + + List records = new ArrayList<>(); + + String stringA = "Árvíztűrő tükörfúrógép"; + org.apache.kafka.connect.header.Headers headersA = new ConnectHeaders(); + String encodingA = "latin2"; + headersA.addString("encoding", encodingA); + + records.add(new SourceRecord(PARTITION, OFFSET, TOPIC, null, Schema.STRING_SCHEMA, "a", Schema.STRING_SCHEMA, stringA, null, headersA)); + + String stringB = "Тестовое сообщение"; + org.apache.kafka.connect.header.Headers headersB = new ConnectHeaders(); + String encodingB = "koi8_r"; + headersB.addString("encoding", encodingB); + + records.add(new SourceRecord(PARTITION, OFFSET, TOPIC, null, Schema.STRING_SCHEMA, "b", Schema.STRING_SCHEMA, stringB, null, headersB)); + + expectTopicCreation(TOPIC); + + Capture> sentRecordA = expectSendRecord(TOPIC, false, false, true, true, false, null); + Capture> sentRecordB = expectSendRecord(TOPIC, false, false, true, true, false, null); + + PowerMock.replayAll(); + + Whitebox.setInternalState(workerTask, "toSend", records); + Whitebox.invokeMethod(workerTask, "sendRecords"); + + assertEquals(ByteBuffer.wrap("a".getBytes()), ByteBuffer.wrap(sentRecordA.getValue().key())); + assertEquals( + ByteBuffer.wrap(stringA.getBytes(encodingA)), + ByteBuffer.wrap(sentRecordA.getValue().value()) + ); + assertEquals(encodingA, new String(sentRecordA.getValue().headers().lastHeader("encoding").value())); + + assertEquals(ByteBuffer.wrap("b".getBytes()), ByteBuffer.wrap(sentRecordB.getValue().key())); + assertEquals( + ByteBuffer.wrap(stringB.getBytes(encodingB)), + ByteBuffer.wrap(sentRecordB.getValue().value()) + ); + assertEquals(encodingB, new String(sentRecordB.getValue().headers().lastHeader("encoding").value())); + + PowerMock.verifyAll(); + } + + @Test + public void testTopicCreateWhenTopicExists() throws Exception { + createWorkerTask(); + + SourceRecord record1 = new SourceRecord(PARTITION, OFFSET, TOPIC, 1, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + SourceRecord record2 = new SourceRecord(PARTITION, OFFSET, TOPIC, 2, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + + expectPreliminaryCalls(); + TopicPartitionInfo topicPartitionInfo = new TopicPartitionInfo(0, null, Collections.emptyList(), Collections.emptyList()); + TopicDescription topicDesc = new TopicDescription(TOPIC, false, Collections.singletonList(topicPartitionInfo)); + EasyMock.expect(admin.describeTopics(TOPIC)).andReturn(Collections.singletonMap(TOPIC, topicDesc)); + + expectSendRecordTaskCommitRecordSucceed(false, false); + expectSendRecordTaskCommitRecordSucceed(false, false); + + PowerMock.replayAll(); + + Whitebox.setInternalState(workerTask, "toSend", Arrays.asList(record1, record2)); + Whitebox.invokeMethod(workerTask, "sendRecords"); + } + + @Test + public void testSendRecordsTopicDescribeRetries() throws Exception { + createWorkerTask(); + + SourceRecord record1 = new SourceRecord(PARTITION, OFFSET, TOPIC, 1, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + SourceRecord record2 = new SourceRecord(PARTITION, OFFSET, TOPIC, 2, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + + expectPreliminaryCalls(); + // First round - call to describe the topic times out + EasyMock.expect(admin.describeTopics(TOPIC)) + .andThrow(new RetriableException(new TimeoutException("timeout"))); + + // Second round - calls to describe and create succeed + expectTopicCreation(TOPIC); + // Exactly two records are sent + expectSendRecordTaskCommitRecordSucceed(false, false); + expectSendRecordTaskCommitRecordSucceed(false, false); + + PowerMock.replayAll(); + + Whitebox.setInternalState(workerTask, "toSend", Arrays.asList(record1, record2)); + Whitebox.invokeMethod(workerTask, "sendRecords"); + assertEquals(true, Whitebox.getInternalState(workerTask, "lastSendFailed")); + assertEquals(Arrays.asList(record1, record2), Whitebox.getInternalState(workerTask, "toSend")); + + // Next they all succeed + Whitebox.invokeMethod(workerTask, "sendRecords"); + assertEquals(false, Whitebox.getInternalState(workerTask, "lastSendFailed")); + assertNull(Whitebox.getInternalState(workerTask, "toSend")); + } + + @Test + public void testSendRecordsTopicCreateRetries() throws Exception { + createWorkerTask(); + + SourceRecord record1 = new SourceRecord(PARTITION, OFFSET, TOPIC, 1, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + SourceRecord record2 = new SourceRecord(PARTITION, OFFSET, TOPIC, 2, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + + // First call to describe the topic times out + expectPreliminaryCalls(); + EasyMock.expect(admin.describeTopics(TOPIC)).andReturn(Collections.emptyMap()); + Capture newTopicCapture = EasyMock.newCapture(); + EasyMock.expect(admin.createTopic(EasyMock.capture(newTopicCapture))) + .andThrow(new RetriableException(new TimeoutException("timeout"))); + + // Second round + expectTopicCreation(TOPIC); + expectSendRecordTaskCommitRecordSucceed(false, false); + expectSendRecordTaskCommitRecordSucceed(false, false); + + PowerMock.replayAll(); + + Whitebox.setInternalState(workerTask, "toSend", Arrays.asList(record1, record2)); + Whitebox.invokeMethod(workerTask, "sendRecords"); + assertEquals(true, Whitebox.getInternalState(workerTask, "lastSendFailed")); + assertEquals(Arrays.asList(record1, record2), Whitebox.getInternalState(workerTask, "toSend")); + + // Next they all succeed + Whitebox.invokeMethod(workerTask, "sendRecords"); + assertEquals(false, Whitebox.getInternalState(workerTask, "lastSendFailed")); + assertNull(Whitebox.getInternalState(workerTask, "toSend")); + } + + @Test + public void testSendRecordsTopicDescribeRetriesMidway() throws Exception { + createWorkerTask(); + + // Differentiate only by Kafka partition so we can reuse conversion expectations + SourceRecord record1 = new SourceRecord(PARTITION, OFFSET, TOPIC, 1, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + SourceRecord record2 = new SourceRecord(PARTITION, OFFSET, TOPIC, 2, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + SourceRecord record3 = new SourceRecord(PARTITION, OFFSET, OTHER_TOPIC, 3, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + + // First round + expectPreliminaryCalls(OTHER_TOPIC); + expectTopicCreation(TOPIC); + expectSendRecordTaskCommitRecordSucceed(false, false); + expectSendRecordTaskCommitRecordSucceed(false, false); + + // First call to describe the topic times out + EasyMock.expect(admin.describeTopics(OTHER_TOPIC)) + .andThrow(new RetriableException(new TimeoutException("timeout"))); + + // Second round + expectTopicCreation(OTHER_TOPIC); + expectSendRecord(OTHER_TOPIC, false, true, true, true, true, emptyHeaders()); + + PowerMock.replayAll(); + + // Try to send 3, make first pass, second fail. Should save last two + Whitebox.setInternalState(workerTask, "toSend", Arrays.asList(record1, record2, record3)); + Whitebox.invokeMethod(workerTask, "sendRecords"); + assertEquals(true, Whitebox.getInternalState(workerTask, "lastSendFailed")); + assertEquals(Arrays.asList(record3), Whitebox.getInternalState(workerTask, "toSend")); + + // Next they all succeed + Whitebox.invokeMethod(workerTask, "sendRecords"); + assertEquals(false, Whitebox.getInternalState(workerTask, "lastSendFailed")); + assertNull(Whitebox.getInternalState(workerTask, "toSend")); + + PowerMock.verifyAll(); + } + + @Test + public void testSendRecordsTopicCreateRetriesMidway() throws Exception { + createWorkerTask(); + + // Differentiate only by Kafka partition so we can reuse conversion expectations + SourceRecord record1 = new SourceRecord(PARTITION, OFFSET, TOPIC, 1, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + SourceRecord record2 = new SourceRecord(PARTITION, OFFSET, TOPIC, 2, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + SourceRecord record3 = new SourceRecord(PARTITION, OFFSET, OTHER_TOPIC, 3, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + + // First round + expectPreliminaryCalls(OTHER_TOPIC); + expectTopicCreation(TOPIC); + expectSendRecordTaskCommitRecordSucceed(false, false); + expectSendRecordTaskCommitRecordSucceed(false, false); + + EasyMock.expect(admin.describeTopics(OTHER_TOPIC)).andReturn(Collections.emptyMap()); + // First call to create the topic times out + Capture newTopicCapture = EasyMock.newCapture(); + EasyMock.expect(admin.createTopic(EasyMock.capture(newTopicCapture))) + .andThrow(new RetriableException(new TimeoutException("timeout"))); + + // Second round + expectTopicCreation(OTHER_TOPIC); + expectSendRecord(OTHER_TOPIC, false, true, true, true, true, emptyHeaders()); + + PowerMock.replayAll(); + + // Try to send 3, make first pass, second fail. Should save last two + Whitebox.setInternalState(workerTask, "toSend", Arrays.asList(record1, record2, record3)); + Whitebox.invokeMethod(workerTask, "sendRecords"); + assertEquals(true, Whitebox.getInternalState(workerTask, "lastSendFailed")); + assertEquals(Arrays.asList(record3), Whitebox.getInternalState(workerTask, "toSend")); + + // Next they all succeed + Whitebox.invokeMethod(workerTask, "sendRecords"); + assertEquals(false, Whitebox.getInternalState(workerTask, "lastSendFailed")); + assertNull(Whitebox.getInternalState(workerTask, "toSend")); + + PowerMock.verifyAll(); + } + + @Test(expected = ConnectException.class) + public void testTopicDescribeFails() throws Exception { + createWorkerTask(); + + SourceRecord record1 = new SourceRecord(PARTITION, OFFSET, TOPIC, 1, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + SourceRecord record2 = new SourceRecord(PARTITION, OFFSET, TOPIC, 2, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + + expectPreliminaryCalls(); + EasyMock.expect(admin.describeTopics(TOPIC)) + .andThrow(new ConnectException(new TopicAuthorizationException("unauthorized"))); + + PowerMock.replayAll(); + + Whitebox.setInternalState(workerTask, "toSend", Arrays.asList(record1, record2)); + Whitebox.invokeMethod(workerTask, "sendRecords"); + } + + @Test(expected = ConnectException.class) + public void testTopicCreateFails() throws Exception { + createWorkerTask(); + + SourceRecord record1 = new SourceRecord(PARTITION, OFFSET, TOPIC, 1, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + SourceRecord record2 = new SourceRecord(PARTITION, OFFSET, TOPIC, 2, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + + expectPreliminaryCalls(); + EasyMock.expect(admin.describeTopics(TOPIC)).andReturn(Collections.emptyMap()); + + Capture newTopicCapture = EasyMock.newCapture(); + EasyMock.expect(admin.createTopic(EasyMock.capture(newTopicCapture))) + .andThrow(new ConnectException(new TopicAuthorizationException("unauthorized"))); + + PowerMock.replayAll(); + + Whitebox.setInternalState(workerTask, "toSend", Arrays.asList(record1, record2)); + Whitebox.invokeMethod(workerTask, "sendRecords"); + assertNotNull(newTopicCapture.getValue()); + } + + @Test(expected = ConnectException.class) + public void testTopicCreateFailsWithExceptionWhenCreateReturnsFalse() throws Exception { + createWorkerTask(); + + SourceRecord record1 = new SourceRecord(PARTITION, OFFSET, TOPIC, 1, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + SourceRecord record2 = new SourceRecord(PARTITION, OFFSET, TOPIC, 2, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD); + + expectPreliminaryCalls(); + EasyMock.expect(admin.describeTopics(TOPIC)).andReturn(Collections.emptyMap()); + + Capture newTopicCapture = EasyMock.newCapture(); + EasyMock.expect(admin.createTopic(EasyMock.capture(newTopicCapture))).andReturn(false); + + PowerMock.replayAll(); + + Whitebox.setInternalState(workerTask, "toSend", Arrays.asList(record1, record2)); + Whitebox.invokeMethod(workerTask, "sendRecords"); + assertNotNull(newTopicCapture.getValue()); + } + + private void expectPreliminaryCalls() { + expectPreliminaryCalls(TOPIC); + } + + private void expectPreliminaryCalls(String topic) { + expectConvertHeadersAndKeyValue(topic, true, emptyHeaders()); + expectApplyTransformationChain(false); + offsetWriter.offset(PARTITION, OFFSET); + PowerMock.expectLastCall(); + } + + private CountDownLatch expectEmptyPolls(int minimum, final AtomicInteger count) throws InterruptedException { + final CountDownLatch latch = new CountDownLatch(minimum); + // Note that we stub these to allow any number of calls because the thread will continue to + // run. The count passed in + latch returned just makes sure we get *at least* that number of + // calls + EasyMock.expect(sourceTask.poll()) + .andStubAnswer(new IAnswer>() { + @Override + public List answer() throws Throwable { + count.incrementAndGet(); + latch.countDown(); + Thread.sleep(10); + return Collections.emptyList(); + } + }); + return latch; + } + + private CountDownLatch expectPolls(int minimum, final AtomicInteger count) throws InterruptedException { + final CountDownLatch latch = new CountDownLatch(minimum); + // Note that we stub these to allow any number of calls because the thread will continue to + // run. The count passed in + latch returned just makes sure we get *at least* that number of + // calls + EasyMock.expect(sourceTask.poll()) + .andStubAnswer(new IAnswer>() { + @Override + public List answer() throws Throwable { + count.incrementAndGet(); + latch.countDown(); + Thread.sleep(10); + return RECORDS; + } + }); + // Fallout of the poll() call + expectSendRecordAnyTimes(); + return latch; + } + + private CountDownLatch expectPolls(int count) throws InterruptedException { + return expectPolls(count, new AtomicInteger()); + } + + @SuppressWarnings("unchecked") + private void expectSendRecordSyncFailure(Throwable error) throws InterruptedException { + expectConvertHeadersAndKeyValue(false); + expectApplyTransformationChain(false); + + offsetWriter.offset(PARTITION, OFFSET); + PowerMock.expectLastCall(); + + EasyMock.expect( + producer.send(EasyMock.anyObject(ProducerRecord.class), + EasyMock.anyObject(org.apache.kafka.clients.producer.Callback.class))) + .andThrow(error); + } + + private Capture> expectSendRecordAnyTimes() throws InterruptedException { + return expectSendRecordTaskCommitRecordSucceed(true, false); + } + + private Capture> expectSendRecordOnce(boolean isRetry) throws InterruptedException { + return expectSendRecordTaskCommitRecordSucceed(false, isRetry); + } + + private Capture> expectSendRecordProducerCallbackFail() throws InterruptedException { + return expectSendRecord(TOPIC, false, false, false, false, true, emptyHeaders()); + } + + private Capture> expectSendRecordTaskCommitRecordSucceed(boolean anyTimes, boolean isRetry) throws InterruptedException { + return expectSendRecord(TOPIC, anyTimes, isRetry, true, true, true, emptyHeaders()); + } + + private Capture> expectSendRecordTaskCommitRecordFail(boolean anyTimes, boolean isRetry) throws InterruptedException { + return expectSendRecord(TOPIC, anyTimes, isRetry, true, false, true, emptyHeaders()); + } + + private Capture> expectSendRecord(boolean anyTimes, boolean isRetry, boolean succeed) throws InterruptedException { + return expectSendRecord(TOPIC, anyTimes, isRetry, succeed, true, true, emptyHeaders()); + } + + private Capture> expectSendRecord( + String topic, + boolean anyTimes, + boolean isRetry, + boolean sendSuccess, + boolean commitSuccess, + boolean isMockedConverters, + Headers headers + ) throws InterruptedException { + if (isMockedConverters) { + expectConvertHeadersAndKeyValue(topic, anyTimes, headers); + } + + expectApplyTransformationChain(anyTimes); + + Capture> sent = EasyMock.newCapture(); + + // 1. Offset data is passed to the offset storage. + if (!isRetry) { + offsetWriter.offset(PARTITION, OFFSET); + if (anyTimes) + PowerMock.expectLastCall().anyTimes(); + else + PowerMock.expectLastCall(); + } + + // 2. Converted data passed to the producer, which will need callbacks invoked for flush to work + IExpectationSetters> expect = EasyMock.expect( + producer.send(EasyMock.capture(sent), + EasyMock.capture(producerCallbacks))); + IAnswer> expectResponse = new IAnswer>() { + @Override + public Future answer() throws Throwable { + synchronized (producerCallbacks) { + for (org.apache.kafka.clients.producer.Callback cb : producerCallbacks.getValues()) { + if (sendSuccess) { + cb.onCompletion(new RecordMetadata(new TopicPartition("foo", 0), 0, 0, + 0L, 0L, 0, 0), null); + } else { + cb.onCompletion(null, new TopicAuthorizationException("foo")); + } + } + producerCallbacks.reset(); + } + return sendFuture; + } + }; + if (anyTimes) + expect.andStubAnswer(expectResponse); + else + expect.andAnswer(expectResponse); + + if (sendSuccess) { + // 3. As a result of a successful producer send callback, we'll notify the source task of the record commit + expectTaskCommitRecordWithOffset(anyTimes, commitSuccess); + expectTaskGetTopic(anyTimes); + } + + return sent; + } + + private void expectConvertHeadersAndKeyValue(boolean anyTimes) { + expectConvertHeadersAndKeyValue(TOPIC, anyTimes, emptyHeaders()); + } + + private void expectConvertHeadersAndKeyValue(String topic, boolean anyTimes, Headers headers) { + for (Header header : headers) { + IExpectationSetters convertHeaderExpect = EasyMock.expect(headerConverter.fromConnectHeader(topic, header.key(), Schema.STRING_SCHEMA, new String(header.value()))); + if (anyTimes) + convertHeaderExpect.andStubReturn(header.value()); + else + convertHeaderExpect.andReturn(header.value()); + } + IExpectationSetters convertKeyExpect = EasyMock.expect(keyConverter.fromConnectData(topic, headers, KEY_SCHEMA, KEY)); + if (anyTimes) + convertKeyExpect.andStubReturn(SERIALIZED_KEY); + else + convertKeyExpect.andReturn(SERIALIZED_KEY); + IExpectationSetters convertValueExpect = EasyMock.expect(valueConverter.fromConnectData(topic, headers, RECORD_SCHEMA, RECORD)); + if (anyTimes) + convertValueExpect.andStubReturn(SERIALIZED_RECORD); + else + convertValueExpect.andReturn(SERIALIZED_RECORD); + } + + private void expectApplyTransformationChain(boolean anyTimes) { + final Capture recordCapture = EasyMock.newCapture(); + IExpectationSetters convertKeyExpect = EasyMock.expect(transformationChain.apply(EasyMock.capture(recordCapture))); + if (anyTimes) + convertKeyExpect.andStubAnswer(new IAnswer() { + @Override + public SourceRecord answer() { + return recordCapture.getValue(); + } + }); + else + convertKeyExpect.andAnswer(new IAnswer() { + @Override + public SourceRecord answer() { + return recordCapture.getValue(); + } + }); + } + + private void expectTaskCommitRecordWithOffset(boolean anyTimes, boolean succeed) throws InterruptedException { + sourceTask.commitRecord(EasyMock.anyObject(SourceRecord.class), EasyMock.anyObject(RecordMetadata.class)); + IExpectationSetters expect = EasyMock.expectLastCall(); + if (!succeed) { + expect = expect.andThrow(new RuntimeException("Error committing record in source task")); + } + if (anyTimes) { + expect.anyTimes(); + } + } + + private void expectTaskGetTopic(boolean anyTimes) { + final Capture connectorCapture = EasyMock.newCapture(); + final Capture topicCapture = EasyMock.newCapture(); + IExpectationSetters expect = EasyMock.expect(statusBackingStore.getTopic( + EasyMock.capture(connectorCapture), + EasyMock.capture(topicCapture))); + if (anyTimes) { + expect.andStubAnswer(() -> new TopicStatus( + topicCapture.getValue(), + new ConnectorTaskId(connectorCapture.getValue(), 0), + Time.SYSTEM.milliseconds())); + } else { + expect.andAnswer(() -> new TopicStatus( + topicCapture.getValue(), + new ConnectorTaskId(connectorCapture.getValue(), 0), + Time.SYSTEM.milliseconds())); + } + if (connectorCapture.hasCaptured() && topicCapture.hasCaptured()) { + assertEquals("job", connectorCapture.getValue()); + assertEquals(TOPIC, topicCapture.getValue()); + } + } + + private boolean awaitLatch(CountDownLatch latch) { + try { + return latch.await(5000, TimeUnit.MILLISECONDS); + } catch (InterruptedException e) { + // ignore + } + return false; + } + + @SuppressWarnings("unchecked") + private void expectOffsetFlush(boolean succeed) throws Exception { + EasyMock.expect(offsetWriter.beginFlush()).andReturn(true); + Future flushFuture = PowerMock.createMock(Future.class); + EasyMock.expect(offsetWriter.doFlush(EasyMock.anyObject(Callback.class))).andReturn(flushFuture); + // Should throw for failure + IExpectationSetters futureGetExpect = EasyMock.expect( + flushFuture.get(EasyMock.anyLong(), EasyMock.anyObject(TimeUnit.class))); + if (succeed) { + sourceTask.commit(); + EasyMock.expectLastCall(); + futureGetExpect.andReturn(null); + } else { + futureGetExpect.andThrow(new TimeoutException()); + offsetWriter.cancelFlush(); + PowerMock.expectLastCall(); + } + } + + private void assertPollMetrics(int minimumPollCountExpected) { + MetricGroup sourceTaskGroup = workerTask.sourceTaskMetricsGroup().metricGroup(); + MetricGroup taskGroup = workerTask.taskMetricsGroup().metricGroup(); + double pollRate = metrics.currentMetricValueAsDouble(sourceTaskGroup, "source-record-poll-rate"); + double pollTotal = metrics.currentMetricValueAsDouble(sourceTaskGroup, "source-record-poll-total"); + if (minimumPollCountExpected > 0) { + assertEquals(RECORDS.size(), metrics.currentMetricValueAsDouble(taskGroup, "batch-size-max"), 0.000001d); + assertEquals(RECORDS.size(), metrics.currentMetricValueAsDouble(taskGroup, "batch-size-avg"), 0.000001d); + assertTrue(pollRate > 0.0d); + } else { + assertTrue(pollRate == 0.0d); + } + assertTrue(pollTotal >= minimumPollCountExpected); + + double writeRate = metrics.currentMetricValueAsDouble(sourceTaskGroup, "source-record-write-rate"); + double writeTotal = metrics.currentMetricValueAsDouble(sourceTaskGroup, "source-record-write-total"); + if (minimumPollCountExpected > 0) { + assertTrue(writeRate > 0.0d); + } else { + assertTrue(writeRate == 0.0d); + } + assertTrue(writeTotal >= minimumPollCountExpected); + + double pollBatchTimeMax = metrics.currentMetricValueAsDouble(sourceTaskGroup, "poll-batch-max-time-ms"); + double pollBatchTimeAvg = metrics.currentMetricValueAsDouble(sourceTaskGroup, "poll-batch-avg-time-ms"); + if (minimumPollCountExpected > 0) { + assertTrue(pollBatchTimeMax >= 0.0d); + } + assertTrue(Double.isNaN(pollBatchTimeAvg) || pollBatchTimeAvg > 0.0d); + double activeCount = metrics.currentMetricValueAsDouble(sourceTaskGroup, "source-record-active-count"); + double activeCountMax = metrics.currentMetricValueAsDouble(sourceTaskGroup, "source-record-active-count-max"); + assertEquals(0, activeCount, 0.000001d); + if (minimumPollCountExpected > 0) { + assertEquals(RECORDS.size(), activeCountMax, 0.000001d); + } + } + + private RecordHeaders emptyHeaders() { + return new RecordHeaders(); + } + + private abstract static class TestSourceTask extends SourceTask { + } + + private void expectClose() { + producer.close(EasyMock.anyObject(Duration.class)); + EasyMock.expectLastCall(); + + admin.close(EasyMock.anyObject(Duration.class)); + EasyMock.expectLastCall(); + + transformationChain.close(); + EasyMock.expectLastCall(); + } + + private void expectTopicCreation(String topic) { + if (config.topicCreationEnable()) { + EasyMock.expect(admin.describeTopics(topic)).andReturn(Collections.emptyMap()); + Capture newTopicCapture = EasyMock.newCapture(); + EasyMock.expect(admin.createTopic(EasyMock.capture(newTopicCapture))).andReturn(true); + } + } +} diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java index b3fb6758f57f6..e32fcb0d9cefb 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java @@ -60,7 +60,8 @@ import org.apache.kafka.connect.storage.StatusBackingStore; import org.apache.kafka.connect.util.ConnectorTaskId; import org.apache.kafka.connect.util.ThreadedTest; -import org.easymock.Capture; +import org.apache.kafka.connect.util.TopicAdmin; +import org.apache.kafka.connect.util.TopicCreationGroup; import org.easymock.EasyMock; import org.junit.Before; import org.junit.Test; @@ -84,6 +85,7 @@ import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ExecutorService; +import static org.apache.kafka.connect.runtime.WorkerConfig.TOPIC_CREATION_ENABLE_CONFIG; import static org.apache.kafka.connect.runtime.errors.RetryWithToleranceOperatorTest.NOOP_OPERATOR; import static org.easymock.EasyMock.anyObject; import static org.easymock.EasyMock.eq; @@ -139,6 +141,10 @@ public class WorkerTest extends ThreadedTest { @Mock private ExecutorService executorService; @MockNice private ConnectorConfig connectorConfig; private String mockFileProviderTestId; + private Map connectorProps; + + // when this test becomes parameterized, this variable will be a test parameter + public boolean enableTopicCreation = false; @Before public void setup() { @@ -155,6 +161,7 @@ public void setup() { workerProps.put("config.providers.file.class", MockFileConfigProvider.class.getName()); mockFileProviderTestId = UUID.randomUUID().toString(); workerProps.put("config.providers.file.param.testId", mockFileProviderTestId); + workerProps.put(TOPIC_CREATION_ENABLE_CONFIG, String.valueOf(enableTopicCreation)); config = new StandaloneConfig(workerProps); defaultProducerConfigs.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); @@ -176,6 +183,8 @@ public void setup() { defaultConsumerConfigs .put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArrayDeserializer"); + // Some common defaults. They might change on individual tests + connectorProps = anyConnectorConfigMap(); PowerMock.mockStatic(Plugins.class); } @@ -190,11 +199,7 @@ public void testStartAndStopConnector() { .andReturn(sourceConnector); EasyMock.expect(sourceConnector.version()).andReturn("1.0"); - Map props = new HashMap<>(); - props.put(SinkConnectorConfig.TOPICS_CONFIG, "foo,bar"); - props.put(ConnectorConfig.TASKS_MAX_CONFIG, "1"); - props.put(ConnectorConfig.NAME_CONFIG, CONNECTOR_ID); - props.put(ConnectorConfig.CONNECTOR_CLASS_CONFIG, WorkerTestConnector.class.getName()); + connectorProps.put(ConnectorConfig.CONNECTOR_CLASS_CONFIG, WorkerTestConnector.class.getName()); EasyMock.expect(sourceConnector.version()).andReturn("1.0"); @@ -205,7 +210,7 @@ public void testStartAndStopConnector() { sourceConnector.initialize(anyObject(ConnectorContext.class)); EasyMock.expectLastCall(); - sourceConnector.start(props); + sourceConnector.start(connectorProps); EasyMock.expectLastCall(); EasyMock.expect(Plugins.compareAndSwapLoaders(delegatingLoader)) @@ -230,10 +235,10 @@ public void testStartAndStopConnector() { worker.start(); assertEquals(Collections.emptySet(), worker.connectorNames()); - worker.startConnector(CONNECTOR_ID, props, ctx, connectorStatusListener, TargetState.STARTED); + worker.startConnector(CONNECTOR_ID, connectorProps, ctx, connectorStatusListener, TargetState.STARTED); assertEquals(new HashSet<>(Arrays.asList(CONNECTOR_ID)), worker.connectorNames()); try { - worker.startConnector(CONNECTOR_ID, props, ctx, connectorStatusListener, TargetState.STARTED); + worker.startConnector(CONNECTOR_ID, connectorProps, ctx, connectorStatusListener, TargetState.STARTED); fail("Should have thrown exception when trying to add connector with same name."); } catch (ConnectException e) { // expected @@ -268,11 +273,7 @@ public void testStartConnectorFailure() { expectStartStorage(); expectFileConfigProvider(); - Map props = new HashMap<>(); - props.put(SinkConnectorConfig.TOPICS_CONFIG, "foo,bar"); - props.put(ConnectorConfig.TASKS_MAX_CONFIG, "1"); - props.put(ConnectorConfig.NAME_CONFIG, CONNECTOR_ID); - props.put(ConnectorConfig.CONNECTOR_CLASS_CONFIG, "java.util.HashMap"); // Bad connector class name + connectorProps.put(ConnectorConfig.CONNECTOR_CLASS_CONFIG, "java.util.HashMap"); // Bad connector class name EasyMock.expect(plugins.currentThreadLoader()).andReturn(delegatingLoader); EasyMock.expect(plugins.newConnector(EasyMock.anyString())) @@ -294,7 +295,7 @@ public void testStartConnectorFailure() { worker.start(); assertStatistics(worker, 0, 0); - assertFalse(worker.startConnector(CONNECTOR_ID, props, ctx, connectorStatusListener, TargetState.STARTED)); + assertFalse(worker.startConnector(CONNECTOR_ID, connectorProps, ctx, connectorStatusListener, TargetState.STARTED)); assertStartupStatistics(worker, 1, 1, 0, 0); assertEquals(Collections.emptySet(), worker.connectorNames()); @@ -315,22 +316,18 @@ public void testAddConnectorByAlias() { expectFileConfigProvider(); EasyMock.expect(plugins.currentThreadLoader()).andReturn(delegatingLoader).times(2); - EasyMock.expect(plugins.newConnector("WorkerTestConnector")).andReturn(sinkConnector); - EasyMock.expect(sinkConnector.version()).andReturn("1.0"); + EasyMock.expect(plugins.newConnector("WorkerTestConnector")).andReturn(sourceConnector); + EasyMock.expect(sourceConnector.version()).andReturn("1.0"); - Map props = new HashMap<>(); - props.put(SinkConnectorConfig.TOPICS_CONFIG, "foo,bar"); - props.put(ConnectorConfig.TASKS_MAX_CONFIG, "1"); - props.put(ConnectorConfig.NAME_CONFIG, CONNECTOR_ID); - props.put(ConnectorConfig.CONNECTOR_CLASS_CONFIG, "WorkerTestConnector"); + connectorProps.put(ConnectorConfig.CONNECTOR_CLASS_CONFIG, "WorkerTestConnector"); - EasyMock.expect(sinkConnector.version()).andReturn("1.0"); - EasyMock.expect(plugins.compareAndSwapLoaders(sinkConnector)) + EasyMock.expect(sourceConnector.version()).andReturn("1.0"); + EasyMock.expect(plugins.compareAndSwapLoaders(sourceConnector)) .andReturn(delegatingLoader) .times(2); - sinkConnector.initialize(anyObject(ConnectorContext.class)); + sourceConnector.initialize(anyObject(ConnectorContext.class)); EasyMock.expectLastCall(); - sinkConnector.start(props); + sourceConnector.start(connectorProps); EasyMock.expectLastCall(); EasyMock.expect(Plugins.compareAndSwapLoaders(delegatingLoader)) @@ -341,7 +338,7 @@ public void testAddConnectorByAlias() { EasyMock.expectLastCall(); // Remove - sinkConnector.stop(); + sourceConnector.stop(); EasyMock.expectLastCall(); connectorStatusListener.onShutdown(CONNECTOR_ID); @@ -357,7 +354,7 @@ public void testAddConnectorByAlias() { assertStatistics(worker, 0, 0); assertEquals(Collections.emptySet(), worker.connectorNames()); - worker.startConnector(CONNECTOR_ID, props, ctx, connectorStatusListener, TargetState.STARTED); + worker.startConnector(CONNECTOR_ID, connectorProps, ctx, connectorStatusListener, TargetState.STARTED); assertEquals(new HashSet<>(Arrays.asList(CONNECTOR_ID)), worker.connectorNames()); assertStatistics(worker, 1, 0); assertStartupStatistics(worker, 1, 0, 0, 0); @@ -381,22 +378,18 @@ public void testAddConnectorByShortAlias() { expectFileConfigProvider(); EasyMock.expect(plugins.currentThreadLoader()).andReturn(delegatingLoader).times(2); - EasyMock.expect(plugins.newConnector("WorkerTest")).andReturn(sinkConnector); - EasyMock.expect(sinkConnector.version()).andReturn("1.0"); + EasyMock.expect(plugins.newConnector("WorkerTest")).andReturn(sourceConnector); + EasyMock.expect(sourceConnector.version()).andReturn("1.0"); - Map props = new HashMap<>(); - props.put(SinkConnectorConfig.TOPICS_CONFIG, "foo,bar"); - props.put(ConnectorConfig.TASKS_MAX_CONFIG, "1"); - props.put(ConnectorConfig.NAME_CONFIG, CONNECTOR_ID); - props.put(ConnectorConfig.CONNECTOR_CLASS_CONFIG, "WorkerTest"); + connectorProps.put(ConnectorConfig.CONNECTOR_CLASS_CONFIG, "WorkerTest"); - EasyMock.expect(sinkConnector.version()).andReturn("1.0"); - EasyMock.expect(plugins.compareAndSwapLoaders(sinkConnector)) + EasyMock.expect(sourceConnector.version()).andReturn("1.0"); + EasyMock.expect(plugins.compareAndSwapLoaders(sourceConnector)) .andReturn(delegatingLoader) .times(2); - sinkConnector.initialize(anyObject(ConnectorContext.class)); + sourceConnector.initialize(anyObject(ConnectorContext.class)); EasyMock.expectLastCall(); - sinkConnector.start(props); + sourceConnector.start(connectorProps); EasyMock.expectLastCall(); EasyMock.expect(Plugins.compareAndSwapLoaders(delegatingLoader)) @@ -407,7 +400,7 @@ public void testAddConnectorByShortAlias() { EasyMock.expectLastCall(); // Remove - sinkConnector.stop(); + sourceConnector.stop(); EasyMock.expectLastCall(); connectorStatusListener.onShutdown(CONNECTOR_ID); @@ -423,7 +416,7 @@ public void testAddConnectorByShortAlias() { assertStatistics(worker, 0, 0); assertEquals(Collections.emptySet(), worker.connectorNames()); - worker.startConnector(CONNECTOR_ID, props, ctx, connectorStatusListener, TargetState.STARTED); + worker.startConnector(CONNECTOR_ID, connectorProps, ctx, connectorStatusListener, TargetState.STARTED); assertEquals(new HashSet<>(Arrays.asList(CONNECTOR_ID)), worker.connectorNames()); assertStatistics(worker, 1, 0); @@ -465,11 +458,8 @@ public void testReconfigureConnectorTasks() { .andReturn(sinkConnector); EasyMock.expect(sinkConnector.version()).andReturn("1.0"); - Map props = new HashMap<>(); - props.put(SinkConnectorConfig.TOPICS_CONFIG, "foo,bar"); - props.put(ConnectorConfig.TASKS_MAX_CONFIG, "1"); - props.put(ConnectorConfig.NAME_CONFIG, CONNECTOR_ID); - props.put(ConnectorConfig.CONNECTOR_CLASS_CONFIG, WorkerTestConnector.class.getName()); + connectorProps.put(SinkConnectorConfig.TOPICS_CONFIG, "foo,bar"); + connectorProps.put(ConnectorConfig.CONNECTOR_CLASS_CONFIG, WorkerTestConnector.class.getName()); EasyMock.expect(sinkConnector.version()).andReturn("1.0"); EasyMock.expect(plugins.compareAndSwapLoaders(sinkConnector)) @@ -477,7 +467,7 @@ public void testReconfigureConnectorTasks() { .times(3); sinkConnector.initialize(anyObject(ConnectorContext.class)); EasyMock.expectLastCall(); - sinkConnector.start(props); + sinkConnector.start(connectorProps); EasyMock.expectLastCall(); EasyMock.expect(Plugins.compareAndSwapLoaders(delegatingLoader)) @@ -510,16 +500,16 @@ public void testReconfigureConnectorTasks() { assertStatistics(worker, 0, 0); assertEquals(Collections.emptySet(), worker.connectorNames()); - worker.startConnector(CONNECTOR_ID, props, ctx, connectorStatusListener, TargetState.STARTED); + worker.startConnector(CONNECTOR_ID, connectorProps, ctx, connectorStatusListener, TargetState.STARTED); assertStatistics(worker, 1, 0); assertEquals(new HashSet<>(Arrays.asList(CONNECTOR_ID)), worker.connectorNames()); try { - worker.startConnector(CONNECTOR_ID, props, ctx, connectorStatusListener, TargetState.STARTED); + worker.startConnector(CONNECTOR_ID, connectorProps, ctx, connectorStatusListener, TargetState.STARTED); fail("Should have thrown exception when trying to add connector with same name."); } catch (ConnectException e) { // expected } - Map connProps = new HashMap<>(props); + Map connProps = new HashMap<>(connectorProps); connProps.put(ConnectorConfig.TASKS_MAX_CONFIG, "2"); ConnectorConfig connConfig = new SinkConnectorConfig(plugins, connProps); List> taskConfigs = worker.connectorTaskConfigs(CONNECTOR_ID, connConfig); @@ -553,26 +543,7 @@ public void testAddRemoveTask() throws Exception { EasyMock.expect(workerTask.id()).andStubReturn(TASK_ID); EasyMock.expect(plugins.currentThreadLoader()).andReturn(delegatingLoader).times(2); - PowerMock.expectNew( - WorkerSourceTask.class, EasyMock.eq(TASK_ID), - EasyMock.eq(task), - anyObject(TaskStatus.Listener.class), - EasyMock.eq(TargetState.STARTED), - anyObject(JsonConverter.class), - anyObject(JsonConverter.class), - anyObject(JsonConverter.class), - EasyMock.eq(new TransformationChain<>(Collections.emptyList(), NOOP_OPERATOR)), - anyObject(KafkaProducer.class), - anyObject(OffsetStorageReader.class), - anyObject(OffsetStorageWriter.class), - EasyMock.eq(config), - anyObject(ClusterConfigState.class), - anyObject(ConnectMetrics.class), - anyObject(ClassLoader.class), - anyObject(Time.class), - anyObject(RetryWithToleranceOperator.class), - anyObject(StatusBackingStore.class)) - .andReturn(workerTask); + expectNewWorkerTask(); Map origProps = new HashMap<>(); origProps.put(TaskConfig.TASK_CLASS_CONFIG, TestSourceTask.class.getName()); @@ -650,24 +621,7 @@ public void testTaskStatusMetricsStatuses() throws Exception { EasyMock.expect(workerTask.id()).andStubReturn(TASK_ID); EasyMock.expect(plugins.currentThreadLoader()).andReturn(delegatingLoader).times(2); - PowerMock.expectNew(WorkerSourceTask.class, EasyMock.eq(TASK_ID), - EasyMock.eq(task), - anyObject(TaskStatus.Listener.class), - EasyMock.eq(TargetState.STARTED), - anyObject(JsonConverter.class), - anyObject(JsonConverter.class), - anyObject(JsonConverter.class), - EasyMock.eq(new TransformationChain<>(Collections.emptyList(), NOOP_OPERATOR)), - anyObject(KafkaProducer.class), - anyObject(OffsetStorageReader.class), - anyObject(OffsetStorageWriter.class), - EasyMock.eq(config), - anyObject(ClusterConfigState.class), - anyObject(ConnectMetrics.class), - anyObject(ClassLoader.class), - anyObject(Time.class), - anyObject(RetryWithToleranceOperator.class), - anyObject(StatusBackingStore.class)).andReturn(workerTask); + expectNewWorkerTask(); Map origProps = new HashMap<>(); origProps.put(TaskConfig.TASK_CLASS_CONFIG, TestSourceTask.class.getName()); @@ -864,26 +818,7 @@ public void testCleanupTasksOnStop() throws Exception { EasyMock.expect(workerTask.id()).andStubReturn(TASK_ID); EasyMock.expect(plugins.currentThreadLoader()).andReturn(delegatingLoader).times(2); - PowerMock.expectNew( - WorkerSourceTask.class, EasyMock.eq(TASK_ID), - EasyMock.eq(task), - anyObject(TaskStatus.Listener.class), - EasyMock.eq(TargetState.STARTED), - anyObject(JsonConverter.class), - anyObject(JsonConverter.class), - anyObject(JsonConverter.class), - EasyMock.eq(new TransformationChain<>(Collections.emptyList(), NOOP_OPERATOR)), - anyObject(KafkaProducer.class), - anyObject(OffsetStorageReader.class), - anyObject(OffsetStorageWriter.class), - anyObject(WorkerConfig.class), - anyObject(ClusterConfigState.class), - anyObject(ConnectMetrics.class), - EasyMock.eq(pluginLoader), - anyObject(Time.class), - anyObject(RetryWithToleranceOperator.class), - anyObject(StatusBackingStore.class)) - .andReturn(workerTask); + expectNewWorkerTask(); Map origProps = new HashMap<>(); origProps.put(TaskConfig.TASK_CLASS_CONFIG, TestSourceTask.class.getName()); @@ -956,31 +891,8 @@ public void testConverterOverrides() throws Exception { EasyMock.expect(workerTask.id()).andStubReturn(TASK_ID); - Capture keyConverter = EasyMock.newCapture(); - Capture valueConverter = EasyMock.newCapture(); - Capture headerConverter = EasyMock.newCapture(); - EasyMock.expect(plugins.currentThreadLoader()).andReturn(delegatingLoader).times(2); - PowerMock.expectNew( - WorkerSourceTask.class, EasyMock.eq(TASK_ID), - EasyMock.eq(task), - anyObject(TaskStatus.Listener.class), - EasyMock.eq(TargetState.STARTED), - EasyMock.capture(keyConverter), - EasyMock.capture(valueConverter), - EasyMock.capture(headerConverter), - EasyMock.eq(new TransformationChain<>(Collections.emptyList(), NOOP_OPERATOR)), - anyObject(KafkaProducer.class), - anyObject(OffsetStorageReader.class), - anyObject(OffsetStorageWriter.class), - anyObject(WorkerConfig.class), - anyObject(ClusterConfigState.class), - anyObject(ConnectMetrics.class), - EasyMock.eq(pluginLoader), - anyObject(Time.class), - anyObject(RetryWithToleranceOperator.class), - anyObject(StatusBackingStore.class)) - .andReturn(workerTask); + expectNewWorkerTask(); Map origProps = new HashMap<>(); origProps.put(TaskConfig.TASK_CLASS_CONFIG, TestSourceTask.class.getName()); @@ -1202,7 +1114,7 @@ public void testAdminConfigsClientOverridesWithAllPolicy() { EasyMock.expect(connectorConfig.originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_ADMIN_OVERRIDES_PREFIX)) .andReturn(connConfig); PowerMock.replayAll(); - assertEquals(expectedConfigs, Worker.adminConfigs(new ConnectorTaskId("test", 1), configWithOverrides, connectorConfig, + assertEquals(expectedConfigs, Worker.adminConfigs(new ConnectorTaskId("test", 1), "", configWithOverrides, connectorConfig, null, allConnectorClientConfigOverridePolicy)); } @@ -1219,7 +1131,7 @@ public void testAdminConfigsClientOverridesWithNonePolicy() { EasyMock.expect(connectorConfig.originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_ADMIN_OVERRIDES_PREFIX)) .andReturn(connConfig); PowerMock.replayAll(); - Worker.adminConfigs(new ConnectorTaskId("test", 1), configWithOverrides, connectorConfig, + Worker.adminConfigs(new ConnectorTaskId("test", 1), "", configWithOverrides, connectorConfig, null, noneConnectorClientConfigOverridePolicy); } @@ -1229,11 +1141,11 @@ private void assertStatusMetrics(long expected, String metricName) { assertNull(statusMetrics); return; } - assertEquals(expected, (long) MockConnectMetrics.currentMetricValue(worker.metrics(), statusMetrics, metricName)); + assertEquals(expected, MockConnectMetrics.currentMetricValue(worker.metrics(), statusMetrics, metricName)); } private void assertStatistics(Worker worker, int connectors, int tasks) { - assertStatusMetrics((long) tasks, "connector-total-task-count"); + assertStatusMetrics(tasks, "connector-total-task-count"); MetricGroup workerMetrics = worker.workerMetricsGroup().metricGroup(); assertEquals(connectors, MockConnectMetrics.currentMetricValueAsDouble(worker.metrics(), workerMetrics, "connector-count"), 0.0001d); assertEquals(tasks, MockConnectMetrics.currentMetricValueAsDouble(worker.metrics(), workerMetrics, "task-count"), 0.0001d); @@ -1360,6 +1272,30 @@ private Map anyConnectorConfigMap() { return props; } + private void expectNewWorkerTask() throws Exception { + PowerMock.expectNew( + WorkerSourceTask.class, EasyMock.eq(TASK_ID), + EasyMock.eq(task), + anyObject(TaskStatus.Listener.class), + EasyMock.eq(TargetState.STARTED), + anyObject(JsonConverter.class), + anyObject(JsonConverter.class), + anyObject(JsonConverter.class), + EasyMock.eq(new TransformationChain<>(Collections.emptyList(), NOOP_OPERATOR)), + anyObject(KafkaProducer.class), + anyObject(TopicAdmin.class), + EasyMock.>anyObject(), + anyObject(OffsetStorageReader.class), + anyObject(OffsetStorageWriter.class), + EasyMock.eq(config), + anyObject(ClusterConfigState.class), + anyObject(ConnectMetrics.class), + EasyMock.eq(pluginLoader), + anyObject(Time.class), + anyObject(RetryWithToleranceOperator.class), + anyObject(StatusBackingStore.class)) + .andReturn(workerTask); + } /* Name here needs to be unique as we are testing the aliasing mechanism */ public static class WorkerTestConnector extends SourceConnector { diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerWithTopicCreationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerWithTopicCreationTest.java new file mode 100644 index 0000000000000..e3c982efc05f4 --- /dev/null +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerWithTopicCreationTest.java @@ -0,0 +1,1409 @@ +/* + * 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.runtime; + +import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.common.Configurable; +import org.apache.kafka.common.config.AbstractConfig; +import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.common.config.ConfigException; +import org.apache.kafka.common.config.provider.MockFileConfigProvider; +import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.connect.connector.ConnectorContext; +import org.apache.kafka.connect.connector.Task; +import org.apache.kafka.connect.connector.policy.AllConnectorClientConfigOverridePolicy; +import org.apache.kafka.connect.connector.policy.ConnectorClientConfigOverridePolicy; +import org.apache.kafka.connect.connector.policy.NoneConnectorClientConfigOverridePolicy; +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.json.JsonConverter; +import org.apache.kafka.connect.json.JsonConverterConfig; +import org.apache.kafka.connect.runtime.ConnectMetrics.MetricGroup; +import org.apache.kafka.connect.runtime.MockConnectMetrics.MockMetricsReporter; +import org.apache.kafka.connect.runtime.distributed.ClusterConfigState; +import org.apache.kafka.connect.runtime.errors.RetryWithToleranceOperator; +import org.apache.kafka.connect.runtime.isolation.DelegatingClassLoader; +import org.apache.kafka.connect.runtime.isolation.PluginClassLoader; +import org.apache.kafka.connect.runtime.isolation.Plugins; +import org.apache.kafka.connect.runtime.isolation.Plugins.ClassLoaderUsage; +import org.apache.kafka.connect.runtime.rest.entities.ConnectorStateInfo; +import org.apache.kafka.connect.runtime.standalone.StandaloneConfig; +import org.apache.kafka.connect.sink.SinkTask; +import org.apache.kafka.connect.source.SourceConnector; +import org.apache.kafka.connect.source.SourceRecord; +import org.apache.kafka.connect.source.SourceTask; +import org.apache.kafka.connect.storage.Converter; +import org.apache.kafka.connect.storage.HeaderConverter; +import org.apache.kafka.connect.storage.OffsetBackingStore; +import org.apache.kafka.connect.storage.OffsetStorageReader; +import org.apache.kafka.connect.storage.OffsetStorageWriter; +import org.apache.kafka.connect.storage.StatusBackingStore; +import org.apache.kafka.connect.util.ConnectorTaskId; +import org.apache.kafka.connect.util.ThreadedTest; +import org.apache.kafka.connect.util.TopicAdmin; +import org.apache.kafka.connect.util.TopicCreationGroup; +import org.easymock.EasyMock; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.powermock.api.easymock.PowerMock; +import org.powermock.api.easymock.annotation.Mock; +import org.powermock.api.easymock.annotation.MockNice; +import org.powermock.api.easymock.annotation.MockStrict; +import org.powermock.core.classloader.annotations.PowerMockIgnore; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ExecutorService; + +import static org.apache.kafka.connect.runtime.TopicCreationConfig.DEFAULT_TOPIC_CREATION_PREFIX; +import static org.apache.kafka.connect.runtime.TopicCreationConfig.PARTITIONS_CONFIG; +import static org.apache.kafka.connect.runtime.TopicCreationConfig.REPLICATION_FACTOR_CONFIG; +import static org.apache.kafka.connect.runtime.WorkerConfig.TOPIC_CREATION_ENABLE_CONFIG; +import static org.apache.kafka.connect.runtime.errors.RetryWithToleranceOperatorTest.NOOP_OPERATOR; +import static org.easymock.EasyMock.anyObject; +import static org.easymock.EasyMock.eq; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.fail; + +@RunWith(PowerMockRunner.class) +@PrepareForTest({Worker.class, Plugins.class}) +@PowerMockIgnore("javax.management.*") +public class WorkerWithTopicCreationTest extends ThreadedTest { + + private static final String CONNECTOR_ID = "test-connector"; + private static final ConnectorTaskId TASK_ID = new ConnectorTaskId("job", 0); + private static final String WORKER_ID = "localhost:8083"; + private final ConnectorClientConfigOverridePolicy noneConnectorClientConfigOverridePolicy = new NoneConnectorClientConfigOverridePolicy(); + private final ConnectorClientConfigOverridePolicy allConnectorClientConfigOverridePolicy = new AllConnectorClientConfigOverridePolicy(); + + private Map workerProps = new HashMap<>(); + private WorkerConfig config; + private Worker worker; + + private Map defaultProducerConfigs = new HashMap<>(); + private Map defaultConsumerConfigs = new HashMap<>(); + + @Mock + private Plugins plugins; + @Mock + private PluginClassLoader pluginLoader; + @Mock + private DelegatingClassLoader delegatingLoader; + @Mock + private OffsetBackingStore offsetBackingStore; + @MockStrict + private TaskStatus.Listener taskStatusListener; + @MockStrict + private ConnectorStatus.Listener connectorStatusListener; + + @Mock private Herder herder; + @Mock private StatusBackingStore statusBackingStore; + @Mock private SourceConnector connector; + @Mock private ConnectorContext ctx; + @Mock private TestSourceTask task; + @Mock private WorkerSourceTask workerTask; + @Mock private Converter keyConverter; + @Mock private Converter valueConverter; + @Mock private Converter taskKeyConverter; + @Mock private Converter taskValueConverter; + @Mock private HeaderConverter taskHeaderConverter; + @Mock private ExecutorService executorService; + @MockNice private ConnectorConfig connectorConfig; + private String mockFileProviderTestId; + private Map connectorProps; + + // when this test becomes parameterized, this variable will be a test parameter + public boolean enableTopicCreation = true; + + @Before + public void setup() { + super.setup(); + workerProps.put("key.converter", "org.apache.kafka.connect.json.JsonConverter"); + workerProps.put("value.converter", "org.apache.kafka.connect.json.JsonConverter"); + workerProps.put("internal.key.converter", "org.apache.kafka.connect.json.JsonConverter"); + workerProps.put("internal.value.converter", "org.apache.kafka.connect.json.JsonConverter"); + workerProps.put("internal.key.converter.schemas.enable", "false"); + workerProps.put("internal.value.converter.schemas.enable", "false"); + workerProps.put("offset.storage.file.filename", "/tmp/connect.offsets"); + workerProps.put(CommonClientConfigs.METRIC_REPORTER_CLASSES_CONFIG, MockMetricsReporter.class.getName()); + workerProps.put("config.providers", "file"); + workerProps.put("config.providers.file.class", MockFileConfigProvider.class.getName()); + mockFileProviderTestId = UUID.randomUUID().toString(); + workerProps.put("config.providers.file.param.testId", mockFileProviderTestId); + workerProps.put(TOPIC_CREATION_ENABLE_CONFIG, String.valueOf(enableTopicCreation)); + config = new StandaloneConfig(workerProps); + + defaultProducerConfigs.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); + defaultProducerConfigs.put( + ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer"); + defaultProducerConfigs.put( + ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer"); + defaultProducerConfigs.put(ProducerConfig.REQUEST_TIMEOUT_MS_CONFIG, Integer.toString(Integer.MAX_VALUE)); + defaultProducerConfigs.put(ProducerConfig.MAX_BLOCK_MS_CONFIG, Long.toString(Long.MAX_VALUE)); + defaultProducerConfigs.put(ProducerConfig.ACKS_CONFIG, "all"); + defaultProducerConfigs.put(ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION, "1"); + defaultProducerConfigs.put(ProducerConfig.DELIVERY_TIMEOUT_MS_CONFIG, Integer.toString(Integer.MAX_VALUE)); + + defaultConsumerConfigs.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); + defaultConsumerConfigs.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false"); + defaultConsumerConfigs.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); + defaultConsumerConfigs + .put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArrayDeserializer"); + defaultConsumerConfigs + .put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArrayDeserializer"); + + // Some common defaults. They might change on individual tests + connectorProps = anyConnectorConfigMap(); + PowerMock.mockStatic(Plugins.class); + } + + @Test + public void testStartAndStopConnector() { + expectConverters(); + expectStartStorage(); + + // Create + EasyMock.expect(plugins.currentThreadLoader()).andReturn(delegatingLoader).times(2); + EasyMock.expect(plugins.newConnector(WorkerTestConnector.class.getName())) + .andReturn(connector); + EasyMock.expect(connector.version()).andReturn("1.0"); + + connectorProps.put(ConnectorConfig.CONNECTOR_CLASS_CONFIG, WorkerTestConnector.class.getName()); + + EasyMock.expect(connector.version()).andReturn("1.0"); + + expectFileConfigProvider(); + EasyMock.expect(plugins.compareAndSwapLoaders(connector)) + .andReturn(delegatingLoader) + .times(2); + + connector.initialize(anyObject(ConnectorContext.class)); + EasyMock.expectLastCall(); + connector.start(connectorProps); + EasyMock.expectLastCall(); + + EasyMock.expect(Plugins.compareAndSwapLoaders(delegatingLoader)) + .andReturn(pluginLoader).times(2); + + connectorStatusListener.onStartup(CONNECTOR_ID); + EasyMock.expectLastCall(); + + // Remove + connector.stop(); + EasyMock.expectLastCall(); + + connectorStatusListener.onShutdown(CONNECTOR_ID); + EasyMock.expectLastCall(); + + expectStopStorage(); + + PowerMock.replayAll(); + + worker = new Worker(WORKER_ID, new MockTime(), plugins, config, offsetBackingStore, noneConnectorClientConfigOverridePolicy); + worker.herder = herder; + worker.start(); + + assertEquals(Collections.emptySet(), worker.connectorNames()); + worker.startConnector(CONNECTOR_ID, connectorProps, ctx, connectorStatusListener, TargetState.STARTED); + assertEquals(new HashSet<>(Arrays.asList(CONNECTOR_ID)), worker.connectorNames()); + try { + worker.startConnector(CONNECTOR_ID, connectorProps, ctx, connectorStatusListener, TargetState.STARTED); + fail("Should have thrown exception when trying to add connector with same name."); + } catch (ConnectException e) { + // expected + } + assertStatistics(worker, 1, 0); + assertStartupStatistics(worker, 1, 0, 0, 0); + worker.stopConnector(CONNECTOR_ID); + assertStatistics(worker, 0, 0); + assertStartupStatistics(worker, 1, 0, 0, 0); + assertEquals(Collections.emptySet(), worker.connectorNames()); + // Nothing should be left, so this should effectively be a nop + worker.stop(); + assertStatistics(worker, 0, 0); + + PowerMock.verifyAll(); + MockFileConfigProvider.assertClosed(mockFileProviderTestId); + } + + private void expectFileConfigProvider() { + EasyMock.expect(plugins.newConfigProvider(EasyMock.anyObject(), + EasyMock.eq("config.providers.file"), EasyMock.anyObject())) + .andAnswer(() -> { + MockFileConfigProvider mockFileConfigProvider = new MockFileConfigProvider(); + mockFileConfigProvider.configure(Collections.singletonMap("testId", mockFileProviderTestId)); + return mockFileConfigProvider; + }); + } + + @Test + public void testStartConnectorFailure() { + expectConverters(); + expectStartStorage(); + expectFileConfigProvider(); + + connectorProps.put(ConnectorConfig.CONNECTOR_CLASS_CONFIG, "java.util.HashMap"); // Bad connector class name + + EasyMock.expect(plugins.currentThreadLoader()).andReturn(delegatingLoader); + EasyMock.expect(plugins.newConnector(EasyMock.anyString())) + .andThrow(new ConnectException("Failed to find Connector")); + + EasyMock.expect(Plugins.compareAndSwapLoaders(delegatingLoader)) + .andReturn(pluginLoader); + + connectorStatusListener.onFailure( + EasyMock.eq(CONNECTOR_ID), + EasyMock.anyObject() + ); + EasyMock.expectLastCall(); + + PowerMock.replayAll(); + + worker = new Worker(WORKER_ID, new MockTime(), plugins, config, offsetBackingStore, noneConnectorClientConfigOverridePolicy); + worker.herder = herder; + worker.start(); + + assertStatistics(worker, 0, 0); + assertFalse(worker.startConnector(CONNECTOR_ID, connectorProps, ctx, connectorStatusListener, TargetState.STARTED)); + + assertStartupStatistics(worker, 1, 1, 0, 0); + assertEquals(Collections.emptySet(), worker.connectorNames()); + + assertStatistics(worker, 0, 0); + assertStartupStatistics(worker, 1, 1, 0, 0); + assertFalse(worker.stopConnector(CONNECTOR_ID)); + assertStatistics(worker, 0, 0); + assertStartupStatistics(worker, 1, 1, 0, 0); + + PowerMock.verifyAll(); + } + + @Test + public void testAddConnectorByAlias() { + expectConverters(); + expectStartStorage(); + expectFileConfigProvider(); + + EasyMock.expect(plugins.currentThreadLoader()).andReturn(delegatingLoader).times(2); + EasyMock.expect(plugins.newConnector("WorkerTestConnector")).andReturn(connector); + EasyMock.expect(connector.version()).andReturn("1.0"); + + connectorProps.put(ConnectorConfig.CONNECTOR_CLASS_CONFIG, "WorkerTestConnector"); + + EasyMock.expect(connector.version()).andReturn("1.0"); + EasyMock.expect(plugins.compareAndSwapLoaders(connector)) + .andReturn(delegatingLoader) + .times(2); + connector.initialize(anyObject(ConnectorContext.class)); + EasyMock.expectLastCall(); + connector.start(connectorProps); + EasyMock.expectLastCall(); + + EasyMock.expect(Plugins.compareAndSwapLoaders(delegatingLoader)) + .andReturn(pluginLoader) + .times(2); + + connectorStatusListener.onStartup(CONNECTOR_ID); + EasyMock.expectLastCall(); + + // Remove + connector.stop(); + EasyMock.expectLastCall(); + + connectorStatusListener.onShutdown(CONNECTOR_ID); + EasyMock.expectLastCall(); + + expectStopStorage(); + + PowerMock.replayAll(); + + worker = new Worker(WORKER_ID, new MockTime(), plugins, config, offsetBackingStore, noneConnectorClientConfigOverridePolicy); + worker.herder = herder; + worker.start(); + + assertStatistics(worker, 0, 0); + assertEquals(Collections.emptySet(), worker.connectorNames()); + worker.startConnector(CONNECTOR_ID, connectorProps, ctx, connectorStatusListener, TargetState.STARTED); + assertEquals(new HashSet<>(Arrays.asList(CONNECTOR_ID)), worker.connectorNames()); + assertStatistics(worker, 1, 0); + assertStartupStatistics(worker, 1, 0, 0, 0); + + worker.stopConnector(CONNECTOR_ID); + assertStatistics(worker, 0, 0); + assertStartupStatistics(worker, 1, 0, 0, 0); + assertEquals(Collections.emptySet(), worker.connectorNames()); + // Nothing should be left, so this should effectively be a nop + worker.stop(); + assertStatistics(worker, 0, 0); + assertStartupStatistics(worker, 1, 0, 0, 0); + + PowerMock.verifyAll(); + } + + @Test + public void testAddConnectorByShortAlias() { + expectConverters(); + expectStartStorage(); + expectFileConfigProvider(); + + EasyMock.expect(plugins.currentThreadLoader()).andReturn(delegatingLoader).times(2); + EasyMock.expect(plugins.newConnector("WorkerTest")).andReturn(connector); + EasyMock.expect(connector.version()).andReturn("1.0"); + + connectorProps.put(ConnectorConfig.CONNECTOR_CLASS_CONFIG, "WorkerTest"); + + EasyMock.expect(connector.version()).andReturn("1.0"); + EasyMock.expect(plugins.compareAndSwapLoaders(connector)) + .andReturn(delegatingLoader) + .times(2); + connector.initialize(anyObject(ConnectorContext.class)); + EasyMock.expectLastCall(); + connector.start(connectorProps); + EasyMock.expectLastCall(); + + EasyMock.expect(Plugins.compareAndSwapLoaders(delegatingLoader)) + .andReturn(pluginLoader) + .times(2); + + connectorStatusListener.onStartup(CONNECTOR_ID); + EasyMock.expectLastCall(); + + // Remove + connector.stop(); + EasyMock.expectLastCall(); + + connectorStatusListener.onShutdown(CONNECTOR_ID); + EasyMock.expectLastCall(); + + expectStopStorage(); + + PowerMock.replayAll(); + + worker = new Worker(WORKER_ID, new MockTime(), plugins, config, offsetBackingStore, noneConnectorClientConfigOverridePolicy); + worker.herder = herder; + worker.start(); + + assertStatistics(worker, 0, 0); + assertEquals(Collections.emptySet(), worker.connectorNames()); + worker.startConnector(CONNECTOR_ID, connectorProps, ctx, connectorStatusListener, TargetState.STARTED); + assertEquals(new HashSet<>(Arrays.asList(CONNECTOR_ID)), worker.connectorNames()); + assertStatistics(worker, 1, 0); + + worker.stopConnector(CONNECTOR_ID); + assertStatistics(worker, 0, 0); + assertEquals(Collections.emptySet(), worker.connectorNames()); + // Nothing should be left, so this should effectively be a nop + worker.stop(); + assertStatistics(worker, 0, 0); + + PowerMock.verifyAll(); + } + + @Test + public void testStopInvalidConnector() { + expectConverters(); + expectStartStorage(); + expectFileConfigProvider(); + + PowerMock.replayAll(); + + worker = new Worker(WORKER_ID, new MockTime(), plugins, config, offsetBackingStore, noneConnectorClientConfigOverridePolicy); + worker.herder = herder; + worker.start(); + + worker.stopConnector(CONNECTOR_ID); + + PowerMock.verifyAll(); + } + + @Test + public void testReconfigureConnectorTasks() { + expectConverters(); + expectStartStorage(); + expectFileConfigProvider(); + + EasyMock.expect(plugins.currentThreadLoader()).andReturn(delegatingLoader).times(3); + EasyMock.expect(plugins.newConnector(WorkerTestConnector.class.getName())) + .andReturn(connector); + EasyMock.expect(connector.version()).andReturn("1.0"); + + connectorProps.put(SinkConnectorConfig.TOPICS_CONFIG, "foo,bar"); + connectorProps.put(ConnectorConfig.CONNECTOR_CLASS_CONFIG, WorkerTestConnector.class.getName()); + + EasyMock.expect(connector.version()).andReturn("1.0"); + EasyMock.expect(plugins.compareAndSwapLoaders(connector)) + .andReturn(delegatingLoader) + .times(3); + connector.initialize(anyObject(ConnectorContext.class)); + EasyMock.expectLastCall(); + connector.start(connectorProps); + EasyMock.expectLastCall(); + + EasyMock.expect(Plugins.compareAndSwapLoaders(delegatingLoader)) + .andReturn(pluginLoader) + .times(3); + + connectorStatusListener.onStartup(CONNECTOR_ID); + EasyMock.expectLastCall(); + + // Reconfigure + EasyMock.>expect(connector.taskClass()).andReturn(TestSourceTask.class); + Map taskProps = new HashMap<>(); + taskProps.put("foo", "bar"); + EasyMock.expect(connector.taskConfigs(2)).andReturn(Arrays.asList(taskProps, taskProps)); + + // Remove + connector.stop(); + EasyMock.expectLastCall(); + + connectorStatusListener.onShutdown(CONNECTOR_ID); + EasyMock.expectLastCall(); + + expectStopStorage(); + + PowerMock.replayAll(); + + worker = new Worker(WORKER_ID, new MockTime(), plugins, config, offsetBackingStore, noneConnectorClientConfigOverridePolicy); + worker.herder = herder; + worker.start(); + + assertStatistics(worker, 0, 0); + assertEquals(Collections.emptySet(), worker.connectorNames()); + worker.startConnector(CONNECTOR_ID, connectorProps, ctx, connectorStatusListener, TargetState.STARTED); + assertStatistics(worker, 1, 0); + assertEquals(new HashSet<>(Arrays.asList(CONNECTOR_ID)), worker.connectorNames()); + try { + worker.startConnector(CONNECTOR_ID, connectorProps, ctx, connectorStatusListener, TargetState.STARTED); + fail("Should have thrown exception when trying to add connector with same name."); + } catch (ConnectException e) { + // expected + } + Map connProps = new HashMap<>(connectorProps); + connProps.put(ConnectorConfig.TASKS_MAX_CONFIG, "2"); + ConnectorConfig connConfig = new SinkConnectorConfig(plugins, connProps); + List> taskConfigs = worker.connectorTaskConfigs(CONNECTOR_ID, connConfig); + Map expectedTaskProps = new HashMap<>(); + expectedTaskProps.put("foo", "bar"); + expectedTaskProps.put(TaskConfig.TASK_CLASS_CONFIG, TestSourceTask.class.getName()); + expectedTaskProps.put(SinkTask.TOPICS_CONFIG, "foo,bar"); + assertEquals(2, taskConfigs.size()); + assertEquals(expectedTaskProps, taskConfigs.get(0)); + assertEquals(expectedTaskProps, taskConfigs.get(1)); + assertStatistics(worker, 1, 0); + assertStartupStatistics(worker, 1, 0, 0, 0); + worker.stopConnector(CONNECTOR_ID); + assertStatistics(worker, 0, 0); + assertStartupStatistics(worker, 1, 0, 0, 0); + assertEquals(Collections.emptySet(), worker.connectorNames()); + // Nothing should be left, so this should effectively be a nop + worker.stop(); + assertStatistics(worker, 0, 0); + + PowerMock.verifyAll(); + } + + + @Test + public void testAddRemoveTask() throws Exception { + expectConverters(); + expectStartStorage(); + expectFileConfigProvider(); + + EasyMock.expect(workerTask.id()).andStubReturn(TASK_ID); + + EasyMock.expect(plugins.currentThreadLoader()).andReturn(delegatingLoader).times(2); + expectNewWorkerTask(); + Map origProps = new HashMap<>(); + origProps.put(TaskConfig.TASK_CLASS_CONFIG, TestSourceTask.class.getName()); + + TaskConfig taskConfig = new TaskConfig(origProps); + // We should expect this call, but the pluginLoader being swapped in is only mocked. + // EasyMock.expect(pluginLoader.loadClass(TestSourceTask.class.getName())) + // .andReturn((Class) TestSourceTask.class); + EasyMock.expect(plugins.newTask(TestSourceTask.class)).andReturn(task); + EasyMock.expect(task.version()).andReturn("1.0"); + + workerTask.initialize(taskConfig); + EasyMock.expectLastCall(); + + // Expect that the worker will create converters and will find them using the current classloader ... + assertNotNull(taskKeyConverter); + assertNotNull(taskValueConverter); + assertNotNull(taskHeaderConverter); + expectTaskKeyConverters(ClassLoaderUsage.CURRENT_CLASSLOADER, taskKeyConverter); + expectTaskValueConverters(ClassLoaderUsage.CURRENT_CLASSLOADER, taskValueConverter); + expectTaskHeaderConverter(ClassLoaderUsage.CURRENT_CLASSLOADER, taskHeaderConverter); + + EasyMock.expect(executorService.submit(workerTask)).andReturn(null); + + EasyMock.expect(plugins.delegatingLoader()).andReturn(delegatingLoader); + EasyMock.expect(delegatingLoader.connectorLoader(WorkerTestConnector.class.getName())) + .andReturn(pluginLoader); + EasyMock.expect(Plugins.compareAndSwapLoaders(pluginLoader)).andReturn(delegatingLoader) + .times(2); + + EasyMock.expect(workerTask.loader()).andReturn(pluginLoader); + + EasyMock.expect(Plugins.compareAndSwapLoaders(delegatingLoader)).andReturn(pluginLoader) + .times(2); + plugins.connectorClass(WorkerTestConnector.class.getName()); + EasyMock.expectLastCall().andReturn(WorkerTestConnector.class); + // Remove + workerTask.stop(); + EasyMock.expectLastCall(); + EasyMock.expect(workerTask.awaitStop(EasyMock.anyLong())).andStubReturn(true); + EasyMock.expectLastCall(); + + expectStopStorage(); + + PowerMock.replayAll(); + + worker = new Worker(WORKER_ID, new MockTime(), plugins, config, offsetBackingStore, executorService, + noneConnectorClientConfigOverridePolicy); + worker.herder = herder; + worker.start(); + assertStatistics(worker, 0, 0); + assertStartupStatistics(worker, 0, 0, 0, 0); + assertEquals(Collections.emptySet(), worker.taskIds()); + worker.startTask(TASK_ID, ClusterConfigState.EMPTY, anyConnectorConfigMap(), origProps, taskStatusListener, TargetState.STARTED); + assertStatistics(worker, 0, 1); + assertStartupStatistics(worker, 0, 0, 1, 0); + assertEquals(new HashSet<>(Arrays.asList(TASK_ID)), worker.taskIds()); + worker.stopAndAwaitTask(TASK_ID); + assertStatistics(worker, 0, 0); + assertStartupStatistics(worker, 0, 0, 1, 0); + assertEquals(Collections.emptySet(), worker.taskIds()); + // Nothing should be left, so this should effectively be a nop + worker.stop(); + assertStatistics(worker, 0, 0); + assertStartupStatistics(worker, 0, 0, 1, 0); + + PowerMock.verifyAll(); + } + + @Test + public void testTaskStatusMetricsStatuses() throws Exception { + expectConverters(); + expectStartStorage(); + expectFileConfigProvider(); + + EasyMock.expect(workerTask.id()).andStubReturn(TASK_ID); + + EasyMock.expect(plugins.currentThreadLoader()).andReturn(delegatingLoader).times(2); + expectNewWorkerTask(); + Map origProps = new HashMap<>(); + origProps.put(TaskConfig.TASK_CLASS_CONFIG, TestSourceTask.class.getName()); + + TaskConfig taskConfig = new TaskConfig(origProps); + // We should expect this call, but the pluginLoader being swapped in is only mocked. + // EasyMock.expect(pluginLoader.loadClass(TestSourceTask.class.getName())) + // .andReturn((Class) TestSourceTask.class); + EasyMock.expect(plugins.newTask(TestSourceTask.class)).andReturn(task); + EasyMock.expect(task.version()).andReturn("1.0"); + + workerTask.initialize(taskConfig); + EasyMock.expectLastCall(); + + // Expect that the worker will create converters and will find them using the current classloader ... + assertNotNull(taskKeyConverter); + assertNotNull(taskValueConverter); + assertNotNull(taskHeaderConverter); + expectTaskKeyConverters(ClassLoaderUsage.CURRENT_CLASSLOADER, taskKeyConverter); + expectTaskValueConverters(ClassLoaderUsage.CURRENT_CLASSLOADER, taskValueConverter); + expectTaskHeaderConverter(ClassLoaderUsage.CURRENT_CLASSLOADER, taskHeaderConverter); + + EasyMock.expect(executorService.submit(workerTask)).andReturn(null); + + EasyMock.expect(plugins.delegatingLoader()).andReturn(delegatingLoader); + EasyMock.expect(delegatingLoader.connectorLoader(WorkerTestConnector.class.getName())) + .andReturn(pluginLoader); + EasyMock.expect(Plugins.compareAndSwapLoaders(pluginLoader)).andReturn(delegatingLoader) + .times(2); + + EasyMock.expect(Plugins.compareAndSwapLoaders(delegatingLoader)).andReturn(pluginLoader) + .times(2); + plugins.connectorClass(WorkerTestConnector.class.getName()); + EasyMock.expectLastCall().andReturn(WorkerTestConnector.class); + + EasyMock.expect(workerTask.awaitStop(EasyMock.anyLong())).andStubReturn(true); + EasyMock.expectLastCall(); + + // Each time we check the task metrics, the worker will call the herder + herder.taskStatus(TASK_ID); + EasyMock.expectLastCall() + .andReturn(new ConnectorStateInfo.TaskState(0, "RUNNING", "worker", "msg")); + + herder.taskStatus(TASK_ID); + EasyMock.expectLastCall() + .andReturn(new ConnectorStateInfo.TaskState(0, "PAUSED", "worker", "msg")); + + herder.taskStatus(TASK_ID); + EasyMock.expectLastCall() + .andReturn(new ConnectorStateInfo.TaskState(0, "FAILED", "worker", "msg")); + + herder.taskStatus(TASK_ID); + EasyMock.expectLastCall() + .andReturn(new ConnectorStateInfo.TaskState(0, "DESTROYED", "worker", "msg")); + + herder.taskStatus(TASK_ID); + EasyMock.expectLastCall() + .andReturn(new ConnectorStateInfo.TaskState(0, "UNASSIGNED", "worker", "msg")); + + // Called when we stop the worker + EasyMock.expect(workerTask.loader()).andReturn(pluginLoader); + workerTask.stop(); + EasyMock.expectLastCall(); + + PowerMock.replayAll(); + + worker = new Worker(WORKER_ID, + new MockTime(), + plugins, + config, + offsetBackingStore, + executorService, + noneConnectorClientConfigOverridePolicy); + + worker.herder = herder; + + worker.start(); + assertStatistics(worker, 0, 0); + assertStartupStatistics(worker, 0, 0, 0, 0); + assertEquals(Collections.emptySet(), worker.taskIds()); + worker.startTask( + TASK_ID, + ClusterConfigState.EMPTY, + anyConnectorConfigMap(), + origProps, + taskStatusListener, + TargetState.STARTED); + + assertStatusMetrics(1L, "connector-running-task-count"); + assertStatusMetrics(1L, "connector-paused-task-count"); + assertStatusMetrics(1L, "connector-failed-task-count"); + assertStatusMetrics(1L, "connector-destroyed-task-count"); + assertStatusMetrics(1L, "connector-unassigned-task-count"); + + worker.stopAndAwaitTask(TASK_ID); + assertStatusMetrics(0L, "connector-running-task-count"); + assertStatusMetrics(0L, "connector-paused-task-count"); + assertStatusMetrics(0L, "connector-failed-task-count"); + assertStatusMetrics(0L, "connector-destroyed-task-count"); + assertStatusMetrics(0L, "connector-unassigned-task-count"); + + PowerMock.verifyAll(); + } + + @Test + public void testConnectorStatusMetricsGroup_taskStatusCounter() { + ConcurrentMap tasks = new ConcurrentHashMap<>(); + tasks.put(new ConnectorTaskId("c1", 0), workerTask); + tasks.put(new ConnectorTaskId("c1", 1), workerTask); + tasks.put(new ConnectorTaskId("c2", 0), workerTask); + + expectConverters(); + expectStartStorage(); + expectFileConfigProvider(); + + EasyMock.expect(Plugins.compareAndSwapLoaders(pluginLoader)).andReturn(delegatingLoader); + EasyMock.expect(Plugins.compareAndSwapLoaders(pluginLoader)).andReturn(delegatingLoader); + + EasyMock.expect(Plugins.compareAndSwapLoaders(delegatingLoader)).andReturn(pluginLoader); + + taskStatusListener.onFailure(EasyMock.eq(TASK_ID), EasyMock.anyObject()); + EasyMock.expectLastCall(); + + PowerMock.replayAll(); + + worker = new Worker(WORKER_ID, + new MockTime(), + plugins, + config, + offsetBackingStore, + noneConnectorClientConfigOverridePolicy); + worker.herder = herder; + + Worker.ConnectorStatusMetricsGroup metricGroup = new Worker.ConnectorStatusMetricsGroup( + worker.metrics(), tasks, herder + ); + assertEquals(2L, (long) metricGroup.taskCounter("c1").metricValue(0L)); + assertEquals(1L, (long) metricGroup.taskCounter("c2").metricValue(0L)); + assertEquals(0L, (long) metricGroup.taskCounter("fakeConnector").metricValue(0L)); + } + + @Test + public void testStartTaskFailure() { + expectConverters(); + expectStartStorage(); + expectFileConfigProvider(); + + Map origProps = new HashMap<>(); + origProps.put(TaskConfig.TASK_CLASS_CONFIG, "missing.From.This.Workers.Classpath"); + + EasyMock.expect(plugins.currentThreadLoader()).andReturn(delegatingLoader); + EasyMock.expect(plugins.delegatingLoader()).andReturn(delegatingLoader); + EasyMock.expect(delegatingLoader.connectorLoader(WorkerTestConnector.class.getName())) + .andReturn(pluginLoader); + + // We would normally expect this since the plugin loader would have been swapped in. However, since we mock out + // all classloader changes, the call actually goes to the normal default classloader. However, this works out + // fine since we just wanted a ClassNotFoundException anyway. + // EasyMock.expect(pluginLoader.loadClass(origProps.get(TaskConfig.TASK_CLASS_CONFIG))) + // .andThrow(new ClassNotFoundException()); + + EasyMock.expect(Plugins.compareAndSwapLoaders(pluginLoader)) + .andReturn(delegatingLoader); + + EasyMock.expect(Plugins.compareAndSwapLoaders(delegatingLoader)) + .andReturn(pluginLoader); + + taskStatusListener.onFailure(EasyMock.eq(TASK_ID), EasyMock.anyObject()); + EasyMock.expectLastCall(); + + PowerMock.replayAll(); + + worker = new Worker(WORKER_ID, new MockTime(), plugins, config, offsetBackingStore, noneConnectorClientConfigOverridePolicy); + worker.herder = herder; + worker.start(); + assertStatistics(worker, 0, 0); + assertStartupStatistics(worker, 0, 0, 0, 0); + + assertFalse(worker.startTask(TASK_ID, ClusterConfigState.EMPTY, anyConnectorConfigMap(), origProps, taskStatusListener, TargetState.STARTED)); + assertStartupStatistics(worker, 0, 0, 1, 1); + + assertStatistics(worker, 0, 0); + assertStartupStatistics(worker, 0, 0, 1, 1); + assertEquals(Collections.emptySet(), worker.taskIds()); + + PowerMock.verifyAll(); + } + + @Test + public void testCleanupTasksOnStop() throws Exception { + expectConverters(); + expectStartStorage(); + expectFileConfigProvider(); + + EasyMock.expect(workerTask.id()).andStubReturn(TASK_ID); + + EasyMock.expect(plugins.currentThreadLoader()).andReturn(delegatingLoader).times(2); + expectNewWorkerTask(); + Map origProps = new HashMap<>(); + origProps.put(TaskConfig.TASK_CLASS_CONFIG, TestSourceTask.class.getName()); + + TaskConfig taskConfig = new TaskConfig(origProps); + // We should expect this call, but the pluginLoader being swapped in is only mocked. + // EasyMock.expect(pluginLoader.loadClass(TestSourceTask.class.getName())) + // .andReturn((Class) TestSourceTask.class); + EasyMock.expect(plugins.newTask(TestSourceTask.class)).andReturn(task); + EasyMock.expect(task.version()).andReturn("1.0"); + + workerTask.initialize(taskConfig); + EasyMock.expectLastCall(); + + // Expect that the worker will create converters and will not initially find them using the current classloader ... + assertNotNull(taskKeyConverter); + assertNotNull(taskValueConverter); + assertNotNull(taskHeaderConverter); + expectTaskKeyConverters(ClassLoaderUsage.CURRENT_CLASSLOADER, null); + expectTaskKeyConverters(ClassLoaderUsage.PLUGINS, taskKeyConverter); + expectTaskValueConverters(ClassLoaderUsage.CURRENT_CLASSLOADER, null); + expectTaskValueConverters(ClassLoaderUsage.PLUGINS, taskValueConverter); + expectTaskHeaderConverter(ClassLoaderUsage.CURRENT_CLASSLOADER, null); + expectTaskHeaderConverter(ClassLoaderUsage.PLUGINS, taskHeaderConverter); + + EasyMock.expect(executorService.submit(workerTask)).andReturn(null); + + EasyMock.expect(plugins.delegatingLoader()).andReturn(delegatingLoader); + EasyMock.expect(delegatingLoader.connectorLoader(WorkerTestConnector.class.getName())) + .andReturn(pluginLoader); + + EasyMock.expect(Plugins.compareAndSwapLoaders(pluginLoader)).andReturn(delegatingLoader) + .times(2); + + EasyMock.expect(workerTask.loader()).andReturn(pluginLoader); + + EasyMock.expect(Plugins.compareAndSwapLoaders(delegatingLoader)).andReturn(pluginLoader) + .times(2); + plugins.connectorClass(WorkerTestConnector.class.getName()); + EasyMock.expectLastCall().andReturn(WorkerTestConnector.class); + // Remove on Worker.stop() + workerTask.stop(); + EasyMock.expectLastCall(); + + EasyMock.expect(workerTask.awaitStop(EasyMock.anyLong())).andReturn(true); + // Note that in this case we *do not* commit offsets since it's an unclean shutdown + EasyMock.expectLastCall(); + + expectStopStorage(); + + PowerMock.replayAll(); + + worker = new Worker(WORKER_ID, new MockTime(), plugins, config, offsetBackingStore, executorService, + noneConnectorClientConfigOverridePolicy); + worker.herder = herder; + worker.start(); + assertStatistics(worker, 0, 0); + worker.startTask(TASK_ID, ClusterConfigState.EMPTY, anyConnectorConfigMap(), origProps, taskStatusListener, TargetState.STARTED); + assertStatistics(worker, 0, 1); + worker.stop(); + assertStatistics(worker, 0, 0); + + PowerMock.verifyAll(); + } + + @Test + public void testConverterOverrides() throws Exception { + expectConverters(); + expectStartStorage(); + expectFileConfigProvider(); + + EasyMock.expect(workerTask.id()).andStubReturn(TASK_ID); + + EasyMock.expect(plugins.currentThreadLoader()).andReturn(delegatingLoader).times(2); + expectNewWorkerTask(); + Map origProps = new HashMap<>(); + origProps.put(TaskConfig.TASK_CLASS_CONFIG, TestSourceTask.class.getName()); + + TaskConfig taskConfig = new TaskConfig(origProps); + // We should expect this call, but the pluginLoader being swapped in is only mocked. + // EasyMock.expect(pluginLoader.loadClass(TestSourceTask.class.getName())) + // .andReturn((Class) TestSourceTask.class); + EasyMock.expect(plugins.newTask(TestSourceTask.class)).andReturn(task); + EasyMock.expect(task.version()).andReturn("1.0"); + + workerTask.initialize(taskConfig); + EasyMock.expectLastCall(); + + // Expect that the worker will create converters and will not initially find them using the current classloader ... + assertNotNull(taskKeyConverter); + assertNotNull(taskValueConverter); + assertNotNull(taskHeaderConverter); + expectTaskKeyConverters(ClassLoaderUsage.CURRENT_CLASSLOADER, null); + expectTaskKeyConverters(ClassLoaderUsage.PLUGINS, taskKeyConverter); + expectTaskValueConverters(ClassLoaderUsage.CURRENT_CLASSLOADER, null); + expectTaskValueConverters(ClassLoaderUsage.PLUGINS, taskValueConverter); + expectTaskHeaderConverter(ClassLoaderUsage.CURRENT_CLASSLOADER, null); + expectTaskHeaderConverter(ClassLoaderUsage.PLUGINS, taskHeaderConverter); + + EasyMock.expect(executorService.submit(workerTask)).andReturn(null); + + EasyMock.expect(plugins.delegatingLoader()).andReturn(delegatingLoader); + EasyMock.expect(delegatingLoader.connectorLoader(WorkerTestConnector.class.getName())) + .andReturn(pluginLoader); + + EasyMock.expect(Plugins.compareAndSwapLoaders(pluginLoader)).andReturn(delegatingLoader) + .times(2); + + EasyMock.expect(workerTask.loader()).andReturn(pluginLoader); + + EasyMock.expect(Plugins.compareAndSwapLoaders(delegatingLoader)).andReturn(pluginLoader) + .times(2); + plugins.connectorClass(WorkerTestConnector.class.getName()); + EasyMock.expectLastCall().andReturn(WorkerTestConnector.class); + + // Remove + workerTask.stop(); + EasyMock.expectLastCall(); + EasyMock.expect(workerTask.awaitStop(EasyMock.anyLong())).andStubReturn(true); + EasyMock.expectLastCall(); + + expectStopStorage(); + + PowerMock.replayAll(); + + worker = new Worker(WORKER_ID, new MockTime(), plugins, config, offsetBackingStore, executorService, + noneConnectorClientConfigOverridePolicy); + worker.herder = herder; + worker.start(); + assertStatistics(worker, 0, 0); + assertEquals(Collections.emptySet(), worker.taskIds()); + Map connProps = anyConnectorConfigMap(); + connProps.put(ConnectorConfig.KEY_CONVERTER_CLASS_CONFIG, TestConverter.class.getName()); + connProps.put("key.converter.extra.config", "foo"); + connProps.put(ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG, TestConfigurableConverter.class.getName()); + connProps.put("value.converter.extra.config", "bar"); + worker.startTask(TASK_ID, ClusterConfigState.EMPTY, connProps, origProps, taskStatusListener, TargetState.STARTED); + assertStatistics(worker, 0, 1); + assertEquals(new HashSet<>(Arrays.asList(TASK_ID)), worker.taskIds()); + worker.stopAndAwaitTask(TASK_ID); + assertStatistics(worker, 0, 0); + assertEquals(Collections.emptySet(), worker.taskIds()); + // Nothing should be left, so this should effectively be a nop + worker.stop(); + assertStatistics(worker, 0, 0); + + // We've mocked the Plugin.newConverter method, so we don't currently configure the converters + + PowerMock.verifyAll(); + } + + @Test + public void testProducerConfigsWithoutOverrides() { + EasyMock.expect(connectorConfig.originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_PRODUCER_OVERRIDES_PREFIX)).andReturn( + new HashMap()); + PowerMock.replayAll(); + Map expectedConfigs = new HashMap<>(defaultProducerConfigs); + expectedConfigs.put("client.id", "connector-producer-job-0"); + assertEquals(expectedConfigs, + Worker.producerConfigs(TASK_ID, "connector-producer-" + TASK_ID, config, connectorConfig, null, noneConnectorClientConfigOverridePolicy)); + } + + @Test + public void testProducerConfigsWithOverrides() { + Map props = new HashMap<>(workerProps); + props.put("producer.acks", "-1"); + props.put("producer.linger.ms", "1000"); + props.put("producer.client.id", "producer-test-id"); + WorkerConfig configWithOverrides = new StandaloneConfig(props); + + Map expectedConfigs = new HashMap<>(defaultProducerConfigs); + expectedConfigs.put("acks", "-1"); + expectedConfigs.put("linger.ms", "1000"); + expectedConfigs.put("client.id", "producer-test-id"); + EasyMock.expect(connectorConfig.originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_PRODUCER_OVERRIDES_PREFIX)).andReturn( + new HashMap()); + PowerMock.replayAll(); + assertEquals(expectedConfigs, + Worker.producerConfigs(TASK_ID, "connector-producer-" + TASK_ID, configWithOverrides, connectorConfig, null, allConnectorClientConfigOverridePolicy)); + } + + @Test + public void testProducerConfigsWithClientOverrides() { + Map props = new HashMap<>(workerProps); + props.put("producer.acks", "-1"); + props.put("producer.linger.ms", "1000"); + props.put("producer.client.id", "producer-test-id"); + WorkerConfig configWithOverrides = new StandaloneConfig(props); + + Map expectedConfigs = new HashMap<>(defaultProducerConfigs); + expectedConfigs.put("acks", "-1"); + expectedConfigs.put("linger.ms", "5000"); + expectedConfigs.put("batch.size", "1000"); + expectedConfigs.put("client.id", "producer-test-id"); + Map connConfig = new HashMap(); + connConfig.put("linger.ms", "5000"); + connConfig.put("batch.size", "1000"); + EasyMock.expect(connectorConfig.originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_PRODUCER_OVERRIDES_PREFIX)) + .andReturn(connConfig); + PowerMock.replayAll(); + assertEquals(expectedConfigs, + Worker.producerConfigs(TASK_ID, "connector-producer-" + TASK_ID, configWithOverrides, connectorConfig, null, allConnectorClientConfigOverridePolicy)); + } + + @Test + public void testConsumerConfigsWithoutOverrides() { + Map expectedConfigs = new HashMap<>(defaultConsumerConfigs); + expectedConfigs.put("group.id", "connect-test"); + expectedConfigs.put("client.id", "connector-consumer-test-1"); + EasyMock.expect(connectorConfig.originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_CONSUMER_OVERRIDES_PREFIX)).andReturn(new HashMap<>()); + PowerMock.replayAll(); + assertEquals(expectedConfigs, Worker.consumerConfigs(new ConnectorTaskId("test", 1), config, connectorConfig, + null, noneConnectorClientConfigOverridePolicy)); + } + + @Test + public void testConsumerConfigsWithOverrides() { + Map props = new HashMap<>(workerProps); + props.put("consumer.auto.offset.reset", "latest"); + props.put("consumer.max.poll.records", "1000"); + props.put("consumer.client.id", "consumer-test-id"); + WorkerConfig configWithOverrides = new StandaloneConfig(props); + + Map expectedConfigs = new HashMap<>(defaultConsumerConfigs); + expectedConfigs.put("group.id", "connect-test"); + expectedConfigs.put("auto.offset.reset", "latest"); + expectedConfigs.put("max.poll.records", "1000"); + expectedConfigs.put("client.id", "consumer-test-id"); + EasyMock.expect(connectorConfig.originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_CONSUMER_OVERRIDES_PREFIX)).andReturn(new HashMap<>()); + PowerMock.replayAll(); + assertEquals(expectedConfigs, Worker.consumerConfigs(new ConnectorTaskId("test", 1), configWithOverrides, connectorConfig, + null, noneConnectorClientConfigOverridePolicy)); + + } + + @Test + public void testConsumerConfigsWithClientOverrides() { + Map props = new HashMap<>(workerProps); + props.put("consumer.auto.offset.reset", "latest"); + props.put("consumer.max.poll.records", "5000"); + WorkerConfig configWithOverrides = new StandaloneConfig(props); + + Map expectedConfigs = new HashMap<>(defaultConsumerConfigs); + expectedConfigs.put("group.id", "connect-test"); + expectedConfigs.put("auto.offset.reset", "latest"); + expectedConfigs.put("max.poll.records", "5000"); + expectedConfigs.put("max.poll.interval.ms", "1000"); + expectedConfigs.put("client.id", "connector-consumer-test-1"); + Map connConfig = new HashMap(); + connConfig.put("max.poll.records", "5000"); + connConfig.put("max.poll.interval.ms", "1000"); + EasyMock.expect(connectorConfig.originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_CONSUMER_OVERRIDES_PREFIX)) + .andReturn(connConfig); + PowerMock.replayAll(); + assertEquals(expectedConfigs, Worker.consumerConfigs(new ConnectorTaskId("test", 1), configWithOverrides, connectorConfig, + null, allConnectorClientConfigOverridePolicy)); + } + + @Test(expected = ConnectException.class) + public void testConsumerConfigsClientOverridesWithNonePolicy() { + Map props = new HashMap<>(workerProps); + props.put("consumer.auto.offset.reset", "latest"); + props.put("consumer.max.poll.records", "5000"); + WorkerConfig configWithOverrides = new StandaloneConfig(props); + + Map connConfig = new HashMap(); + connConfig.put("max.poll.records", "5000"); + connConfig.put("max.poll.interval.ms", "1000"); + EasyMock.expect(connectorConfig.originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_CONSUMER_OVERRIDES_PREFIX)) + .andReturn(connConfig); + PowerMock.replayAll(); + Worker.consumerConfigs(new ConnectorTaskId("test", 1), configWithOverrides, connectorConfig, + null, noneConnectorClientConfigOverridePolicy); + } + + @Test + public void testAdminConfigsClientOverridesWithAllPolicy() { + Map props = new HashMap<>(workerProps); + props.put("admin.client.id", "testid"); + props.put("admin.metadata.max.age.ms", "5000"); + props.put("producer.bootstrap.servers", "cbeauho.com"); + props.put("consumer.bootstrap.servers", "localhost:4761"); + WorkerConfig configWithOverrides = new StandaloneConfig(props); + + Map connConfig = new HashMap(); + connConfig.put("metadata.max.age.ms", "10000"); + + Map expectedConfigs = new HashMap<>(workerProps); + + expectedConfigs.put("bootstrap.servers", "localhost:9092"); + expectedConfigs.put("client.id", "testid"); + expectedConfigs.put("metadata.max.age.ms", "10000"); + + EasyMock.expect(connectorConfig.originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_ADMIN_OVERRIDES_PREFIX)) + .andReturn(connConfig); + PowerMock.replayAll(); + assertEquals(expectedConfigs, Worker.adminConfigs(new ConnectorTaskId("test", 1), "", configWithOverrides, connectorConfig, + null, allConnectorClientConfigOverridePolicy)); + } + + @Test(expected = ConnectException.class) + public void testAdminConfigsClientOverridesWithNonePolicy() { + Map props = new HashMap<>(workerProps); + props.put("admin.client.id", "testid"); + props.put("admin.metadata.max.age.ms", "5000"); + WorkerConfig configWithOverrides = new StandaloneConfig(props); + + Map connConfig = new HashMap(); + connConfig.put("metadata.max.age.ms", "10000"); + + EasyMock.expect(connectorConfig.originalsWithPrefix(ConnectorConfig.CONNECTOR_CLIENT_ADMIN_OVERRIDES_PREFIX)) + .andReturn(connConfig); + PowerMock.replayAll(); + Worker.adminConfigs(new ConnectorTaskId("test", 1), "", configWithOverrides, connectorConfig, + null, noneConnectorClientConfigOverridePolicy); + } + + private void assertStatusMetrics(long expected, String metricName) { + MetricGroup statusMetrics = worker.connectorStatusMetricsGroup().metricGroup(TASK_ID.connector()); + if (expected == 0L) { + assertNull(statusMetrics); + return; + } + assertEquals(expected, MockConnectMetrics.currentMetricValue(worker.metrics(), statusMetrics, metricName)); + } + + private void assertStatistics(Worker worker, int connectors, int tasks) { + assertStatusMetrics(tasks, "connector-total-task-count"); + MetricGroup workerMetrics = worker.workerMetricsGroup().metricGroup(); + assertEquals(connectors, MockConnectMetrics.currentMetricValueAsDouble(worker.metrics(), workerMetrics, "connector-count"), 0.0001d); + assertEquals(tasks, MockConnectMetrics.currentMetricValueAsDouble(worker.metrics(), workerMetrics, "task-count"), 0.0001d); + assertEquals(tasks, MockConnectMetrics.currentMetricValueAsDouble(worker.metrics(), workerMetrics, "task-count"), 0.0001d); + } + + private void assertStartupStatistics(Worker worker, int connectorStartupAttempts, int connectorStartupFailures, int taskStartupAttempts, int taskStartupFailures) { + double connectStartupSuccesses = connectorStartupAttempts - connectorStartupFailures; + double taskStartupSuccesses = taskStartupAttempts - taskStartupFailures; + double connectStartupSuccessPct = 0.0d; + double connectStartupFailurePct = 0.0d; + double taskStartupSuccessPct = 0.0d; + double taskStartupFailurePct = 0.0d; + if (connectorStartupAttempts != 0) { + connectStartupSuccessPct = connectStartupSuccesses / connectorStartupAttempts; + connectStartupFailurePct = (double) connectorStartupFailures / connectorStartupAttempts; + } + if (taskStartupAttempts != 0) { + taskStartupSuccessPct = taskStartupSuccesses / taskStartupAttempts; + taskStartupFailurePct = (double) taskStartupFailures / taskStartupAttempts; + } + MetricGroup workerMetrics = worker.workerMetricsGroup().metricGroup(); + assertEquals(connectorStartupAttempts, MockConnectMetrics.currentMetricValueAsDouble(worker.metrics(), workerMetrics, "connector-startup-attempts-total"), 0.0001d); + assertEquals(connectStartupSuccesses, MockConnectMetrics.currentMetricValueAsDouble(worker.metrics(), workerMetrics, "connector-startup-success-total"), 0.0001d); + assertEquals(connectorStartupFailures, MockConnectMetrics.currentMetricValueAsDouble(worker.metrics(), workerMetrics, "connector-startup-failure-total"), 0.0001d); + assertEquals(connectStartupSuccessPct, MockConnectMetrics.currentMetricValueAsDouble(worker.metrics(), workerMetrics, "connector-startup-success-percentage"), 0.0001d); + assertEquals(connectStartupFailurePct, MockConnectMetrics.currentMetricValueAsDouble(worker.metrics(), workerMetrics, "connector-startup-failure-percentage"), 0.0001d); + assertEquals(taskStartupAttempts, MockConnectMetrics.currentMetricValueAsDouble(worker.metrics(), workerMetrics, "task-startup-attempts-total"), 0.0001d); + assertEquals(taskStartupSuccesses, MockConnectMetrics.currentMetricValueAsDouble(worker.metrics(), workerMetrics, "task-startup-success-total"), 0.0001d); + assertEquals(taskStartupFailures, MockConnectMetrics.currentMetricValueAsDouble(worker.metrics(), workerMetrics, "task-startup-failure-total"), 0.0001d); + assertEquals(taskStartupSuccessPct, MockConnectMetrics.currentMetricValueAsDouble(worker.metrics(), workerMetrics, "task-startup-success-percentage"), 0.0001d); + assertEquals(taskStartupFailurePct, MockConnectMetrics.currentMetricValueAsDouble(worker.metrics(), workerMetrics, "task-startup-failure-percentage"), 0.0001d); + } + + private void expectStartStorage() { + offsetBackingStore.configure(anyObject(WorkerConfig.class)); + EasyMock.expectLastCall(); + offsetBackingStore.start(); + EasyMock.expectLastCall(); + EasyMock.expect(herder.statusBackingStore()) + .andReturn(statusBackingStore).anyTimes(); + } + + private void expectStopStorage() { + offsetBackingStore.stop(); + EasyMock.expectLastCall(); + } + + private void expectConverters() { + expectConverters(JsonConverter.class, false); + } + + private void expectConverters(Boolean expectDefaultConverters) { + expectConverters(JsonConverter.class, expectDefaultConverters); + } + + @SuppressWarnings("deprecation") + private void expectConverters(Class converterClass, Boolean expectDefaultConverters) { + // As default converters are instantiated when a task starts, they are expected only if the `startTask` method is called + if (expectDefaultConverters) { + + // Instantiate and configure default + EasyMock.expect(plugins.newConverter(config, WorkerConfig.KEY_CONVERTER_CLASS_CONFIG, ClassLoaderUsage.PLUGINS)) + .andReturn(keyConverter); + EasyMock.expect(plugins.newConverter(config, WorkerConfig.VALUE_CONVERTER_CLASS_CONFIG, ClassLoaderUsage.PLUGINS)) + .andReturn(valueConverter); + EasyMock.expectLastCall(); + } + + //internal + Converter internalKeyConverter = PowerMock.createMock(converterClass); + Converter internalValueConverter = PowerMock.createMock(converterClass); + + // Instantiate and configure internal + EasyMock.expect( + plugins.newConverter( + config, + WorkerConfig.INTERNAL_KEY_CONVERTER_CLASS_CONFIG, + ClassLoaderUsage.PLUGINS + ) + ).andReturn(internalKeyConverter); + EasyMock.expect( + plugins.newConverter( + config, + WorkerConfig.INTERNAL_VALUE_CONVERTER_CLASS_CONFIG, + ClassLoaderUsage.PLUGINS + ) + ).andReturn(internalValueConverter); + EasyMock.expectLastCall(); + } + + private void expectTaskKeyConverters(ClassLoaderUsage classLoaderUsage, Converter returning) { + EasyMock.expect( + plugins.newConverter( + anyObject(AbstractConfig.class), + eq(WorkerConfig.KEY_CONVERTER_CLASS_CONFIG), + eq(classLoaderUsage))) + .andReturn(returning); + } + + private void expectTaskValueConverters(ClassLoaderUsage classLoaderUsage, Converter returning) { + EasyMock.expect( + plugins.newConverter( + anyObject(AbstractConfig.class), + eq(WorkerConfig.VALUE_CONVERTER_CLASS_CONFIG), + eq(classLoaderUsage))) + .andReturn(returning); + } + + private void expectTaskHeaderConverter(ClassLoaderUsage classLoaderUsage, HeaderConverter returning) { + EasyMock.expect( + plugins.newHeaderConverter( + anyObject(AbstractConfig.class), + eq(WorkerConfig.HEADER_CONVERTER_CLASS_CONFIG), + eq(classLoaderUsage))) + .andReturn(returning); + } + + private Map anyConnectorConfigMap() { + Map props = new HashMap<>(); + props.put(ConnectorConfig.NAME_CONFIG, CONNECTOR_ID); + props.put(ConnectorConfig.CONNECTOR_CLASS_CONFIG, WorkerTestConnector.class.getName()); + props.put(ConnectorConfig.TASKS_MAX_CONFIG, "1"); + props.put(DEFAULT_TOPIC_CREATION_PREFIX + REPLICATION_FACTOR_CONFIG, String.valueOf(1)); + props.put(DEFAULT_TOPIC_CREATION_PREFIX + PARTITIONS_CONFIG, String.valueOf(1)); + return props; + } + + private void expectNewWorkerTask() throws Exception { + PowerMock.expectNew( + WorkerSourceTask.class, EasyMock.eq(TASK_ID), + EasyMock.eq(task), + anyObject(TaskStatus.Listener.class), + EasyMock.eq(TargetState.STARTED), + anyObject(JsonConverter.class), + anyObject(JsonConverter.class), + anyObject(JsonConverter.class), + EasyMock.eq(new TransformationChain<>(Collections.emptyList(), NOOP_OPERATOR)), + anyObject(KafkaProducer.class), + anyObject(TopicAdmin.class), + EasyMock.>anyObject(), + anyObject(OffsetStorageReader.class), + anyObject(OffsetStorageWriter.class), + EasyMock.eq(config), + anyObject(ClusterConfigState.class), + anyObject(ConnectMetrics.class), + EasyMock.eq(pluginLoader), + anyObject(Time.class), + anyObject(RetryWithToleranceOperator.class), + anyObject(StatusBackingStore.class)) + .andReturn(workerTask); + } + /* Name here needs to be unique as we are testing the aliasing mechanism */ + public static class WorkerTestConnector extends SourceConnector { + + private static final ConfigDef CONFIG_DEF = new ConfigDef() + .define("configName", ConfigDef.Type.STRING, ConfigDef.Importance.HIGH, "Test configName."); + + @Override + public String version() { + return "1.0"; + } + + @Override + public void start(Map props) { + + } + + @Override + public Class taskClass() { + return null; + } + + @Override + public List> taskConfigs(int maxTasks) { + return null; + } + + @Override + public void stop() { + + } + + @Override + public ConfigDef config() { + return CONFIG_DEF; + } + } + + private static class TestSourceTask extends SourceTask { + public TestSourceTask() { + } + + @Override + public String version() { + return "1.0"; + } + + @Override + public void start(Map props) { + } + + @Override + public List poll() throws InterruptedException { + return null; + } + + @Override + public void stop() { + } + } + + public static class TestConverter implements Converter { + public Map configs; + + @Override + public void configure(Map configs, boolean isKey) { + this.configs = configs; + } + + @Override + public byte[] fromConnectData(String topic, Schema schema, Object value) { + return new byte[0]; + } + + @Override + public SchemaAndValue toConnectData(String topic, byte[] value) { + return null; + } + } + + public static class TestConfigurableConverter implements Converter, Configurable { + public Map configs; + + public ConfigDef config() { + return JsonConverterConfig.configDef(); + } + + @Override + public void configure(Map configs) { + this.configs = configs; + new JsonConverterConfig(configs); // requires the `converter.type` config be set + } + + @Override + public void configure(Map configs, boolean isKey) { + this.configs = configs; + } + + @Override + public byte[] fromConnectData(String topic, Schema schema, Object value) { + return new byte[0]; + } + + @Override + public SchemaAndValue toConnectData(String topic, byte[] value) { + return null; + } + } +} diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerderTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerderTest.java index feb0f995b7f45..6f08a4f87419e 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerderTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerderTest.java @@ -81,6 +81,9 @@ import static java.util.Collections.singleton; import static java.util.Collections.singletonList; import static java.util.Collections.singletonMap; +import static org.apache.kafka.connect.runtime.TopicCreationConfig.DEFAULT_TOPIC_CREATION_PREFIX; +import static org.apache.kafka.connect.runtime.TopicCreationConfig.PARTITIONS_CONFIG; +import static org.apache.kafka.connect.runtime.TopicCreationConfig.REPLICATION_FACTOR_CONFIG; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -540,8 +543,9 @@ public void testPutConnectorConfig() throws Exception { EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); EasyMock.expectLastCall().andReturn(true); EasyMock.expect(worker.isRunning(CONNECTOR_NAME)).andReturn(true); + EasyMock.expect(worker.isTopicCreationEnabled()).andReturn(true); // Generate same task config, which should result in no additional action to restart tasks - EasyMock.expect(worker.connectorTaskConfigs(CONNECTOR_NAME, new SourceConnectorConfig(plugins, newConnConfig))) + EasyMock.expect(worker.connectorTaskConfigs(CONNECTOR_NAME, new SourceConnectorConfig(plugins, newConnConfig, true))) .andReturn(singletonList(taskConfig(SourceSink.SOURCE))); worker.isSinkConnector(CONNECTOR_NAME); EasyMock.expectLastCall().andReturn(false); @@ -629,13 +633,16 @@ public void testCorruptConfig() { private void expectAdd(SourceSink sourceSink) { Map connectorProps = connectorConfig(sourceSink); ConnectorConfig connConfig = sourceSink == SourceSink.SOURCE ? - new SourceConnectorConfig(plugins, connectorProps) : + new SourceConnectorConfig(plugins, connectorProps, true) : new SinkConnectorConfig(plugins, connectorProps); worker.startConnector(EasyMock.eq(CONNECTOR_NAME), EasyMock.eq(connectorProps), EasyMock.anyObject(HerderConnectorContext.class), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); EasyMock.expectLastCall().andReturn(true); EasyMock.expect(worker.isRunning(CONNECTOR_NAME)).andReturn(true); + if (sourceSink == SourceSink.SOURCE) { + EasyMock.expect(worker.isTopicCreationEnabled()).andReturn(true); + } ConnectorInfo connInfo = new ConnectorInfo(CONNECTOR_NAME, connectorProps, Arrays.asList(new ConnectorTaskId(CONNECTOR_NAME, 0)), SourceSink.SOURCE == sourceSink ? ConnectorType.SOURCE : ConnectorType.SINK); @@ -687,8 +694,12 @@ private static Map connectorConfig(SourceSink sourceSink) { Class connectorClass = sourceSink == SourceSink.SINK ? BogusSinkConnector.class : BogusSourceConnector.class; props.put(ConnectorConfig.CONNECTOR_CLASS_CONFIG, connectorClass.getName()); props.put(ConnectorConfig.TASKS_MAX_CONFIG, "1"); - if (sourceSink == SourceSink.SINK) + if (sourceSink == SourceSink.SINK) { props.put(SinkTask.TOPICS_CONFIG, TOPICS_LIST_STR); + } else { + props.put(DEFAULT_TOPIC_CREATION_PREFIX + REPLICATION_FACTOR_CONFIG, String.valueOf(1)); + props.put(DEFAULT_TOPIC_CREATION_PREFIX + PARTITIONS_CONFIG, String.valueOf(1)); + } return props; } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/util/TopicAdminTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/util/TopicAdminTest.java index 6a19c418b15db..caaa63922ff66 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/util/TopicAdminTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/util/TopicAdminTest.java @@ -17,22 +17,28 @@ package org.apache.kafka.connect.util; import org.apache.kafka.clients.NodeApiVersions; +import org.apache.kafka.clients.admin.AdminClientUnitTestEnv; import org.apache.kafka.clients.admin.DescribeTopicsResult; import org.apache.kafka.clients.admin.MockAdminClient; -import org.apache.kafka.clients.admin.AdminClientUnitTestEnv; import org.apache.kafka.clients.admin.NewTopic; import org.apache.kafka.clients.admin.TopicDescription; -import org.apache.kafka.common.KafkaFuture; -import org.apache.kafka.common.message.CreateTopicsResponseData; -import org.apache.kafka.common.message.CreateTopicsResponseData.CreatableTopicResult; import org.apache.kafka.common.Cluster; +import org.apache.kafka.common.KafkaFuture; import org.apache.kafka.common.Node; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartitionInfo; +import org.apache.kafka.common.errors.ClusterAuthorizationException; +import org.apache.kafka.common.errors.TopicAuthorizationException; +import org.apache.kafka.common.errors.UnsupportedVersionException; +import org.apache.kafka.common.message.CreateTopicsResponseData; +import org.apache.kafka.common.message.CreateTopicsResponseData.CreatableTopicResult; +import org.apache.kafka.common.message.MetadataResponseData; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.requests.ApiError; import org.apache.kafka.common.requests.CreateTopicsResponse; +import org.apache.kafka.common.requests.MetadataResponse; import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.connect.errors.ConnectException; import org.junit.Test; import java.util.Collections; @@ -41,8 +47,10 @@ import java.util.Set; import java.util.concurrent.ExecutionException; +import static org.apache.kafka.common.message.MetadataResponseData.MetadataResponseTopic; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertThrows; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -54,7 +62,7 @@ public class TopicAdminTest { * create no topics, and return false. */ @Test - public void returnNullWithApiVersionMismatch() { + public void returnNullWithApiVersionMismatchOnCreate() { final NewTopic newTopic = TopicAdmin.defineTopic("myTopic").partitions(1).compacted().build(); Cluster cluster = createCluster(1); try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(new MockTime(), cluster)) { @@ -66,8 +74,26 @@ public void returnNullWithApiVersionMismatch() { } } + /** + * 0.11.0.0 clients can talk with older brokers, but the DESCRIBE_TOPIC API was added in 0.10.0.0. That means, + * if our TopicAdmin talks to a pre 0.10.0 broker, it should receive an UnsupportedVersionException, should + * create no topics, and return false. + */ @Test - public void returnNullWithClusterAuthorizationFailure() { + public void throwsWithApiVersionMismatchOnDescribe() { + final NewTopic newTopic = TopicAdmin.defineTopic("myTopic").partitions(1).compacted().build(); + Cluster cluster = createCluster(1); + try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(new MockTime(), cluster)) { + env.kafkaClient().setNodeApiVersions(NodeApiVersions.create()); + env.kafkaClient().prepareResponse(describeTopicResponseWithUnsupportedVersion(newTopic)); + TopicAdmin admin = new TopicAdmin(null, env.adminClient()); + Exception e = assertThrows(ConnectException.class, () -> admin.describeTopics(newTopic.name())); + assertTrue(e.getCause() instanceof UnsupportedVersionException); + } + } + + @Test + public void returnNullWithClusterAuthorizationFailureOnCreate() { final NewTopic newTopic = TopicAdmin.defineTopic("myTopic").partitions(1).compacted().build(); Cluster cluster = createCluster(1); try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(new MockTime(), cluster)) { @@ -79,7 +105,19 @@ public void returnNullWithClusterAuthorizationFailure() { } @Test - public void returnNullWithTopicAuthorizationFailure() { + public void throwsWithClusterAuthorizationFailureOnDescribe() { + final NewTopic newTopic = TopicAdmin.defineTopic("myTopic").partitions(1).compacted().build(); + Cluster cluster = createCluster(1); + try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(new MockTime(), cluster)) { + env.kafkaClient().prepareResponse(describeTopicResponseWithClusterAuthorizationException(newTopic)); + TopicAdmin admin = new TopicAdmin(null, env.adminClient()); + Exception e = assertThrows(ConnectException.class, () -> admin.describeTopics(newTopic.name())); + assertTrue(e.getCause() instanceof ClusterAuthorizationException); + } + } + + @Test + public void returnNullWithTopicAuthorizationFailureOnCreate() { final NewTopic newTopic = TopicAdmin.defineTopic("myTopic").partitions(1).compacted().build(); Cluster cluster = createCluster(1); try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(new MockTime(), cluster)) { @@ -90,6 +128,18 @@ public void returnNullWithTopicAuthorizationFailure() { } } + @Test + public void throwsWithTopicAuthorizationFailureOnDescribe() { + final NewTopic newTopic = TopicAdmin.defineTopic("myTopic").partitions(1).compacted().build(); + Cluster cluster = createCluster(1); + try (AdminClientUnitTestEnv env = new AdminClientUnitTestEnv(new MockTime(), cluster)) { + env.kafkaClient().prepareResponse(describeTopicResponseWithTopicAuthorizationException(newTopic)); + TopicAdmin admin = new TopicAdmin(null, env.adminClient()); + Exception e = assertThrows(ConnectException.class, () -> admin.describeTopics(newTopic.name())); + assertTrue(e.getCause() instanceof TopicAuthorizationException); + } + } + @Test public void shouldNotCreateTopicWhenItAlreadyExists() { NewTopic newTopic = TopicAdmin.defineTopic("myTopic").partitions(1).compacted().build(); @@ -170,8 +220,7 @@ public void shouldCreateOneTopicWhenProvidedMultipleDefinitionsWithSameTopicName NewTopic newTopic1 = TopicAdmin.defineTopic("myTopic").partitions(1).compacted().build(); NewTopic newTopic2 = TopicAdmin.defineTopic("myTopic").partitions(1).compacted().build(); Cluster cluster = createCluster(1); - try (MockAdminClient mockAdminClient = new MockAdminClient(cluster.nodes(), cluster.nodeById(0))) { - TopicAdmin admin = new TopicAdmin(null, mockAdminClient); + try (TopicAdmin admin = new TopicAdmin(null, new MockAdminClient(cluster.nodes(), cluster.nodeById(0)))) { Set newTopicNames = admin.createTopics(newTopic1, newTopic2); assertEquals(1, newTopicNames.size()); assertEquals(newTopic2.name(), newTopicNames.iterator().next()); @@ -179,15 +228,39 @@ public void shouldCreateOneTopicWhenProvidedMultipleDefinitionsWithSameTopicName } @Test - public void shouldReturnFalseWhenSuppliedNullTopicDescription() { + public void createShouldReturnFalseWhenSuppliedNullTopicDescription() { Cluster cluster = createCluster(1); - try (MockAdminClient mockAdminClient = new MockAdminClient(cluster.nodes(), cluster.nodeById(0))) { - TopicAdmin admin = new TopicAdmin(null, mockAdminClient); + try (TopicAdmin admin = new TopicAdmin(null, new MockAdminClient(cluster.nodes(), cluster.nodeById(0)))) { boolean created = admin.createTopic(null); assertFalse(created); } } + @Test + public void describeShouldReturnEmptyWhenTopicDoesNotExist() { + NewTopic newTopic = TopicAdmin.defineTopic("myTopic").partitions(1).compacted().build(); + Cluster cluster = createCluster(1); + try (TopicAdmin admin = new TopicAdmin(null, new MockAdminClient(cluster.nodes(), cluster.nodeById(0)))) { + assertTrue(admin.describeTopics(newTopic.name()).isEmpty()); + } + } + + @Test + public void describeShouldReturnTopicDescriptionWhenTopicExists() { + String topicName = "myTopic"; + NewTopic newTopic = TopicAdmin.defineTopic(topicName).partitions(1).compacted().build(); + Cluster cluster = createCluster(1); + try (MockAdminClient mockAdminClient = new MockAdminClient(cluster.nodes(), cluster.nodeById(0))) { + TopicPartitionInfo topicPartitionInfo = new TopicPartitionInfo(0, cluster.nodeById(0), cluster.nodes(), Collections.emptyList()); + mockAdminClient.addTopic(false, topicName, Collections.singletonList(topicPartitionInfo), null); + TopicAdmin admin = new TopicAdmin(null, mockAdminClient); + Map desc = admin.describeTopics(newTopic.name()); + assertFalse(desc.isEmpty()); + TopicDescription topicDesc = new TopicDescription(topicName, false, Collections.singletonList(topicPartitionInfo)); + assertEquals(desc.get("myTopic"), topicDesc); + } + } + private Cluster createCluster(int numNodes) { HashMap nodes = new HashMap<>(); for (int i = 0; i < numNodes; ++i) { @@ -267,4 +340,27 @@ protected TopicDescription topicDescription(MockAdminClient admin, String topicN Map> byName = result.values(); return byName.get(topicName).get(); } + + private MetadataResponse describeTopicResponseWithUnsupportedVersion(NewTopic... topics) { + return describeTopicResponse(new ApiError(Errors.UNSUPPORTED_VERSION, "This version of the API is not supported"), topics); + } + + private MetadataResponse describeTopicResponseWithClusterAuthorizationException(NewTopic... topics) { + return describeTopicResponse(new ApiError(Errors.CLUSTER_AUTHORIZATION_FAILED, "Not authorized to create topic(s)"), topics); + } + + private MetadataResponse describeTopicResponseWithTopicAuthorizationException(NewTopic... topics) { + return describeTopicResponse(new ApiError(Errors.TOPIC_AUTHORIZATION_FAILED, "Not authorized to create topic(s)"), topics); + } + + private MetadataResponse describeTopicResponse(ApiError error, NewTopic... topics) { + if (error == null) error = new ApiError(Errors.NONE, ""); + MetadataResponseData response = new MetadataResponseData(); + for (NewTopic topic : topics) { + response.topics().add(new MetadataResponseTopic() + .setName(topic.name()) + .setErrorCode(error.error().code())); + } + return new MetadataResponse(response); + } } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/util/TopicCreationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/util/TopicCreationTest.java new file mode 100644 index 0000000000000..3a51d10f88822 --- /dev/null +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/util/TopicCreationTest.java @@ -0,0 +1,488 @@ +/* + * 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.util; + +import org.apache.kafka.clients.admin.NewTopic; +import org.apache.kafka.connect.runtime.SourceConnectorConfig; +import org.apache.kafka.connect.runtime.WorkerConfig; +import org.apache.kafka.connect.runtime.distributed.DistributedConfig; +import org.apache.kafka.connect.storage.StringConverter; +import org.junit.Before; +import org.junit.Test; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +import static org.apache.kafka.common.config.TopicConfig.CLEANUP_POLICY_COMPACT; +import static org.apache.kafka.common.config.TopicConfig.CLEANUP_POLICY_CONFIG; +import static org.apache.kafka.common.config.TopicConfig.COMPRESSION_TYPE_CONFIG; +import static org.apache.kafka.common.config.TopicConfig.RETENTION_MS_CONFIG; +import static org.apache.kafka.connect.runtime.ConnectorConfig.CONNECTOR_CLASS_CONFIG; +import static org.apache.kafka.connect.runtime.ConnectorConfig.NAME_CONFIG; +import static org.apache.kafka.connect.runtime.ConnectorConfigTest.MOCK_PLUGINS; +import static org.apache.kafka.connect.runtime.SourceConnectorConfig.TOPIC_CREATION_GROUPS_CONFIG; +import static org.apache.kafka.connect.runtime.SourceConnectorConfig.TOPIC_CREATION_PREFIX; +import static org.apache.kafka.connect.runtime.TopicCreationConfig.DEFAULT_TOPIC_CREATION_GROUP; +import static org.apache.kafka.connect.runtime.TopicCreationConfig.DEFAULT_TOPIC_CREATION_PREFIX; +import static org.apache.kafka.connect.runtime.TopicCreationConfig.EXCLUDE_REGEX_CONFIG; +import static org.apache.kafka.connect.runtime.TopicCreationConfig.INCLUDE_REGEX_CONFIG; +import static org.apache.kafka.connect.runtime.TopicCreationConfig.PARTITIONS_CONFIG; +import static org.apache.kafka.connect.runtime.TopicCreationConfig.REPLICATION_FACTOR_CONFIG; +import static org.apache.kafka.connect.runtime.WorkerConfig.BOOTSTRAP_SERVERS_CONFIG; +import static org.apache.kafka.connect.runtime.WorkerConfig.KEY_CONVERTER_CLASS_CONFIG; +import static org.apache.kafka.connect.runtime.WorkerConfig.TOPIC_CREATION_ENABLE_CONFIG; +import static org.apache.kafka.connect.runtime.WorkerConfig.VALUE_CONVERTER_CLASS_CONFIG; +import static org.apache.kafka.connect.runtime.distributed.DistributedConfig.CONFIG_TOPIC_CONFIG; +import static org.apache.kafka.connect.runtime.distributed.DistributedConfig.GROUP_ID_CONFIG; +import static org.apache.kafka.connect.runtime.distributed.DistributedConfig.OFFSET_STORAGE_TOPIC_CONFIG; +import static org.apache.kafka.connect.runtime.distributed.DistributedConfig.STATUS_STORAGE_TOPIC_CONFIG; +import static org.hamcrest.CoreMatchers.hasItem; +import static org.hamcrest.CoreMatchers.hasItems; +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; + +public class TopicCreationTest { + + private static final String FOO_CONNECTOR = "foo-source"; + private static final String FOO_GROUP = "foo"; + private static final String FOO_TOPIC = "foo-topic"; + private static final String FOO_REGEX = ".*foo.*"; + + private static final String BAR_GROUP = "bar"; + private static final String BAR_TOPIC = "bar-topic"; + private static final String BAR_REGEX = ".*bar.*"; + + private static final short DEFAULT_REPLICATION_FACTOR = -1; + private static final int DEFAULT_PARTITIONS = -1; + + Map workerProps; + WorkerConfig workerConfig; + Map sourceProps; + SourceConnectorConfig sourceConfig; + + @Before + public void setup() { + workerProps = defaultWorkerProps(); + workerConfig = new DistributedConfig(workerProps); + } + + public Map defaultWorkerProps() { + Map props = new HashMap<>(); + props.put(GROUP_ID_CONFIG, "connect-cluster"); + props.put(BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); + props.put(CONFIG_TOPIC_CONFIG, "connect-configs"); + props.put(OFFSET_STORAGE_TOPIC_CONFIG, "connect-offsets"); + props.put(STATUS_STORAGE_TOPIC_CONFIG, "connect-status"); + props.put(KEY_CONVERTER_CLASS_CONFIG, StringConverter.class.getName()); + props.put(VALUE_CONVERTER_CLASS_CONFIG, StringConverter.class.getName()); + props.put(TOPIC_CREATION_ENABLE_CONFIG, String.valueOf(true)); + return props; + } + + public Map defaultConnectorProps() { + Map props = new HashMap<>(); + props.put(NAME_CONFIG, FOO_CONNECTOR); + props.put(CONNECTOR_CLASS_CONFIG, "TestConnector"); + return props; + } + + public Map defaultConnectorPropsWithTopicCreation() { + Map props = defaultConnectorProps(); + props.put(DEFAULT_TOPIC_CREATION_PREFIX + REPLICATION_FACTOR_CONFIG, String.valueOf(DEFAULT_REPLICATION_FACTOR)); + props.put(DEFAULT_TOPIC_CREATION_PREFIX + PARTITIONS_CONFIG, String.valueOf(DEFAULT_PARTITIONS)); + return props; + } + + @Test + public void testTopicCreationWhenTopicCreationIsEnabled() { + sourceProps = defaultConnectorPropsWithTopicCreation(); + sourceProps.put(TOPIC_CREATION_GROUPS_CONFIG, String.join(",", FOO_GROUP, BAR_GROUP)); + sourceConfig = new SourceConnectorConfig(MOCK_PLUGINS, sourceProps, true); + + Map groups = TopicCreationGroup.configuredGroups(sourceConfig); + TopicCreation topicCreation = TopicCreation.newTopicCreation(workerConfig, groups); + + assertTrue(topicCreation.isTopicCreationEnabled()); + assertTrue(topicCreation.isTopicCreationRequired(FOO_TOPIC)); + assertThat(topicCreation.defaultTopicGroup(), is(groups.get(DEFAULT_TOPIC_CREATION_GROUP))); + assertEquals(2, topicCreation.topicGroups().size()); + assertThat(topicCreation.topicGroups().keySet(), hasItems(FOO_GROUP, BAR_GROUP)); + assertEquals(topicCreation.defaultTopicGroup(), topicCreation.findFirstGroup(FOO_TOPIC)); + topicCreation.addTopic(FOO_TOPIC); + assertFalse(topicCreation.isTopicCreationRequired(FOO_TOPIC)); + } + + @Test + public void testTopicCreationWhenTopicCreationIsDisabled() { + workerProps.put(TOPIC_CREATION_ENABLE_CONFIG, String.valueOf(false)); + workerConfig = new DistributedConfig(workerProps); + sourceProps = defaultConnectorPropsWithTopicCreation(); + sourceConfig = new SourceConnectorConfig(MOCK_PLUGINS, sourceProps, true); + + TopicCreation topicCreation = TopicCreation.newTopicCreation(workerConfig, + TopicCreationGroup.configuredGroups(sourceConfig)); + + assertFalse(topicCreation.isTopicCreationEnabled()); + assertFalse(topicCreation.isTopicCreationRequired(FOO_TOPIC)); + assertNull(topicCreation.defaultTopicGroup()); + assertThat(topicCreation.topicGroups(), is(Collections.emptyMap())); + assertNull(topicCreation.findFirstGroup(FOO_TOPIC)); + topicCreation.addTopic(FOO_TOPIC); + assertFalse(topicCreation.isTopicCreationRequired(FOO_TOPIC)); + } + + @Test + public void testEmptyTopicCreation() { + TopicCreation topicCreation = TopicCreation.newTopicCreation(workerConfig, null); + + assertEquals(TopicCreation.empty(), topicCreation); + assertFalse(topicCreation.isTopicCreationEnabled()); + assertFalse(topicCreation.isTopicCreationRequired(FOO_TOPIC)); + assertNull(topicCreation.defaultTopicGroup()); + assertEquals(0, topicCreation.topicGroups().size()); + assertThat(topicCreation.topicGroups(), is(Collections.emptyMap())); + assertNull(topicCreation.findFirstGroup(FOO_TOPIC)); + topicCreation.addTopic(FOO_TOPIC); + assertFalse(topicCreation.isTopicCreationRequired(FOO_TOPIC)); + } + + @Test + public void withDefaultTopicCreation() { + sourceProps = defaultConnectorPropsWithTopicCreation(); + // Setting here but they should be ignored for the default group + sourceProps.put(TOPIC_CREATION_PREFIX + DEFAULT_TOPIC_CREATION_GROUP + "." + INCLUDE_REGEX_CONFIG, FOO_REGEX); + sourceProps.put(TOPIC_CREATION_PREFIX + DEFAULT_TOPIC_CREATION_GROUP + "." + EXCLUDE_REGEX_CONFIG, BAR_REGEX); + + // verify config creation + sourceConfig = new SourceConnectorConfig(MOCK_PLUGINS, sourceProps, true); + assertTrue(sourceConfig.usesTopicCreation()); + assertEquals(DEFAULT_REPLICATION_FACTOR, (short) sourceConfig.topicCreationReplicationFactor(DEFAULT_TOPIC_CREATION_GROUP)); + assertEquals(DEFAULT_PARTITIONS, (int) sourceConfig.topicCreationPartitions(DEFAULT_TOPIC_CREATION_GROUP)); + assertThat(sourceConfig.topicCreationInclude(DEFAULT_TOPIC_CREATION_GROUP), is(Collections.singletonList(".*"))); + assertThat(sourceConfig.topicCreationExclude(DEFAULT_TOPIC_CREATION_GROUP), is(Collections.emptyList())); + assertThat(sourceConfig.topicCreationOtherConfigs(DEFAULT_TOPIC_CREATION_GROUP), is(Collections.emptyMap())); + + // verify topic creation group is instantiated correctly + Map groups = TopicCreationGroup.configuredGroups(sourceConfig); + assertEquals(1, groups.size()); + assertThat(groups.keySet(), hasItem(DEFAULT_TOPIC_CREATION_GROUP)); + + // verify topic creation + TopicCreation topicCreation = TopicCreation.newTopicCreation(workerConfig, groups); + TopicCreationGroup group = topicCreation.defaultTopicGroup(); + // Default group will match all topics besides empty string + assertTrue(group.matches(" ")); + assertTrue(group.matches(FOO_TOPIC)); + assertEquals(DEFAULT_TOPIC_CREATION_GROUP, group.name()); + assertTrue(topicCreation.isTopicCreationEnabled()); + assertTrue(topicCreation.isTopicCreationRequired(FOO_TOPIC)); + assertThat(topicCreation.topicGroups(), is(Collections.emptyMap())); + assertEquals(topicCreation.defaultTopicGroup(), topicCreation.findFirstGroup(FOO_TOPIC)); + topicCreation.addTopic(FOO_TOPIC); + assertFalse(topicCreation.isTopicCreationRequired(FOO_TOPIC)); + + // verify new topic properties + NewTopic topicSpec = topicCreation.findFirstGroup(FOO_TOPIC).newTopic(FOO_TOPIC); + assertEquals(FOO_TOPIC, topicSpec.name()); + assertEquals(DEFAULT_REPLICATION_FACTOR, topicSpec.replicationFactor()); + assertEquals(DEFAULT_PARTITIONS, topicSpec.numPartitions()); + assertThat(topicSpec.configs(), is(Collections.emptyMap())); + } + + @Test + public void topicCreationWithDefaultGroupAndCustomProps() { + short replicas = 3; + int partitions = 5; + long retentionMs = TimeUnit.DAYS.toMillis(30); + String compressionType = "lz4"; + Map topicProps = new HashMap<>(); + topicProps.put(COMPRESSION_TYPE_CONFIG, compressionType); + topicProps.put(RETENTION_MS_CONFIG, String.valueOf(retentionMs)); + + sourceProps = defaultConnectorPropsWithTopicCreation(); + sourceProps.put(DEFAULT_TOPIC_CREATION_PREFIX + REPLICATION_FACTOR_CONFIG, String.valueOf(replicas)); + sourceProps.put(DEFAULT_TOPIC_CREATION_PREFIX + PARTITIONS_CONFIG, String.valueOf(partitions)); + topicProps.forEach((k, v) -> sourceProps.put(DEFAULT_TOPIC_CREATION_PREFIX + k, v)); + // Setting here but they should be ignored for the default group + sourceProps.put(TOPIC_CREATION_PREFIX + DEFAULT_TOPIC_CREATION_GROUP + "." + INCLUDE_REGEX_CONFIG, FOO_REGEX); + sourceProps.put(TOPIC_CREATION_PREFIX + DEFAULT_TOPIC_CREATION_GROUP + "." + EXCLUDE_REGEX_CONFIG, BAR_REGEX); + + // verify config creation + sourceConfig = new SourceConnectorConfig(MOCK_PLUGINS, sourceProps, true); + assertTrue(sourceConfig.usesTopicCreation()); + assertEquals(replicas, (short) sourceConfig.topicCreationReplicationFactor(DEFAULT_TOPIC_CREATION_GROUP)); + assertEquals(partitions, (int) sourceConfig.topicCreationPartitions(DEFAULT_TOPIC_CREATION_GROUP)); + assertThat(sourceConfig.topicCreationInclude(DEFAULT_TOPIC_CREATION_GROUP), is(Collections.singletonList(".*"))); + assertThat(sourceConfig.topicCreationExclude(DEFAULT_TOPIC_CREATION_GROUP), is(Collections.emptyList())); + assertThat(sourceConfig.topicCreationOtherConfigs(DEFAULT_TOPIC_CREATION_GROUP), is(topicProps)); + + // verify topic creation group is instantiated correctly + Map groups = TopicCreationGroup.configuredGroups(sourceConfig); + assertEquals(1, groups.size()); + assertThat(groups.keySet(), hasItem(DEFAULT_TOPIC_CREATION_GROUP)); + + // verify topic creation + TopicCreation topicCreation = TopicCreation.newTopicCreation(workerConfig, groups); + TopicCreationGroup group = topicCreation.defaultTopicGroup(); + // Default group will match all topics besides empty string + assertTrue(group.matches(" ")); + assertTrue(group.matches(FOO_TOPIC)); + assertEquals(DEFAULT_TOPIC_CREATION_GROUP, group.name()); + assertTrue(topicCreation.isTopicCreationEnabled()); + assertTrue(topicCreation.isTopicCreationRequired(FOO_TOPIC)); + assertThat(topicCreation.topicGroups(), is(Collections.emptyMap())); + assertEquals(topicCreation.defaultTopicGroup(), topicCreation.findFirstGroup(FOO_TOPIC)); + topicCreation.addTopic(FOO_TOPIC); + assertFalse(topicCreation.isTopicCreationRequired(FOO_TOPIC)); + + // verify new topic properties + NewTopic topicSpec = topicCreation.findFirstGroup(FOO_TOPIC).newTopic(FOO_TOPIC); + assertEquals(FOO_TOPIC, topicSpec.name()); + assertEquals(replicas, topicSpec.replicationFactor()); + assertEquals(partitions, topicSpec.numPartitions()); + assertThat(topicSpec.configs(), is(topicProps)); + } + + @Test + public void topicCreationWithOneGroup() { + short fooReplicas = 3; + int partitions = 5; + sourceProps = defaultConnectorPropsWithTopicCreation(); + sourceProps.put(TOPIC_CREATION_GROUPS_CONFIG, String.join(",", FOO_GROUP)); + sourceProps.put(DEFAULT_TOPIC_CREATION_PREFIX + PARTITIONS_CONFIG, String.valueOf(partitions)); + sourceProps.put(TOPIC_CREATION_PREFIX + FOO_GROUP + "." + INCLUDE_REGEX_CONFIG, FOO_REGEX); + sourceProps.put(TOPIC_CREATION_PREFIX + FOO_GROUP + "." + EXCLUDE_REGEX_CONFIG, BAR_REGEX); + sourceProps.put(TOPIC_CREATION_PREFIX + FOO_GROUP + "." + REPLICATION_FACTOR_CONFIG, String.valueOf(fooReplicas)); + + Map topicProps = new HashMap<>(); + topicProps.put(CLEANUP_POLICY_CONFIG, CLEANUP_POLICY_COMPACT); + topicProps.forEach((k, v) -> sourceProps.put(TOPIC_CREATION_PREFIX + FOO_GROUP + "." + k, v)); + + // verify config creation + sourceConfig = new SourceConnectorConfig(MOCK_PLUGINS, sourceProps, true); + assertTrue(sourceConfig.usesTopicCreation()); + assertEquals(DEFAULT_REPLICATION_FACTOR, (short) sourceConfig.topicCreationReplicationFactor(DEFAULT_TOPIC_CREATION_GROUP)); + assertEquals(partitions, (int) sourceConfig.topicCreationPartitions(DEFAULT_TOPIC_CREATION_GROUP)); + assertThat(sourceConfig.topicCreationInclude(DEFAULT_TOPIC_CREATION_GROUP), is(Collections.singletonList(".*"))); + assertThat(sourceConfig.topicCreationExclude(DEFAULT_TOPIC_CREATION_GROUP), is(Collections.emptyList())); + assertThat(sourceConfig.topicCreationOtherConfigs(DEFAULT_TOPIC_CREATION_GROUP), is(Collections.emptyMap())); + + // verify topic creation group is instantiated correctly + Map groups = TopicCreationGroup.configuredGroups(sourceConfig); + assertEquals(2, groups.size()); + assertThat(groups.keySet(), hasItems(DEFAULT_TOPIC_CREATION_GROUP, FOO_GROUP)); + + // verify topic creation + TopicCreation topicCreation = TopicCreation.newTopicCreation(workerConfig, groups); + TopicCreationGroup defaultGroup = topicCreation.defaultTopicGroup(); + // Default group will match all topics besides empty string + assertTrue(defaultGroup.matches(" ")); + assertTrue(defaultGroup.matches(FOO_TOPIC)); + assertTrue(defaultGroup.matches(BAR_TOPIC)); + assertEquals(DEFAULT_TOPIC_CREATION_GROUP, defaultGroup.name()); + TopicCreationGroup fooGroup = groups.get(FOO_GROUP); + assertFalse(fooGroup.matches(" ")); + assertTrue(fooGroup.matches(FOO_TOPIC)); + assertFalse(fooGroup.matches(BAR_TOPIC)); + assertEquals(FOO_GROUP, fooGroup.name()); + + assertTrue(topicCreation.isTopicCreationEnabled()); + assertTrue(topicCreation.isTopicCreationRequired(FOO_TOPIC)); + assertEquals(1, topicCreation.topicGroups().size()); + assertThat(topicCreation.topicGroups().keySet(), hasItems(FOO_GROUP)); + assertEquals(fooGroup, topicCreation.findFirstGroup(FOO_TOPIC)); + topicCreation.addTopic(FOO_TOPIC); + assertFalse(topicCreation.isTopicCreationRequired(FOO_TOPIC)); + + // verify new topic properties + NewTopic defaultTopicSpec = topicCreation.findFirstGroup(BAR_TOPIC).newTopic(BAR_TOPIC); + assertEquals(BAR_TOPIC, defaultTopicSpec.name()); + assertEquals(DEFAULT_REPLICATION_FACTOR, defaultTopicSpec.replicationFactor()); + assertEquals(partitions, defaultTopicSpec.numPartitions()); + assertThat(defaultTopicSpec.configs(), is(Collections.emptyMap())); + + NewTopic fooTopicSpec = topicCreation.findFirstGroup(FOO_TOPIC).newTopic(FOO_TOPIC); + assertEquals(FOO_TOPIC, fooTopicSpec.name()); + assertEquals(fooReplicas, fooTopicSpec.replicationFactor()); + assertEquals(partitions, fooTopicSpec.numPartitions()); + assertThat(fooTopicSpec.configs(), is(topicProps)); + } + + @Test + public void topicCreationWithOneGroupAndCombinedRegex() { + short fooReplicas = 3; + int partitions = 5; + sourceProps = defaultConnectorPropsWithTopicCreation(); + sourceProps.put(TOPIC_CREATION_GROUPS_CONFIG, String.join(",", FOO_GROUP)); + sourceProps.put(DEFAULT_TOPIC_CREATION_PREFIX + PARTITIONS_CONFIG, String.valueOf(partitions)); + // Setting here but they should be ignored for the default group + sourceProps.put(TOPIC_CREATION_PREFIX + FOO_GROUP + "." + INCLUDE_REGEX_CONFIG, String.join("|", FOO_REGEX, BAR_REGEX)); + sourceProps.put(TOPIC_CREATION_PREFIX + FOO_GROUP + "." + REPLICATION_FACTOR_CONFIG, String.valueOf(fooReplicas)); + + Map topicProps = new HashMap<>(); + topicProps.put(CLEANUP_POLICY_CONFIG, CLEANUP_POLICY_COMPACT); + topicProps.forEach((k, v) -> sourceProps.put(TOPIC_CREATION_PREFIX + FOO_GROUP + "." + k, v)); + + // verify config creation + sourceConfig = new SourceConnectorConfig(MOCK_PLUGINS, sourceProps, true); + assertTrue(sourceConfig.usesTopicCreation()); + assertEquals(DEFAULT_REPLICATION_FACTOR, (short) sourceConfig.topicCreationReplicationFactor(DEFAULT_TOPIC_CREATION_GROUP)); + assertEquals(partitions, (int) sourceConfig.topicCreationPartitions(DEFAULT_TOPIC_CREATION_GROUP)); + assertThat(sourceConfig.topicCreationInclude(DEFAULT_TOPIC_CREATION_GROUP), is(Collections.singletonList(".*"))); + assertThat(sourceConfig.topicCreationExclude(DEFAULT_TOPIC_CREATION_GROUP), is(Collections.emptyList())); + assertThat(sourceConfig.topicCreationOtherConfigs(DEFAULT_TOPIC_CREATION_GROUP), is(Collections.emptyMap())); + + // verify topic creation group is instantiated correctly + Map groups = TopicCreationGroup.configuredGroups(sourceConfig); + assertEquals(2, groups.size()); + assertThat(groups.keySet(), hasItems(DEFAULT_TOPIC_CREATION_GROUP, FOO_GROUP)); + + // verify topic creation + TopicCreation topicCreation = TopicCreation.newTopicCreation(workerConfig, groups); + TopicCreationGroup defaultGroup = topicCreation.defaultTopicGroup(); + // Default group will match all topics besides empty string + assertTrue(defaultGroup.matches(" ")); + assertTrue(defaultGroup.matches(FOO_TOPIC)); + assertTrue(defaultGroup.matches(BAR_TOPIC)); + assertEquals(DEFAULT_TOPIC_CREATION_GROUP, defaultGroup.name()); + TopicCreationGroup fooGroup = groups.get(FOO_GROUP); + assertFalse(fooGroup.matches(" ")); + assertTrue(fooGroup.matches(FOO_TOPIC)); + assertTrue(fooGroup.matches(BAR_TOPIC)); + assertEquals(FOO_GROUP, fooGroup.name()); + + assertTrue(topicCreation.isTopicCreationEnabled()); + assertTrue(topicCreation.isTopicCreationRequired(FOO_TOPIC)); + assertTrue(topicCreation.isTopicCreationRequired(BAR_TOPIC)); + assertEquals(1, topicCreation.topicGroups().size()); + assertThat(topicCreation.topicGroups().keySet(), hasItems(FOO_GROUP)); + assertEquals(fooGroup, topicCreation.findFirstGroup(FOO_TOPIC)); + assertEquals(fooGroup, topicCreation.findFirstGroup(BAR_TOPIC)); + topicCreation.addTopic(FOO_TOPIC); + topicCreation.addTopic(BAR_TOPIC); + assertFalse(topicCreation.isTopicCreationRequired(FOO_TOPIC)); + assertFalse(topicCreation.isTopicCreationRequired(BAR_TOPIC)); + + // verify new topic properties + NewTopic fooTopicSpec = topicCreation.findFirstGroup(FOO_TOPIC).newTopic(FOO_TOPIC); + assertEquals(FOO_TOPIC, fooTopicSpec.name()); + assertEquals(fooReplicas, fooTopicSpec.replicationFactor()); + assertEquals(partitions, fooTopicSpec.numPartitions()); + assertThat(fooTopicSpec.configs(), is(topicProps)); + + NewTopic barTopicSpec = topicCreation.findFirstGroup(BAR_TOPIC).newTopic(BAR_TOPIC); + assertEquals(BAR_TOPIC, barTopicSpec.name()); + assertEquals(fooReplicas, barTopicSpec.replicationFactor()); + assertEquals(partitions, barTopicSpec.numPartitions()); + assertThat(barTopicSpec.configs(), is(topicProps)); + } + + @Test + public void topicCreationWithTwoGroups() { + short fooReplicas = 3; + int partitions = 5; + int barPartitions = 1; + + sourceProps = defaultConnectorPropsWithTopicCreation(); + sourceProps.put(TOPIC_CREATION_GROUPS_CONFIG, String.join(",", FOO_GROUP, BAR_GROUP)); + sourceProps.put(DEFAULT_TOPIC_CREATION_PREFIX + PARTITIONS_CONFIG, String.valueOf(partitions)); + // Setting here but they should be ignored for the default group + sourceProps.put(TOPIC_CREATION_PREFIX + FOO_GROUP + "." + INCLUDE_REGEX_CONFIG, FOO_TOPIC); + sourceProps.put(TOPIC_CREATION_PREFIX + FOO_GROUP + "." + REPLICATION_FACTOR_CONFIG, String.valueOf(fooReplicas)); + sourceProps.put(TOPIC_CREATION_PREFIX + BAR_GROUP + "." + INCLUDE_REGEX_CONFIG, BAR_REGEX); + sourceProps.put(TOPIC_CREATION_PREFIX + BAR_GROUP + "." + PARTITIONS_CONFIG, String.valueOf(barPartitions)); + + Map fooTopicProps = new HashMap<>(); + fooTopicProps.put(RETENTION_MS_CONFIG, String.valueOf(TimeUnit.DAYS.toMillis(30))); + fooTopicProps.forEach((k, v) -> sourceProps.put(TOPIC_CREATION_PREFIX + FOO_GROUP + "." + k, v)); + + Map barTopicProps = new HashMap<>(); + barTopicProps.put(CLEANUP_POLICY_CONFIG, CLEANUP_POLICY_COMPACT); + barTopicProps.forEach((k, v) -> sourceProps.put(TOPIC_CREATION_PREFIX + BAR_GROUP + "." + k, v)); + + // verify config creation + sourceConfig = new SourceConnectorConfig(MOCK_PLUGINS, sourceProps, true); + assertTrue(sourceConfig.usesTopicCreation()); + assertEquals(DEFAULT_REPLICATION_FACTOR, (short) sourceConfig.topicCreationReplicationFactor(DEFAULT_TOPIC_CREATION_GROUP)); + assertEquals(partitions, (int) sourceConfig.topicCreationPartitions(DEFAULT_TOPIC_CREATION_GROUP)); + assertThat(sourceConfig.topicCreationInclude(DEFAULT_TOPIC_CREATION_GROUP), is(Collections.singletonList(".*"))); + assertThat(sourceConfig.topicCreationExclude(DEFAULT_TOPIC_CREATION_GROUP), is(Collections.emptyList())); + assertThat(sourceConfig.topicCreationOtherConfigs(DEFAULT_TOPIC_CREATION_GROUP), is(Collections.emptyMap())); + + // verify topic creation group is instantiated correctly + Map groups = TopicCreationGroup.configuredGroups(sourceConfig); + assertEquals(3, groups.size()); + assertThat(groups.keySet(), hasItems(DEFAULT_TOPIC_CREATION_GROUP, FOO_GROUP, BAR_GROUP)); + + // verify topic creation + TopicCreation topicCreation = TopicCreation.newTopicCreation(workerConfig, groups); + TopicCreationGroup defaultGroup = topicCreation.defaultTopicGroup(); + // Default group will match all topics besides empty string + assertTrue(defaultGroup.matches(" ")); + assertTrue(defaultGroup.matches(FOO_TOPIC)); + assertTrue(defaultGroup.matches(BAR_TOPIC)); + assertEquals(DEFAULT_TOPIC_CREATION_GROUP, defaultGroup.name()); + TopicCreationGroup fooGroup = groups.get(FOO_GROUP); + assertFalse(fooGroup.matches(" ")); + assertTrue(fooGroup.matches(FOO_TOPIC)); + assertFalse(fooGroup.matches(BAR_TOPIC)); + assertEquals(FOO_GROUP, fooGroup.name()); + TopicCreationGroup barGroup = groups.get(BAR_GROUP); + assertTrue(barGroup.matches(BAR_TOPIC)); + assertFalse(barGroup.matches(FOO_TOPIC)); + assertEquals(BAR_GROUP, barGroup.name()); + + assertTrue(topicCreation.isTopicCreationEnabled()); + assertTrue(topicCreation.isTopicCreationRequired(FOO_TOPIC)); + assertTrue(topicCreation.isTopicCreationRequired(BAR_TOPIC)); + assertEquals(2, topicCreation.topicGroups().size()); + assertThat(topicCreation.topicGroups().keySet(), hasItems(FOO_GROUP, BAR_GROUP)); + assertEquals(fooGroup, topicCreation.findFirstGroup(FOO_TOPIC)); + assertEquals(barGroup, topicCreation.findFirstGroup(BAR_TOPIC)); + topicCreation.addTopic(FOO_TOPIC); + topicCreation.addTopic(BAR_TOPIC); + assertFalse(topicCreation.isTopicCreationRequired(FOO_TOPIC)); + assertFalse(topicCreation.isTopicCreationRequired(BAR_TOPIC)); + + // verify new topic properties + String otherTopic = "any-other-topic"; + NewTopic defaultTopicSpec = topicCreation.findFirstGroup(otherTopic).newTopic(otherTopic); + assertEquals(otherTopic, defaultTopicSpec.name()); + assertEquals(DEFAULT_REPLICATION_FACTOR, defaultTopicSpec.replicationFactor()); + assertEquals(partitions, defaultTopicSpec.numPartitions()); + assertThat(defaultTopicSpec.configs(), is(Collections.emptyMap())); + + NewTopic fooTopicSpec = topicCreation.findFirstGroup(FOO_TOPIC).newTopic(FOO_TOPIC); + assertEquals(FOO_TOPIC, fooTopicSpec.name()); + assertEquals(fooReplicas, fooTopicSpec.replicationFactor()); + assertEquals(partitions, fooTopicSpec.numPartitions()); + assertThat(fooTopicSpec.configs(), is(fooTopicProps)); + + NewTopic barTopicSpec = topicCreation.findFirstGroup(BAR_TOPIC).newTopic(BAR_TOPIC); + assertEquals(BAR_TOPIC, barTopicSpec.name()); + assertEquals(DEFAULT_REPLICATION_FACTOR, barTopicSpec.replicationFactor()); + assertEquals(barPartitions, barTopicSpec.numPartitions()); + assertThat(barTopicSpec.configs(), is(barTopicProps)); + } +} diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedConnectCluster.java b/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedConnectCluster.java index 778cb7ee9f0bc..2e40769b10049 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedConnectCluster.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedConnectCluster.java @@ -70,8 +70,8 @@ public class EmbeddedConnectCluster { private static final Logger log = LoggerFactory.getLogger(EmbeddedConnectCluster.class); - private static final int DEFAULT_NUM_BROKERS = 1; - private static final int DEFAULT_NUM_WORKERS = 1; + public static final int DEFAULT_NUM_BROKERS = 1; + public static final int DEFAULT_NUM_WORKERS = 1; private static final Properties DEFAULT_BROKER_CONFIG = new Properties(); private static final String REST_HOST_NAME = "localhost"; diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedConnectClusterAssertions.java b/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedConnectClusterAssertions.java index cb82407164dd3..0cfc1b14b2732 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedConnectClusterAssertions.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedConnectClusterAssertions.java @@ -16,12 +16,6 @@ */ package org.apache.kafka.connect.util.clusters; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashSet; -import java.util.List; -import java.util.Map; - import org.apache.kafka.clients.admin.TopicDescription; import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.common.PartitionInfo; @@ -33,7 +27,12 @@ import org.slf4j.LoggerFactory; import javax.ws.rs.core.Response; +import java.util.Arrays; import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Map; import java.util.Optional; import java.util.Set; import java.util.UUID; @@ -155,7 +154,7 @@ public void assertTopicsDoNotExist(String... topicNames) throws InterruptedExcep existingTopics.set(actual); return actual.isEmpty(); }).orElse(false), - WORKER_SETUP_DURATION_MS, + CONNECTOR_SETUP_DURATION_MS, "Unexpectedly found topics " + existingTopics.get()); } @@ -174,7 +173,7 @@ public void assertTopicsExist(String... topicNames) throws InterruptedException missingTopics.set(missing); return missing.isEmpty(); }).orElse(false), - WORKER_SETUP_DURATION_MS, + CONNECTOR_SETUP_DURATION_MS, "Didn't find the topics " + missingTopics.get()); }