From ef610e376b7fb60dd0c0e5f31797bf9c289dd88a Mon Sep 17 00:00:00 2001 From: Arjun Satish Date: Wed, 15 Aug 2018 22:14:01 -0700 Subject: [PATCH 01/52] Connect integration test harness (WIP) Signed-off-by: Arjun Satish --- build.gradle | 2 + .../kafka/connect/integration/DLQTest.java | 37 +++++ .../util/clusters/EmbeddedConnectCluster.java | 132 ++++++++++++++++ .../util/clusters/EmbeddedKafkaCluster.java | 142 ++++++++++++++++++ 4 files changed, 313 insertions(+) create mode 100644 connect/runtime/src/test/java/org/apache/kafka/connect/integration/DLQTest.java create mode 100644 connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedConnectCluster.java create mode 100644 connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java diff --git a/build.gradle b/build.gradle index 75a4354cc9439..4dbe7d7907c99 100644 --- a/build.gradle +++ b/build.gradle @@ -1453,6 +1453,8 @@ project(':connect:runtime') { testCompile libs.powermockEasymock testCompile project(':clients').sourceSets.test.output + testCompile project(':core') + testCompile project(':core').sourceSets.test.output testRuntime libs.slf4jlog4j } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/DLQTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/DLQTest.java new file mode 100644 index 0000000000000..6484696680c83 --- /dev/null +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/DLQTest.java @@ -0,0 +1,37 @@ +/* + * 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.util.clusters.EmbeddedConnectCluster; +import org.junit.ClassRule; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class DLQTest { + + private static final Logger log = LoggerFactory.getLogger(DLQTest.class); + + @ClassRule + public static EmbeddedConnectCluster CONNECT = new EmbeddedConnectCluster(); + + @Test + public void startConnect() { + + } + +} 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 new file mode 100644 index 0000000000000..15f1c70f9a6a7 --- /dev/null +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedConnectCluster.java @@ -0,0 +1,132 @@ +/* + * 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.clusters; + +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.connect.runtime.Connect; +import org.apache.kafka.connect.runtime.Worker; +import org.apache.kafka.connect.runtime.isolation.Plugins; +import org.apache.kafka.connect.runtime.rest.RestServer; +import org.apache.kafka.connect.runtime.standalone.StandaloneConfig; +import org.apache.kafka.connect.runtime.standalone.StandaloneHerder; +import org.apache.kafka.connect.storage.FileOffsetBackingStore; +import org.apache.kafka.connect.util.ConnectUtils; +import org.junit.rules.ExternalResource; +import org.junit.rules.TemporaryFolder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; +import java.net.URI; +import java.util.HashMap; +import java.util.Map; +import java.util.Properties; + +public class EmbeddedConnectCluster extends ExternalResource { + + private static final Logger log = LoggerFactory.getLogger(EmbeddedConnectCluster.class); + + private static final int NUM_BROKERS = 1; + private static final Properties BROKER_CONFIG = new Properties(); + + private final EmbeddedKafkaCluster kafkaCluster = new EmbeddedKafkaCluster(NUM_BROKERS, BROKER_CONFIG); + + private final Map workerProps; + + private File offsetsDirectory; + private Connect connect; + private URI advertisedUrl; + + public EmbeddedConnectCluster() { + // this empty map will be populated with defaults before starting Connect. + this(new HashMap<>()); + } + + public EmbeddedConnectCluster(Map workerProps) { + this.workerProps = workerProps; + } + + @Override + protected void before() throws IOException { + kafkaCluster.before(); + start(); + log.info("Started connect at {} with kafka cluster at {}", restUrl(), kafkaCluster().bootstrapServers()); + } + + @Override + protected void after() { + try { + log.info("Cleaning up connect offset dir at {}", offsetsDirectory); + Utils.delete(offsetsDirectory); + } catch (IOException e) { + log.error("Could not delete directory at {}", offsetsDirectory, e); + } + + connect.stop(); + kafkaCluster.after(); + } + + public void start() throws IOException { + log.info("Starting standalone connect cluster.."); + workerProps.put("bootstrap.servers", kafkaCluster().bootstrapServers()); + workerProps.put("rest.host.name", "localhost"); + + putIfAbsent(workerProps, "key.converter", "org.apache.kafka.connect.json.JsonConverter"); + putIfAbsent(workerProps, "value.converter", "org.apache.kafka.connect.json.JsonConverter"); + putIfAbsent(workerProps, "key.converter.schemas.enable", "false"); + putIfAbsent(workerProps, "value.converter.schemas.enable", "false"); + + offsetsDirectory = createTmpDir(); + putIfAbsent(workerProps, "offset.storage.file.filename", new File(offsetsDirectory, "connect.integration.offsets").getAbsolutePath()); + + log.info("Scanning for plugins..."); + Plugins plugins = new Plugins(workerProps); + plugins.compareAndSwapWithDelegatingLoader(); + + StandaloneConfig config = new StandaloneConfig(workerProps); + + RestServer rest = new RestServer(config); + advertisedUrl = rest.advertisedUrl(); + String workerId = advertisedUrl.getHost() + ":" + advertisedUrl.getPort(); + + Worker worker = new Worker(workerId, kafkaCluster.time(), plugins, config, new FileOffsetBackingStore()); + + connect = new Connect(new StandaloneHerder(worker, ConnectUtils.lookupKafkaClusterId(config)), rest); + connect.start(); + } + + private void putIfAbsent(Map props, String propertyKey, String propertyValue) { + if (!props.containsKey(propertyKey)) { + props.put(propertyKey, propertyValue); + } + } + + private File createTmpDir() throws IOException { + TemporaryFolder tmpFolder = new TemporaryFolder(); + tmpFolder.create(); + return tmpFolder.newFolder(); + } + + public URI restUrl() { + return advertisedUrl; + } + + public EmbeddedKafkaCluster kafkaCluster() { + return kafkaCluster; + } +} \ No newline at end of file diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java b/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java new file mode 100644 index 0000000000000..f7dbf17672967 --- /dev/null +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.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.util.clusters; + +import kafka.server.KafkaConfig; +import kafka.server.KafkaConfig$; +import kafka.server.KafkaServer; +import kafka.utils.CoreUtils; +import kafka.utils.TestUtils; +import kafka.zk.EmbeddedZookeeper; +import org.apache.kafka.common.network.ListenerName; +import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.common.utils.Time; +import org.junit.rules.ExternalResource; +import org.junit.rules.TemporaryFolder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Properties; + +public class EmbeddedKafkaCluster extends ExternalResource { + + private static final Logger log = LoggerFactory.getLogger(EmbeddedKafkaCluster.class); + + // Kafka Config + private final KafkaServer[] brokers; + private final Properties brokerConfig; + private final Time time = new MockTime(); + + private EmbeddedZookeeper zookeeper = null; + private ListenerName listenerName = new ListenerName("PLAINTEXT"); + + public EmbeddedKafkaCluster(final int numBrokers, + final Properties brokerConfig) { + brokers = new KafkaServer[numBrokers]; + this.brokerConfig = brokerConfig; + } + + @Override + protected void before() throws IOException { + start(); + } + + @Override + protected void after() { + stop(); + } + + Time time() { + return time; + } + + public void start() throws IOException { + zookeeper = new EmbeddedZookeeper(); + + brokerConfig.put(KafkaConfig$.MODULE$.ZkConnectProp(), zKConnectString()); + brokerConfig.put(KafkaConfig$.MODULE$.PortProp(), 9999); // pick a random port + + putIfAbsent(brokerConfig, KafkaConfig$.MODULE$.HostNameProp(), "localhost"); + putIfAbsent(brokerConfig, KafkaConfig$.MODULE$.DeleteTopicEnableProp(), true); + putIfAbsent(brokerConfig, KafkaConfig$.MODULE$.GroupInitialRebalanceDelayMsProp(), 0); + putIfAbsent(brokerConfig, KafkaConfig$.MODULE$.OffsetsTopicReplicationFactorProp(), (short) 1); + putIfAbsent(brokerConfig, KafkaConfig$.MODULE$.AutoCreateTopicsEnableProp(), true); + + Object listenerConfig = brokerConfig.get(KafkaConfig$.MODULE$.InterBrokerListenerNameProp()); + if (listenerConfig != null) { + listenerName = new ListenerName(listenerConfig.toString()); + } + + for (int i = 0; i < brokers.length; i++) { + brokerConfig.put(KafkaConfig$.MODULE$.BrokerIdProp(), i); + brokerConfig.put(KafkaConfig$.MODULE$.LogDirProp(), createLogDir()); + brokers[i] = TestUtils.createServer(new KafkaConfig(brokerConfig, true), time); + } + } + + private void stop() { + + for (KafkaServer broker : brokers) { + try { + broker.shutdown(); + } catch (Throwable t) { + log.error("Could not shutdown broker at " + address(broker), t); + } + } + + for (KafkaServer broker : brokers) { + try { + log.info("Cleaning up kafka log dirs at {}", broker.config().logDirs()); + CoreUtils.delete(broker.config().logDirs()); + } catch (Throwable t) { + log.error("Could not clean up log dirs for broker " + address(broker), t); + } + } + + try { + zookeeper.shutdown(); + } catch (Throwable t) { + log.error("Could not shutdown zookeeper at " + zKConnectString(), t); + } + } + + private void putIfAbsent(final Properties props, final String propertyKey, final Object propertyValue) { + if (!props.containsKey(propertyKey)) { + props.put(propertyKey, propertyValue); + } + } + + private String createLogDir() throws IOException { + TemporaryFolder tmpFolder = new TemporaryFolder(); + tmpFolder.create(); + return tmpFolder.newFolder().getAbsolutePath(); + } + + public String bootstrapServers() { + return address(brokers[0]); + } + + public String address(KafkaServer server) { + return server.config().hostName() + ":" + server.boundPort(listenerName); + } + + public String zKConnectString() { + return "127.0.0.1:" + zookeeper.port(); + } + +} From 97dc0fb54a81b94e2c55974765fd9b9ea4ffeefd Mon Sep 17 00:00:00 2001 From: Arjun Satish Date: Thu, 16 Aug 2018 13:41:26 -0700 Subject: [PATCH 02/52] MINOR: Fix checkstyle issues Signed-off-by: Arjun Satish --- checkstyle/import-control.xml | 11 +++++++++-- .../org/apache/kafka/connect/integration/DLQTest.java | 4 ++-- connect/runtime/src/test/resources/log4j.properties | 8 +++++++- 3 files changed, 18 insertions(+), 5 deletions(-) diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index c69f94d25bbce..4ae3a49999b9b 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -347,8 +347,6 @@ - - @@ -366,6 +364,15 @@ + + + + + + + + + diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/DLQTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/DLQTest.java index 6484696680c83..dced9b2d8d11b 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/DLQTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/DLQTest.java @@ -27,11 +27,11 @@ public class DLQTest { private static final Logger log = LoggerFactory.getLogger(DLQTest.class); @ClassRule - public static EmbeddedConnectCluster CONNECT = new EmbeddedConnectCluster(); + public static EmbeddedConnectCluster connectCluster = new EmbeddedConnectCluster(); @Test public void startConnect() { - + } } diff --git a/connect/runtime/src/test/resources/log4j.properties b/connect/runtime/src/test/resources/log4j.properties index d5e90fe788f76..ee2c5c8619cdd 100644 --- a/connect/runtime/src/test/resources/log4j.properties +++ b/connect/runtime/src/test/resources/log4j.properties @@ -14,10 +14,16 @@ # See the License for the specific language governing permissions and # limitations under the License. ## -log4j.rootLogger=OFF, stdout +log4j.rootLogger=INFO, stdout +# Redirect log messages to console log4j.appender.stdout=org.apache.log4j.ConsoleAppender +log4j.appender.stdout.Target=System.out log4j.appender.stdout.layout=org.apache.log4j.PatternLayout log4j.appender.stdout.layout.ConversionPattern=[%d] %p %m (%c:%L)%n +log4j.logger.org.reflections=ERROR log4j.logger.org.apache.kafka=ERROR +log4j.logger.org.apache.kafka.connect=INFO +log4j.logger.org.eclipse.jetty=ERROR +log4j.logger.org.apache.zookeeper=ERROR From 1ec70a5e93c3aae0aa0cd0300aa554dec14bcf71 Mon Sep 17 00:00:00 2001 From: Arjun Satish Date: Thu, 16 Aug 2018 16:47:04 -0700 Subject: [PATCH 03/52] MINOR: add util functions to produce and consume records Signed-off-by: Arjun Satish --- .../kafka/connect/integration/DLQTest.java | 19 ++- .../util/clusters/EmbeddedConnectCluster.java | 6 +- .../util/clusters/EmbeddedKafkaCluster.java | 149 +++++++++++++++++- .../src/test/resources/log4j.properties | 8 +- 4 files changed, 165 insertions(+), 17 deletions(-) diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/DLQTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/DLQTest.java index dced9b2d8d11b..e7919626f7b39 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/DLQTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/DLQTest.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.connect.integration; +import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.connect.util.clusters.EmbeddedConnectCluster; import org.junit.ClassRule; import org.junit.Test; @@ -27,11 +28,23 @@ public class DLQTest { private static final Logger log = LoggerFactory.getLogger(DLQTest.class); @ClassRule - public static EmbeddedConnectCluster connectCluster = new EmbeddedConnectCluster(); + public static EmbeddedConnectCluster connect = new EmbeddedConnectCluster(); @Test public void startConnect() { - - } + // create test topic + connect.kafka().createTopic("test-topic"); + + // produce some strings into test topic + for (int i = 0; i < 1000; i++) { + connect.kafka().produce("test-topic", "hello-" + i); + connect.kafka().produce("test-topic", "world-" + i); + } + // consume all records from test topic + log.info("Consuming records from test topic"); + for (ConsumerRecord recs : connect.kafka().consumeNRecords(2000, 5000, "test-topic")) { + log.info("Consumed record ({}, {}) from topic {}", recs.key(), new String(recs.value()), recs.topic()); + } + } } 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 15f1c70f9a6a7..9d59a7e5c2663 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 @@ -65,7 +65,7 @@ public EmbeddedConnectCluster(Map workerProps) { protected void before() throws IOException { kafkaCluster.before(); start(); - log.info("Started connect at {} with kafka cluster at {}", restUrl(), kafkaCluster().bootstrapServers()); + log.info("Started connect at {} with kafka cluster at {}", restUrl(), kafka().bootstrapServers()); } @Override @@ -83,7 +83,7 @@ protected void after() { public void start() throws IOException { log.info("Starting standalone connect cluster.."); - workerProps.put("bootstrap.servers", kafkaCluster().bootstrapServers()); + workerProps.put("bootstrap.servers", kafka().bootstrapServers()); workerProps.put("rest.host.name", "localhost"); putIfAbsent(workerProps, "key.converter", "org.apache.kafka.connect.json.JsonConverter"); @@ -126,7 +126,7 @@ public URI restUrl() { return advertisedUrl; } - public EmbeddedKafkaCluster kafkaCluster() { + public EmbeddedKafkaCluster kafka() { return kafkaCluster; } } \ No newline at end of file diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java b/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java index f7dbf17672967..7da8fa9855dc5 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java @@ -22,16 +22,40 @@ import kafka.utils.CoreUtils; import kafka.utils.TestUtils; import kafka.zk.EmbeddedZookeeper; +import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.clients.admin.AdminClient; +import org.apache.kafka.clients.admin.AdminClientConfig; +import org.apache.kafka.clients.admin.NewTopic; +import org.apache.kafka.clients.consumer.ConsumerConfig; +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.clients.producer.ProducerConfig; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.config.SslConfigs; +import org.apache.kafka.common.config.types.Password; import org.apache.kafka.common.network.ListenerName; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Time; +import org.apache.kafka.connect.errors.ConnectException; import org.junit.rules.ExternalResource; import org.junit.rules.TemporaryFolder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; +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.Properties; +import java.util.UUID; +import java.util.concurrent.ExecutionException; public class EmbeddedKafkaCluster extends ExternalResource { @@ -44,6 +68,7 @@ public class EmbeddedKafkaCluster extends ExternalResource { private EmbeddedZookeeper zookeeper = null; private ListenerName listenerName = new ListenerName("PLAINTEXT"); + private KafkaProducer dlqProducer; public EmbeddedKafkaCluster(final int numBrokers, final Properties brokerConfig) { @@ -75,7 +100,7 @@ public void start() throws IOException { putIfAbsent(brokerConfig, KafkaConfig$.MODULE$.DeleteTopicEnableProp(), true); putIfAbsent(brokerConfig, KafkaConfig$.MODULE$.GroupInitialRebalanceDelayMsProp(), 0); putIfAbsent(brokerConfig, KafkaConfig$.MODULE$.OffsetsTopicReplicationFactorProp(), (short) 1); - putIfAbsent(brokerConfig, KafkaConfig$.MODULE$.AutoCreateTopicsEnableProp(), true); + putIfAbsent(brokerConfig, KafkaConfig$.MODULE$.AutoCreateTopicsEnableProp(), false); Object listenerConfig = brokerConfig.get(KafkaConfig$.MODULE$.InterBrokerListenerNameProp()); if (listenerConfig != null) { @@ -87,15 +112,27 @@ public void start() throws IOException { brokerConfig.put(KafkaConfig$.MODULE$.LogDirProp(), createLogDir()); brokers[i] = TestUtils.createServer(new KafkaConfig(brokerConfig, true), time); } + + Map producerProps = new HashMap<>(); + producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers()); + producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer"); + producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer"); + dlqProducer = new KafkaProducer<>(producerProps); } private void stop() { + try { + dlqProducer.close(); + } catch (Exception e) { + log.error("Could not shutdown producer ", e); + } + for (KafkaServer broker : brokers) { try { broker.shutdown(); } catch (Throwable t) { - log.error("Could not shutdown broker at " + address(broker), t); + log.error("Could not shutdown broker at {}", address(broker), t); } } @@ -104,14 +141,14 @@ private void stop() { log.info("Cleaning up kafka log dirs at {}", broker.config().logDirs()); CoreUtils.delete(broker.config().logDirs()); } catch (Throwable t) { - log.error("Could not clean up log dirs for broker " + address(broker), t); + log.error("Could not clean up log dirs for broker at {}", address(broker), t); } } try { zookeeper.shutdown(); } catch (Throwable t) { - log.error("Could not shutdown zookeeper at " + zKConnectString(), t); + log.error("Could not shutdown zookeeper at {}", zKConnectString(), t); } } @@ -139,4 +176,108 @@ public String zKConnectString() { return "127.0.0.1:" + zookeeper.port(); } + /** + * Create a Kafka topic with 1 partition and a replication factor of 1. + * + * @param topic The name of the topic. + */ + public void createTopic(String topic) { + createTopic(topic, 1, 1, new HashMap<>()); + } + + /** + * Create a Kafka topic with the given parameters. + * + * @param topic The name of the topic. + * @param partitions The number of partitions for this topic. + * @param replication The replication factor for (partitions of) this topic. + * @param topicConfig Additional topic-level configuration settings. + */ + public void createTopic(String topic, int partitions, int replication, Map topicConfig) { + replication = Math.min(replication, brokers.length); + log.debug("Creating topic { name: {}, partitions: {}, replication: {}, config: {} }", + topic, partitions, replication, topicConfig); + final NewTopic newTopic = new NewTopic(topic, partitions, (short) replication); + newTopic.configs(topicConfig); + + try (final AdminClient adminClient = createAdminClient()) { + adminClient.createTopics(Collections.singletonList(newTopic)).all().get(); + } catch (final InterruptedException | ExecutionException e) { + throw new RuntimeException(e); + } + } + + public void produce(String topic, String value) { + produce(topic, null, value); + } + + public void produce(String topic, String key, String value) { + dlqProducer.send(new ProducerRecord<>(topic, key == null ? null : key.getBytes(), value == null ? null : value.getBytes())); + } + + public AdminClient createAdminClient() { + final Properties adminClientConfig = new Properties(); + adminClientConfig.put(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers()); + final Object listeners = brokerConfig.get(KafkaConfig$.MODULE$.ListenersProp()); + if (listeners != null && listeners.toString().contains("SSL")) { + adminClientConfig.put(SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG, brokerConfig.get(SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG)); + adminClientConfig.put(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, ((Password) brokerConfig.get(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG)).value()); + adminClientConfig.put(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, "SSL"); + } + return AdminClient.create(adminClientConfig); + } + + public ConsumerRecords consumeNRecords(int n, long maxDuration, String... topics) { + long num = 1; + long duration = maxDuration; + if (duration > 250) { + num = 1 + (maxDuration / 250); + duration = 250; + } + + Map>> records = new HashMap<>(); + int consumedRecords = 0; + try (KafkaConsumer consumer = createConsumer(topics)) { + for (int i = 0; i < num; i++) { + ConsumerRecords rec = consumer.poll(Duration.ofMillis(duration)); + if (rec.isEmpty()) { + continue; + } + for (TopicPartition partition: rec.partitions()) { + final List> r = rec.records(partition); + records.computeIfAbsent(partition, t -> new ArrayList<>()).addAll(r); + consumedRecords += r.size(); + } + if (consumedRecords >= n) { + return new ConsumerRecords<>(records); + } + } + } + + throw new RuntimeException("Could not find enough records. found " + consumedRecords + ", expected " + n); + } + + public KafkaConsumer createConsumer(String... topics) { + // Include any unknown worker configs so consumer configs can be set globally on the worker + // and through to the task + Map props = new HashMap<>(); + + props.put(ConsumerConfig.GROUP_ID_CONFIG, UUID.randomUUID().toString()); + props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers()); + props.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false"); + props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); + props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArrayDeserializer"); + props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArrayDeserializer"); + + KafkaConsumer consumer; + try { + consumer = new KafkaConsumer<>(props); + } catch (Throwable t) { + throw new ConnectException("Failed to create consumer", t); + } + + consumer.subscribe(Arrays.asList(topics)); + return consumer; + } + } diff --git a/connect/runtime/src/test/resources/log4j.properties b/connect/runtime/src/test/resources/log4j.properties index ee2c5c8619cdd..d5e90fe788f76 100644 --- a/connect/runtime/src/test/resources/log4j.properties +++ b/connect/runtime/src/test/resources/log4j.properties @@ -14,16 +14,10 @@ # See the License for the specific language governing permissions and # limitations under the License. ## -log4j.rootLogger=INFO, stdout +log4j.rootLogger=OFF, stdout -# Redirect log messages to console log4j.appender.stdout=org.apache.log4j.ConsoleAppender -log4j.appender.stdout.Target=System.out log4j.appender.stdout.layout=org.apache.log4j.PatternLayout log4j.appender.stdout.layout.ConversionPattern=[%d] %p %m (%c:%L)%n -log4j.logger.org.reflections=ERROR log4j.logger.org.apache.kafka=ERROR -log4j.logger.org.apache.kafka.connect=INFO -log4j.logger.org.eclipse.jetty=ERROR -log4j.logger.org.apache.zookeeper=ERROR From 305a5d473ab72dea8c1f8dda2518a40c726761cd Mon Sep 17 00:00:00 2001 From: Arjun Satish Date: Thu, 16 Aug 2018 18:05:34 -0700 Subject: [PATCH 04/52] MINOR: Add monitorable sink connector Signed-off-by: Arjun Satish --- .../kafka/connect/integration/DLQTest.java | 25 ++++- .../util/MonitorableSinkConnector.java | 92 +++++++++++++++++++ .../util/clusters/EmbeddedConnectCluster.java | 31 ++++++- 3 files changed, 146 insertions(+), 2 deletions(-) create mode 100644 connect/runtime/src/test/java/org/apache/kafka/connect/util/MonitorableSinkConnector.java diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/DLQTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/DLQTest.java index e7919626f7b39..230f1c5955eb6 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/DLQTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/DLQTest.java @@ -17,12 +17,17 @@ package org.apache.kafka.connect.integration; import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.connect.storage.StringConverter; +import org.apache.kafka.connect.util.MonitorableSinkConnector; import org.apache.kafka.connect.util.clusters.EmbeddedConnectCluster; import org.junit.ClassRule; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.HashMap; +import java.util.Map; + public class DLQTest { private static final Logger log = LoggerFactory.getLogger(DLQTest.class); @@ -31,7 +36,7 @@ public class DLQTest { public static EmbeddedConnectCluster connect = new EmbeddedConnectCluster(); @Test - public void startConnect() { + public void startConnect() throws InterruptedException { // create test topic connect.kafka().createTopic("test-topic"); @@ -46,5 +51,23 @@ public void startConnect() { for (ConsumerRecord recs : connect.kafka().consumeNRecords(2000, 5000, "test-topic")) { log.info("Consumed record ({}, {}) from topic {}", recs.key(), new String(recs.value()), recs.topic()); } + + log.info("Connect endpoint: {}", connect.restUrl()); + + Map confs = new HashMap<>(); + confs.put("connector.class", "MonitorableSink"); + confs.put("task.max", "2"); + confs.put("topics", "test-topic"); + confs.put("key.converter", StringConverter.class.getName()); + confs.put("value.converter", StringConverter.class.getName()); + + connect.startConnector("simple-conn", confs); + + while (MonitorableSinkConnector.COUNTER.get() < 2000) { + Thread.sleep(500); + } + + log.info("Connector read {} records from topic", MonitorableSinkConnector.COUNTER.get()); + connect.deleteConnector("simple-conn"); } } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/util/MonitorableSinkConnector.java b/connect/runtime/src/test/java/org/apache/kafka/connect/util/MonitorableSinkConnector.java new file mode 100644 index 0000000000000..5fc4940013b45 --- /dev/null +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/util/MonitorableSinkConnector.java @@ -0,0 +1,92 @@ +/* + * 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.common.config.ConfigDef; +import org.apache.kafka.connect.connector.Task; +import org.apache.kafka.connect.runtime.TestSinkConnector; +import org.apache.kafka.connect.sink.SinkRecord; +import org.apache.kafka.connect.sink.SinkTask; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicInteger; + +public class MonitorableSinkConnector extends TestSinkConnector { + + private static final Logger log = LoggerFactory.getLogger(MonitorableSinkConnector.class); + + public static AtomicInteger COUNTER = new AtomicInteger(); + + @Override + public void start(Map props) { + log.info("Starting connector"); + } + + @Override + public Class taskClass() { + return SimpleTestSinkTask.class; + } + + @Override + public List> taskConfigs(int maxTasks) { + List> configs = new ArrayList<>(); + for (int i = 0; i < maxTasks; i++) { + Map config = new HashMap<>(); + config.put("required", "dummy-val"); + configs.add(config); + } + return configs; + } + + @Override + public ConfigDef config() { + return new ConfigDef(); + } + + public static class SimpleTestSinkTask extends SinkTask { + + @Override + public String version() { + return null; + } + + @Override + public void start(Map props) { + log.debug("Starting task {}", context); + } + + @Override + public void put(Collection records) { + for (SinkRecord rec : records) { + COUNTER.incrementAndGet(); + log.debug("Obtained record: {} at {}", rec.value(), context); + } + } + + @Override + public void stop() { + } + + } + +} 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 9d59a7e5c2663..77b1da1e4526c 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 @@ -18,13 +18,16 @@ import org.apache.kafka.common.utils.Utils; import org.apache.kafka.connect.runtime.Connect; +import org.apache.kafka.connect.runtime.Herder; import org.apache.kafka.connect.runtime.Worker; import org.apache.kafka.connect.runtime.isolation.Plugins; import org.apache.kafka.connect.runtime.rest.RestServer; +import org.apache.kafka.connect.runtime.rest.entities.ConnectorInfo; import org.apache.kafka.connect.runtime.standalone.StandaloneConfig; import org.apache.kafka.connect.runtime.standalone.StandaloneHerder; import org.apache.kafka.connect.storage.FileOffsetBackingStore; import org.apache.kafka.connect.util.ConnectUtils; +import org.apache.kafka.connect.util.FutureCallback; import org.junit.rules.ExternalResource; import org.junit.rules.TemporaryFolder; import org.slf4j.Logger; @@ -36,6 +39,9 @@ import java.util.HashMap; import java.util.Map; import java.util.Properties; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; public class EmbeddedConnectCluster extends ExternalResource { @@ -51,6 +57,7 @@ public class EmbeddedConnectCluster extends ExternalResource { private File offsetsDirectory; private Connect connect; private URI advertisedUrl; + private Herder herder; public EmbeddedConnectCluster() { // this empty map will be populated with defaults before starting Connect. @@ -106,10 +113,32 @@ public void start() throws IOException { Worker worker = new Worker(workerId, kafkaCluster.time(), plugins, config, new FileOffsetBackingStore()); - connect = new Connect(new StandaloneHerder(worker, ConnectUtils.lookupKafkaClusterId(config)), rest); + herder = new StandaloneHerder(worker, ConnectUtils.lookupKafkaClusterId(config)); + connect = new Connect(herder, rest); connect.start(); } + public void startConnector(String connName, Map connConfig) { + connConfig.put("name", connName); + FutureCallback> cb = new FutureCallback<>(); + herder.putConnectorConfig(connName, connConfig, true, cb); + try { + cb.get(60, TimeUnit.SECONDS); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + public void deleteConnector(String connName) { + FutureCallback> cb = new FutureCallback<>(); + herder.deleteConnectorConfig(connName, cb); + try { + cb.get(60, TimeUnit.SECONDS); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + private void putIfAbsent(Map props, String propertyKey, String propertyValue) { if (!props.containsKey(propertyKey)) { props.put(propertyKey, propertyValue); From b0478254ed7c06678988b07e82eb84a9348680fb Mon Sep 17 00:00:00 2001 From: Arjun Satish Date: Mon, 20 Aug 2018 15:37:14 -0700 Subject: [PATCH 05/52] Rename variable to producer Signed-off-by: Arjun Satish --- .../kafka/connect/util/clusters/EmbeddedKafkaCluster.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java b/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java index 7da8fa9855dc5..633dcc0b666d3 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java @@ -68,7 +68,7 @@ public class EmbeddedKafkaCluster extends ExternalResource { private EmbeddedZookeeper zookeeper = null; private ListenerName listenerName = new ListenerName("PLAINTEXT"); - private KafkaProducer dlqProducer; + private KafkaProducer producer; public EmbeddedKafkaCluster(final int numBrokers, final Properties brokerConfig) { @@ -117,13 +117,13 @@ public void start() throws IOException { producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers()); producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer"); producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer"); - dlqProducer = new KafkaProducer<>(producerProps); + producer = new KafkaProducer<>(producerProps); } private void stop() { try { - dlqProducer.close(); + producer.close(); } catch (Exception e) { log.error("Could not shutdown producer ", e); } @@ -212,7 +212,7 @@ public void produce(String topic, String value) { } public void produce(String topic, String key, String value) { - dlqProducer.send(new ProducerRecord<>(topic, key == null ? null : key.getBytes(), value == null ? null : value.getBytes())); + producer.send(new ProducerRecord<>(topic, key == null ? null : key.getBytes(), value == null ? null : value.getBytes())); } public AdminClient createAdminClient() { From 52d44ceda0c80ce3e15e4a4c0efe1406f52d9011 Mon Sep 17 00:00:00 2001 From: Arjun Satish Date: Fri, 24 Aug 2018 11:03:23 -0700 Subject: [PATCH 06/52] Add some javadocs Signed-off-by: Arjun Satish --- .../{DLQTest.java => ConnectIntegrationTest.java} | 10 +++++++--- .../connect/util/clusters/EmbeddedKafkaCluster.java | 8 ++++++++ 2 files changed, 15 insertions(+), 3 deletions(-) rename connect/runtime/src/test/java/org/apache/kafka/connect/integration/{DLQTest.java => ConnectIntegrationTest.java} (86%) diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/DLQTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTest.java similarity index 86% rename from connect/runtime/src/test/java/org/apache/kafka/connect/integration/DLQTest.java rename to connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTest.java index 230f1c5955eb6..2d4d2fa876140 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/DLQTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTest.java @@ -28,15 +28,19 @@ import java.util.HashMap; import java.util.Map; -public class DLQTest { +public class ConnectIntegrationTest { - private static final Logger log = LoggerFactory.getLogger(DLQTest.class); + private static final Logger log = LoggerFactory.getLogger(ConnectIntegrationTest.class); @ClassRule public static EmbeddedConnectCluster connect = new EmbeddedConnectCluster(); + /** + * Simple test case to bring up an embedded Connect cluster along a backing Kafka and Zk process. + * The test will produce and consume records, and start up a sink connector which will consume these records. + */ @Test - public void startConnect() throws InterruptedException { + public void testProduceConsumeConnector() throws Exception { // create test topic connect.kafka().createTopic("test-topic"); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java b/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java index 633dcc0b666d3..b6f9daedaa436 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java @@ -227,6 +227,14 @@ public AdminClient createAdminClient() { return AdminClient.create(adminClientConfig); } + /** + * Consume at least n records in a given duration or throw an exception. + * + * @param n the number of expected records in this topic. + * @param maxDuration the max duration to wait for these records. + * @param topics the topics to subscribe and consume records from. + * @return a {@link ConsumerRecords} collection containing at least n records. + */ public ConsumerRecords consumeNRecords(int n, long maxDuration, String... topics) { long num = 1; long duration = maxDuration; From b9b8c86e36e6c6d18146066fe2ad4896cb5b57b2 Mon Sep 17 00:00:00 2001 From: Arjun Satish Date: Fri, 24 Aug 2018 11:03:43 -0700 Subject: [PATCH 07/52] DLQ integration test Signed-off-by: Arjun Satish --- .../DeadLetterQueueIntegrationTest.java | 132 ++++++++++++++++++ 1 file changed, 132 insertions(+) create mode 100644 connect/runtime/src/test/java/org/apache/kafka/connect/integration/DeadLetterQueueIntegrationTest.java diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/DeadLetterQueueIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/DeadLetterQueueIntegrationTest.java new file mode 100644 index 0000000000000..f47e3b698bacb --- /dev/null +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/DeadLetterQueueIntegrationTest.java @@ -0,0 +1,132 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.connect.integration; + +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.connect.connector.ConnectRecord; +import org.apache.kafka.connect.errors.RetriableException; +import org.apache.kafka.connect.storage.StringConverter; +import org.apache.kafka.connect.transforms.Transformation; +import org.apache.kafka.connect.util.MonitorableSinkConnector; +import org.apache.kafka.connect.util.clusters.EmbeddedConnectCluster; +import org.junit.Assert; +import org.junit.ClassRule; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.HashMap; +import java.util.Map; + +public class DeadLetterQueueIntegrationTest { + + private static final Logger log = LoggerFactory.getLogger(DeadLetterQueueIntegrationTest.class); + + private static final String DLQ_TOPIC = "my-connector-errors"; + + @ClassRule + public static EmbeddedConnectCluster connect = new EmbeddedConnectCluster(); + + @Test + public void testTransformationErrorHandlingWithDeadLetterQueue() throws Exception { + // create test topic + connect.kafka().createTopic("test-topic"); + + // produce some strings into test topic + for (int i = 0; i < 10; i++) { + connect.kafka().produce("test-topic", "hello-" + i, String.valueOf(i)); + connect.kafka().produce("test-topic", "world-" + i, String.valueOf(i)); + } + + // consume all records from test topic + log.info("Consuming records from test topic"); + for (ConsumerRecord recs : connect.kafka().consumeNRecords(20, 5000, "test-topic")) { + log.info("Consumed record ({}, {}) from topic {}", recs.key(), new String(recs.value()), recs.topic()); + } + + Map confs = new HashMap<>(); + confs.put("connector.class", "MonitorableSink"); + confs.put("task.max", "2"); + confs.put("topics", "test-topic"); + confs.put("key.converter", StringConverter.class.getName()); + confs.put("value.converter", StringConverter.class.getName()); + + confs.put("errors.log.enable", "true"); + confs.put("errors.log.include.messages", "false"); + + confs.put("errors.deadletterqueue.topic.name", DLQ_TOPIC); + confs.put("errors.deadletterqueue.topic.replication.factor", "1"); + confs.put("errors.tolerance", "all"); + confs.put("transforms", "failing_transform"); + confs.put("transforms.failing_transform.type", FaultyPassthrough.class.getName()); + + connect.startConnector("simple-conn", confs); + + int attempts = 0; + while (attempts++ < 5 && MonitorableSinkConnector.COUNTER.get() < 18) { + log.info("Received only {} records. Waiting .. ", MonitorableSinkConnector.COUNTER.get()); + Thread.sleep(500); + } + + Assert.assertEquals(18, MonitorableSinkConnector.COUNTER.get()); + + // consume failed records from dead letter queue topic + log.info("Consuming records from test topic"); + for (ConsumerRecord recs : connect.kafka().consumeNRecords(2, 5000, DLQ_TOPIC)) { + log.info("Consumed record ({}, {}) from dead letter queue topic {}", new String(recs.key()), new String(recs.value()), DLQ_TOPIC); + } + + connect.deleteConnector("simple-conn"); + + } + + public static class FaultyPassthrough> implements Transformation { + + private static final Logger log = LoggerFactory.getLogger(FaultyPassthrough.class); + + public static final ConfigDef CONFIG_DEF = new ConfigDef(); + + private int invocations = 0; + + @Override + public R apply(R record) { + long val = Long.parseLong(String.valueOf(record.value())); + if (val == 7) { + log.debug("Failing record: {} at invocations={}", record, invocations); + throw new RetriableException("Bad invocations " + invocations + " for val = " + val); + } + return record; + } + + @Override + public ConfigDef config() { + return CONFIG_DEF; + } + + @Override + public void close() { + log.info("Shutting down transform"); + } + + @Override + public void configure(Map configs) { + log.info("Configuring {}.", this.getClass()); + } + } + +} From 411b678255cb2d972aa882ad0e227845e518d49d Mon Sep 17 00:00:00 2001 From: Arjun Satish Date: Fri, 24 Aug 2018 11:18:05 -0700 Subject: [PATCH 08/52] Wait for only limited time for records to come through Signed-off-by: Arjun Satish --- .../kafka/connect/integration/ConnectIntegrationTest.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTest.java index 2d4d2fa876140..0c5f7f900f77e 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTest.java @@ -67,7 +67,8 @@ public void testProduceConsumeConnector() throws Exception { connect.startConnector("simple-conn", confs); - while (MonitorableSinkConnector.COUNTER.get() < 2000) { + int attempts = 0; + while (attempts++ < 5 && MonitorableSinkConnector.COUNTER.get() < 2000) { Thread.sleep(500); } From c5c3494de596394a676f1690f44228954308dec3 Mon Sep 17 00:00:00 2001 From: Arjun Satish Date: Fri, 24 Aug 2018 13:52:25 -0700 Subject: [PATCH 09/52] Use REST endpoint to start/delete connectors Signed-off-by: Arjun Satish --- .../integration/ConnectIntegrationTest.java | 3 + .../DeadLetterQueueIntegrationTest.java | 3 + .../util/clusters/EmbeddedConnectCluster.java | 96 ++++++++++++++----- 3 files changed, 80 insertions(+), 22 deletions(-) diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTest.java index 0c5f7f900f77e..274259ac8f40f 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTest.java @@ -20,14 +20,17 @@ import org.apache.kafka.connect.storage.StringConverter; import org.apache.kafka.connect.util.MonitorableSinkConnector; import org.apache.kafka.connect.util.clusters.EmbeddedConnectCluster; +import org.apache.kafka.test.IntegrationTest; import org.junit.ClassRule; import org.junit.Test; +import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.util.HashMap; import java.util.Map; +@Category(IntegrationTest.class) public class ConnectIntegrationTest { private static final Logger log = LoggerFactory.getLogger(ConnectIntegrationTest.class); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/DeadLetterQueueIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/DeadLetterQueueIntegrationTest.java index f47e3b698bacb..0cff8fff1983a 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/DeadLetterQueueIntegrationTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/DeadLetterQueueIntegrationTest.java @@ -24,15 +24,18 @@ import org.apache.kafka.connect.transforms.Transformation; import org.apache.kafka.connect.util.MonitorableSinkConnector; import org.apache.kafka.connect.util.clusters.EmbeddedConnectCluster; +import org.apache.kafka.test.IntegrationTest; import org.junit.Assert; import org.junit.ClassRule; import org.junit.Test; +import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.util.HashMap; import java.util.Map; +@Category(IntegrationTest.class) public class DeadLetterQueueIntegrationTest { private static final Logger log = LoggerFactory.getLogger(DeadLetterQueueIntegrationTest.class); 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 77b1da1e4526c..d52dfa6abfa75 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 @@ -16,18 +16,18 @@ */ package org.apache.kafka.connect.util.clusters; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.connect.runtime.Connect; import org.apache.kafka.connect.runtime.Herder; import org.apache.kafka.connect.runtime.Worker; import org.apache.kafka.connect.runtime.isolation.Plugins; import org.apache.kafka.connect.runtime.rest.RestServer; -import org.apache.kafka.connect.runtime.rest.entities.ConnectorInfo; import org.apache.kafka.connect.runtime.standalone.StandaloneConfig; import org.apache.kafka.connect.runtime.standalone.StandaloneHerder; import org.apache.kafka.connect.storage.FileOffsetBackingStore; import org.apache.kafka.connect.util.ConnectUtils; -import org.apache.kafka.connect.util.FutureCallback; import org.junit.rules.ExternalResource; import org.junit.rules.TemporaryFolder; import org.slf4j.Logger; @@ -35,13 +35,14 @@ import java.io.File; import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStreamWriter; +import java.net.HttpURLConnection; import java.net.URI; +import java.net.URL; import java.util.HashMap; import java.util.Map; import java.util.Properties; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; public class EmbeddedConnectCluster extends ExternalResource { @@ -68,10 +69,21 @@ public EmbeddedConnectCluster(Map workerProps) { this.workerProps = workerProps; } + public void start() throws IOException { + before(); + } + + /** + * Stop the connect cluster and the embedded Kafka and + */ + public void stop() { + after(); + } + @Override protected void before() throws IOException { kafkaCluster.before(); - start(); + startConnect(); log.info("Started connect at {} with kafka cluster at {}", restUrl(), kafka().bootstrapServers()); } @@ -84,11 +96,20 @@ protected void after() { log.error("Could not delete directory at {}", offsetsDirectory, e); } - connect.stop(); - kafkaCluster.after(); + try { + connect.stop(); + } catch (Exception e) { + log.error("Could not stop connect", e); + } + + try { + kafkaCluster.after(); + } catch (Exception e) { + log.error("Could not stop kafka", e); + } } - public void start() throws IOException { + public void startConnect() throws IOException { log.info("Starting standalone connect cluster.."); workerProps.put("bootstrap.servers", kafka().bootstrapServers()); workerProps.put("rest.host.name", "localhost"); @@ -118,24 +139,25 @@ public void start() throws IOException { connect.start(); } - public void startConnector(String connName, Map connConfig) { - connConfig.put("name", connName); - FutureCallback> cb = new FutureCallback<>(); - herder.putConnectorConfig(connName, connConfig, true, cb); + public void startConnector(String connName, Map connConfig) throws IOException { + String url = String.format("http://localhost:8083/connectors/%s/config", connName); + ObjectMapper mapper = new ObjectMapper(); try { - cb.get(60, TimeUnit.SECONDS); - } catch (Exception e) { + String content = mapper.writeValueAsString(connConfig); + int status = executePut(url , content); + if (status >= 400) { + throw new IOException("Could not execute PUT request. status=" + status); + } + } catch (JsonProcessingException e) { + log.error("Could not serialize config", e); throw new RuntimeException(e); } } - public void deleteConnector(String connName) { - FutureCallback> cb = new FutureCallback<>(); - herder.deleteConnectorConfig(connName, cb); - try { - cb.get(60, TimeUnit.SECONDS); - } catch (Exception e) { - throw new RuntimeException(e); + public void deleteConnector(String connName) throws IOException { + int status = executeDelete(String.format("http://localhost:8083/connectors/%s", connName)); + if (status >= 400) { + throw new IOException("Could not execute DELETE request. status=" + status); } } @@ -158,4 +180,34 @@ public URI restUrl() { public EmbeddedKafkaCluster kafka() { return kafkaCluster; } + + public int executePut(String url, String body) throws IOException { + log.debug("Executing PUT request to URL={}. Payload={}", url, body); + HttpURLConnection httpCon = (HttpURLConnection) new URL(url).openConnection(); + httpCon.setDoOutput(true); + httpCon.setRequestProperty("Content-Type", "application/json"); + httpCon.setRequestMethod("PUT"); + OutputStreamWriter out = new OutputStreamWriter(httpCon.getOutputStream()); + out.write(body); + out.close(); + try (InputStream is = httpCon.getInputStream()) { + int c; + StringBuilder response = new StringBuilder(); + while ((c = is.read()) != -1) { + response.append((char) c); + } + log.info("Put response for URL={} is {}", url, response); + } + return httpCon.getResponseCode(); + } + + public int executeDelete(String url) throws IOException { + log.debug("Executing DELETE request to URL={}", url); + HttpURLConnection httpCon = (HttpURLConnection) new URL(url).openConnection(); + httpCon.setDoOutput(true); + httpCon.setRequestMethod("DELETE"); + httpCon.connect(); + return httpCon.getResponseCode(); + } + } \ No newline at end of file From 041490a885a22ec17292284ac1e3cf7c7c338162 Mon Sep 17 00:00:00 2001 From: Arjun Satish Date: Fri, 24 Aug 2018 14:27:21 -0700 Subject: [PATCH 10/52] Use distributed herder to bring up connect cluster Signed-off-by: Arjun Satish --- .../integration/ConnectIntegrationTest.java | 2 +- .../DeadLetterQueueIntegrationTest.java | 4 +- .../util/clusters/EmbeddedConnectCluster.java | 76 +++++++++++-------- .../src/test/resources/log4j.properties | 12 ++- 4 files changed, 59 insertions(+), 35 deletions(-) diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTest.java index 274259ac8f40f..1a69caa6b6739 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTest.java @@ -36,7 +36,7 @@ public class ConnectIntegrationTest { private static final Logger log = LoggerFactory.getLogger(ConnectIntegrationTest.class); @ClassRule - public static EmbeddedConnectCluster connect = new EmbeddedConnectCluster(); + public static EmbeddedConnectCluster connect = new EmbeddedConnectCluster(ConnectIntegrationTest.class); /** * Simple test case to bring up an embedded Connect cluster along a backing Kafka and Zk process. diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/DeadLetterQueueIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/DeadLetterQueueIntegrationTest.java index 0cff8fff1983a..1500037e49ade 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/DeadLetterQueueIntegrationTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/DeadLetterQueueIntegrationTest.java @@ -43,7 +43,7 @@ public class DeadLetterQueueIntegrationTest { private static final String DLQ_TOPIC = "my-connector-errors"; @ClassRule - public static EmbeddedConnectCluster connect = new EmbeddedConnectCluster(); + public static EmbeddedConnectCluster connect = new EmbeddedConnectCluster(DeadLetterQueueIntegrationTest.class); @Test public void testTransformationErrorHandlingWithDeadLetterQueue() throws Exception { @@ -59,7 +59,7 @@ public void testTransformationErrorHandlingWithDeadLetterQueue() throws Exceptio // consume all records from test topic log.info("Consuming records from test topic"); for (ConsumerRecord recs : connect.kafka().consumeNRecords(20, 5000, "test-topic")) { - log.info("Consumed record ({}, {}) from topic {}", recs.key(), new String(recs.value()), recs.topic()); + log.debug("Consumed record ({}, {}) from topic {}", new String(recs.key()), new String(recs.value()), recs.topic()); } Map confs = new HashMap<>(); 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 d52dfa6abfa75..f63fb027759f2 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 @@ -18,22 +18,25 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.kafka.common.utils.Utils; import org.apache.kafka.connect.runtime.Connect; import org.apache.kafka.connect.runtime.Herder; import org.apache.kafka.connect.runtime.Worker; +import org.apache.kafka.connect.runtime.WorkerConfigTransformer; +import org.apache.kafka.connect.runtime.distributed.DistributedConfig; +import org.apache.kafka.connect.runtime.distributed.DistributedHerder; import org.apache.kafka.connect.runtime.isolation.Plugins; import org.apache.kafka.connect.runtime.rest.RestServer; -import org.apache.kafka.connect.runtime.standalone.StandaloneConfig; -import org.apache.kafka.connect.runtime.standalone.StandaloneHerder; -import org.apache.kafka.connect.storage.FileOffsetBackingStore; +import org.apache.kafka.connect.storage.ConfigBackingStore; +import org.apache.kafka.connect.storage.Converter; +import org.apache.kafka.connect.storage.KafkaConfigBackingStore; +import org.apache.kafka.connect.storage.KafkaOffsetBackingStore; +import org.apache.kafka.connect.storage.KafkaStatusBackingStore; +import org.apache.kafka.connect.storage.StatusBackingStore; import org.apache.kafka.connect.util.ConnectUtils; import org.junit.rules.ExternalResource; -import org.junit.rules.TemporaryFolder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.File; import java.io.IOException; import java.io.InputStream; import java.io.OutputStreamWriter; @@ -54,19 +57,23 @@ public class EmbeddedConnectCluster extends ExternalResource { private final EmbeddedKafkaCluster kafkaCluster = new EmbeddedKafkaCluster(NUM_BROKERS, BROKER_CONFIG); private final Map workerProps; + private final String clusterName; - private File offsetsDirectory; private Connect connect; private URI advertisedUrl; - private Herder herder; - public EmbeddedConnectCluster() { + public EmbeddedConnectCluster(Class klass) { + this(klass.getName()); + } + + public EmbeddedConnectCluster(String name) { // this empty map will be populated with defaults before starting Connect. - this(new HashMap<>()); + this(name, new HashMap<>()); } - public EmbeddedConnectCluster(Map workerProps) { + public EmbeddedConnectCluster(String name, Map workerProps) { this.workerProps = workerProps; + this.clusterName = name; } public void start() throws IOException { @@ -89,13 +96,6 @@ protected void before() throws IOException { @Override protected void after() { - try { - log.info("Cleaning up connect offset dir at {}", offsetsDirectory); - Utils.delete(offsetsDirectory); - } catch (IOException e) { - log.error("Could not delete directory at {}", offsetsDirectory, e); - } - try { connect.stop(); } catch (Exception e) { @@ -109,32 +109,52 @@ protected void after() { } } - public void startConnect() throws IOException { + public void startConnect() { log.info("Starting standalone connect cluster.."); workerProps.put("bootstrap.servers", kafka().bootstrapServers()); workerProps.put("rest.host.name", "localhost"); + putIfAbsent(workerProps, "group.id", "connect-integration-test-" + clusterName); + putIfAbsent(workerProps, "offset.storage.topic", "connect-offset-topic-" + clusterName); + putIfAbsent(workerProps, "offset.storage.replication.factor", "1"); + putIfAbsent(workerProps, "config.storage.topic", "connect-config-topic-" + clusterName); + putIfAbsent(workerProps, "config.storage.replication.factor", "1"); + putIfAbsent(workerProps, "status.storage.topic", "connect-storage-topic-" + clusterName); + putIfAbsent(workerProps, "status.storage.replication.factor", "1"); putIfAbsent(workerProps, "key.converter", "org.apache.kafka.connect.json.JsonConverter"); putIfAbsent(workerProps, "value.converter", "org.apache.kafka.connect.json.JsonConverter"); putIfAbsent(workerProps, "key.converter.schemas.enable", "false"); putIfAbsent(workerProps, "value.converter.schemas.enable", "false"); - offsetsDirectory = createTmpDir(); - putIfAbsent(workerProps, "offset.storage.file.filename", new File(offsetsDirectory, "connect.integration.offsets").getAbsolutePath()); - log.info("Scanning for plugins..."); Plugins plugins = new Plugins(workerProps); plugins.compareAndSwapWithDelegatingLoader(); - StandaloneConfig config = new StandaloneConfig(workerProps); + DistributedConfig config = new DistributedConfig(workerProps); RestServer rest = new RestServer(config); advertisedUrl = rest.advertisedUrl(); String workerId = advertisedUrl.getHost() + ":" + advertisedUrl.getPort(); - Worker worker = new Worker(workerId, kafkaCluster.time(), plugins, config, new FileOffsetBackingStore()); + KafkaOffsetBackingStore offsetBackingStore = new KafkaOffsetBackingStore(); + offsetBackingStore.configure(config); - herder = new StandaloneHerder(worker, ConnectUtils.lookupKafkaClusterId(config)); + Worker worker = new Worker(workerId, kafkaCluster.time(), plugins, config, offsetBackingStore); + + WorkerConfigTransformer configTransformer = worker.configTransformer(); + Converter internalValueConverter = worker.getInternalValueConverter(); + + StatusBackingStore statusBackingStore = new KafkaStatusBackingStore(kafkaCluster.time(), internalValueConverter); + statusBackingStore.configure(config); + + ConfigBackingStore configBackingStore = new KafkaConfigBackingStore( + internalValueConverter, + config, + configTransformer); + + Herder herder = new DistributedHerder(config, kafkaCluster.time(), worker, + ConnectUtils.lookupKafkaClusterId(config), statusBackingStore, configBackingStore, + advertisedUrl.toString()); connect = new Connect(herder, rest); connect.start(); } @@ -167,12 +187,6 @@ private void putIfAbsent(Map props, String propertyKey, String p } } - private File createTmpDir() throws IOException { - TemporaryFolder tmpFolder = new TemporaryFolder(); - tmpFolder.create(); - return tmpFolder.newFolder(); - } - public URI restUrl() { return advertisedUrl; } diff --git a/connect/runtime/src/test/resources/log4j.properties b/connect/runtime/src/test/resources/log4j.properties index d5e90fe788f76..7d54ae4ab9567 100644 --- a/connect/runtime/src/test/resources/log4j.properties +++ b/connect/runtime/src/test/resources/log4j.properties @@ -14,10 +14,20 @@ # See the License for the specific language governing permissions and # limitations under the License. ## -log4j.rootLogger=OFF, stdout +# Root logger option +log4j.rootLogger=INFO, stdout +# Redirect log messages to console log4j.appender.stdout=org.apache.log4j.ConsoleAppender +log4j.appender.stdout.Target=System.out log4j.appender.stdout.layout=org.apache.log4j.PatternLayout log4j.appender.stdout.layout.ConversionPattern=[%d] %p %m (%c:%L)%n +log4j.logger.org.reflections=ERROR log4j.logger.org.apache.kafka=ERROR +log4j.logger.org.apache.kafka.connect=INFO +log4j.logger.org.eclipse.jetty=ERROR +log4j.logger.org.apache.zookeeper=ERROR +log4j.logger.kafka=ERROR +log4j.logger.org.apache.kafka.connect.util.MonitorableSinkConnector=DEBUG + From 93212482d6e2ca68b90337402aef0bafa406ad93 Mon Sep 17 00:00:00 2001 From: Arjun Satish Date: Fri, 24 Aug 2018 15:19:26 -0700 Subject: [PATCH 11/52] Make checkstyle happy and setup and tear down clusters with @Before and @After Signed-off-by: Arjun Satish --- checkstyle/import-control.xml | 2 ++ .../integration/ConnectIntegrationTest.java | 18 ++++++++++++++---- .../DeadLetterQueueIntegrationTest.java | 18 ++++++++++++++---- .../MonitorableSinkConnector.java | 5 +++-- .../util/clusters/EmbeddedConnectCluster.java | 7 +++---- 5 files changed, 36 insertions(+), 14 deletions(-) rename connect/runtime/src/test/java/org/apache/kafka/connect/{util => integration}/MonitorableSinkConnector.java (95%) diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index 4ae3a49999b9b..bd48dedff86f4 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -364,6 +364,7 @@ + @@ -373,6 +374,7 @@ + diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTest.java index 1a69caa6b6739..ee0a359e73c77 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTest.java @@ -18,15 +18,16 @@ import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.connect.storage.StringConverter; -import org.apache.kafka.connect.util.MonitorableSinkConnector; import org.apache.kafka.connect.util.clusters.EmbeddedConnectCluster; import org.apache.kafka.test.IntegrationTest; -import org.junit.ClassRule; +import org.junit.After; +import org.junit.Before; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.IOException; import java.util.HashMap; import java.util.Map; @@ -35,8 +36,17 @@ public class ConnectIntegrationTest { private static final Logger log = LoggerFactory.getLogger(ConnectIntegrationTest.class); - @ClassRule - public static EmbeddedConnectCluster connect = new EmbeddedConnectCluster(ConnectIntegrationTest.class); + private final EmbeddedConnectCluster connect = new EmbeddedConnectCluster(ConnectIntegrationTest.class); + + @Before + public void setup() throws IOException { + connect.start(); + } + + @After + public void close() { + connect.stop(); + } /** * Simple test case to bring up an embedded Connect cluster along a backing Kafka and Zk process. diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/DeadLetterQueueIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/DeadLetterQueueIntegrationTest.java index 1500037e49ade..862b9d4114e72 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/DeadLetterQueueIntegrationTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/DeadLetterQueueIntegrationTest.java @@ -22,16 +22,17 @@ import org.apache.kafka.connect.errors.RetriableException; import org.apache.kafka.connect.storage.StringConverter; import org.apache.kafka.connect.transforms.Transformation; -import org.apache.kafka.connect.util.MonitorableSinkConnector; import org.apache.kafka.connect.util.clusters.EmbeddedConnectCluster; import org.apache.kafka.test.IntegrationTest; +import org.junit.After; import org.junit.Assert; -import org.junit.ClassRule; +import org.junit.Before; import org.junit.Test; import org.junit.experimental.categories.Category; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.IOException; import java.util.HashMap; import java.util.Map; @@ -42,8 +43,17 @@ public class DeadLetterQueueIntegrationTest { private static final String DLQ_TOPIC = "my-connector-errors"; - @ClassRule - public static EmbeddedConnectCluster connect = new EmbeddedConnectCluster(DeadLetterQueueIntegrationTest.class); + private final EmbeddedConnectCluster connect = new EmbeddedConnectCluster(ConnectIntegrationTest.class); + + @Before + public void setup() throws IOException { + connect.start(); + } + + @After + public void close() { + connect.stop(); + } @Test public void testTransformationErrorHandlingWithDeadLetterQueue() throws Exception { diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/util/MonitorableSinkConnector.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/MonitorableSinkConnector.java similarity index 95% rename from connect/runtime/src/test/java/org/apache/kafka/connect/util/MonitorableSinkConnector.java rename to connect/runtime/src/test/java/org/apache/kafka/connect/integration/MonitorableSinkConnector.java index 5fc4940013b45..353b558f60a50 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/util/MonitorableSinkConnector.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/MonitorableSinkConnector.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.kafka.connect.util; +package org.apache.kafka.connect.integration; import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.connect.connector.Task; @@ -35,11 +35,12 @@ public class MonitorableSinkConnector extends TestSinkConnector { private static final Logger log = LoggerFactory.getLogger(MonitorableSinkConnector.class); - public static AtomicInteger COUNTER = new AtomicInteger(); + public static final AtomicInteger COUNTER = new AtomicInteger(); @Override public void start(Map props) { log.info("Starting connector"); + COUNTER.set(0); } @Override 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 f63fb027759f2..249be08468382 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 @@ -16,7 +16,6 @@ */ package org.apache.kafka.connect.util.clusters; -import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.kafka.connect.runtime.Connect; import org.apache.kafka.connect.runtime.Herder; @@ -164,13 +163,13 @@ public void startConnector(String connName, Map connConfig) thro ObjectMapper mapper = new ObjectMapper(); try { String content = mapper.writeValueAsString(connConfig); - int status = executePut(url , content); + int status = executePut(url, content); if (status >= 400) { throw new IOException("Could not execute PUT request. status=" + status); } - } catch (JsonProcessingException e) { + } catch (IOException e) { log.error("Could not serialize config", e); - throw new RuntimeException(e); + throw new IOException(e); } } From eaa9835462d066fd8cdf7ea3e226091eddf1b85a Mon Sep 17 00:00:00 2001 From: Arjun Satish Date: Fri, 24 Aug 2018 15:39:30 -0700 Subject: [PATCH 12/52] Clean up ConnectIntegrationTest and reset COUNTER in stop Signed-off-by: Arjun Satish --- .../connect/integration/ConnectIntegrationTest.java | 10 +++++----- .../integration/DeadLetterQueueIntegrationTest.java | 2 +- .../connect/integration/MonitorableSinkConnector.java | 5 +++++ 3 files changed, 11 insertions(+), 6 deletions(-) diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTest.java index ee0a359e73c77..bd6cc7cfa19f6 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTest.java @@ -16,11 +16,11 @@ */ package org.apache.kafka.connect.integration; -import org.apache.kafka.clients.consumer.ConsumerRecord; 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.Assert; import org.junit.Before; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -63,11 +63,9 @@ public void testProduceConsumeConnector() throws Exception { connect.kafka().produce("test-topic", "world-" + i); } - // consume all records from test topic + // consume all records from test topic or fail log.info("Consuming records from test topic"); - for (ConsumerRecord recs : connect.kafka().consumeNRecords(2000, 5000, "test-topic")) { - log.info("Consumed record ({}, {}) from topic {}", recs.key(), new String(recs.value()), recs.topic()); - } + connect.kafka().consumeNRecords(2000, 5000, "test-topic"); log.info("Connect endpoint: {}", connect.restUrl()); @@ -85,6 +83,8 @@ public void testProduceConsumeConnector() throws Exception { Thread.sleep(500); } + Assert.assertEquals(2000, MonitorableSinkConnector.COUNTER.get()); + log.info("Connector read {} records from topic", MonitorableSinkConnector.COUNTER.get()); connect.deleteConnector("simple-conn"); } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/DeadLetterQueueIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/DeadLetterQueueIntegrationTest.java index 862b9d4114e72..b70a10b60b149 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/DeadLetterQueueIntegrationTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/DeadLetterQueueIntegrationTest.java @@ -56,7 +56,7 @@ public void close() { } @Test - public void testTransformationErrorHandlingWithDeadLetterQueue() throws Exception { + public void testTransformationErrorHandling() throws Exception { // create test topic connect.kafka().createTopic("test-topic"); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/MonitorableSinkConnector.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/MonitorableSinkConnector.java index 353b558f60a50..fa4dd791418d1 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/MonitorableSinkConnector.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/MonitorableSinkConnector.java @@ -59,6 +59,11 @@ public List> taskConfigs(int maxTasks) { return configs; } + @Override + public void stop() { + COUNTER.set(0); + } + @Override public ConfigDef config() { return new ConfigDef(); From 054ba50ae30759cf149384f3274803270fdd095e Mon Sep 17 00:00:00 2001 From: Arjun Satish Date: Mon, 17 Sep 2018 23:26:03 -0700 Subject: [PATCH 13/52] Address reviewer comments Signed-off-by: Arjun Satish --- .../kafka/connect/cli/ConnectDistributed.java | 97 +++++++++--------- .../integration/ConnectIntegrationTest.java | 13 ++- .../DeadLetterQueueIntegrationTest.java | 65 ++++++------ .../integration/MonitorableSinkConnector.java | 7 +- .../util/clusters/EmbeddedConnectCluster.java | 99 ++++++++----------- .../util/clusters/EmbeddedKafkaCluster.java | 10 +- 6 files changed, 145 insertions(+), 146 deletions(-) diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/cli/ConnectDistributed.java b/connect/runtime/src/main/java/org/apache/kafka/connect/cli/ConnectDistributed.java index dd43c37cadda5..4aa0b2798ffc9 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/cli/ConnectDistributed.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/cli/ConnectDistributed.java @@ -54,62 +54,26 @@ public class ConnectDistributed { private static final Logger log = LoggerFactory.getLogger(ConnectDistributed.class); + private Time time = Time.SYSTEM; + private long initStart = time.hiResClockMs(); + public static void main(String[] args) { + if (args.length < 1 || Arrays.asList(args).contains("--help")) { log.info("Usage: ConnectDistributed worker.properties"); Exit.exit(1); } try { - Time time = Time.SYSTEM; - log.info("Kafka Connect distributed worker initializing ..."); - long initStart = time.hiResClockMs(); WorkerInfo initInfo = new WorkerInfo(); initInfo.logAll(); String workerPropsFile = args[0]; Map workerProps = !workerPropsFile.isEmpty() ? - Utils.propsToStringMap(Utils.loadProps(workerPropsFile)) : Collections.emptyMap(); - - log.info("Scanning for plugin classes. This might take a moment ..."); - Plugins plugins = new Plugins(workerProps); - plugins.compareAndSwapWithDelegatingLoader(); - DistributedConfig config = new DistributedConfig(workerProps); - - String kafkaClusterId = ConnectUtils.lookupKafkaClusterId(config); - log.debug("Kafka cluster ID: {}", kafkaClusterId); - - RestServer rest = new RestServer(config); - URI advertisedUrl = rest.advertisedUrl(); - String workerId = advertisedUrl.getHost() + ":" + advertisedUrl.getPort(); - - KafkaOffsetBackingStore offsetBackingStore = new KafkaOffsetBackingStore(); - offsetBackingStore.configure(config); - - Worker worker = new Worker(workerId, time, plugins, config, offsetBackingStore); - WorkerConfigTransformer configTransformer = worker.configTransformer(); - - Converter internalValueConverter = worker.getInternalValueConverter(); - StatusBackingStore statusBackingStore = new KafkaStatusBackingStore(time, internalValueConverter); - statusBackingStore.configure(config); - - ConfigBackingStore configBackingStore = new KafkaConfigBackingStore( - internalValueConverter, - config, - configTransformer); - - DistributedHerder herder = new DistributedHerder(config, time, worker, - kafkaClusterId, statusBackingStore, configBackingStore, - advertisedUrl.toString()); - final Connect connect = new Connect(herder, rest); - log.info("Kafka Connect distributed worker initialization took {}ms", time.hiResClockMs() - initStart); - try { - connect.start(); - } catch (Exception e) { - log.error("Failed to start Connect", e); - connect.stop(); - Exit.exit(3); - } + Utils.propsToStringMap(Utils.loadProps(workerPropsFile)) : Collections.emptyMap(); + + ConnectDistributed connectDistributed = new ConnectDistributed(); + Connect connect = connectDistributed.startConnect(workerProps); // Shutdown will be triggered by Ctrl-C or via HTTP shutdown request connect.awaitStop(); @@ -119,4 +83,49 @@ public static void main(String[] args) { Exit.exit(2); } } + + public Connect startConnect(Map workerProps) { + log.info("Scanning for plugin classes. This might take a moment ..."); + Plugins plugins = new Plugins(workerProps); + plugins.compareAndSwapWithDelegatingLoader(); + DistributedConfig config = new DistributedConfig(workerProps); + + String kafkaClusterId = ConnectUtils.lookupKafkaClusterId(config); + log.debug("Kafka cluster ID: {}", kafkaClusterId); + + RestServer rest = new RestServer(config); + URI advertisedUrl = rest.advertisedUrl(); + String workerId = advertisedUrl.getHost() + ":" + advertisedUrl.getPort(); + + KafkaOffsetBackingStore offsetBackingStore = new KafkaOffsetBackingStore(); + offsetBackingStore.configure(config); + + Worker worker = new Worker(workerId, time, plugins, config, offsetBackingStore); + WorkerConfigTransformer configTransformer = worker.configTransformer(); + + Converter internalValueConverter = worker.getInternalValueConverter(); + StatusBackingStore statusBackingStore = new KafkaStatusBackingStore(time, internalValueConverter); + statusBackingStore.configure(config); + + ConfigBackingStore configBackingStore = new KafkaConfigBackingStore( + internalValueConverter, + config, + configTransformer); + + DistributedHerder herder = new DistributedHerder(config, time, worker, + kafkaClusterId, statusBackingStore, configBackingStore, + advertisedUrl.toString()); + final Connect connect = new Connect(herder, rest); + log.info("Kafka Connect distributed worker initialization took {}ms", time.hiResClockMs() - initStart); + try { + connect.start(); + } catch (Exception e) { + log.error("Failed to start Connect", e); + connect.stop(); + Exit.exit(3); + } + + return connect; + } + } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTest.java index bd6cc7cfa19f6..f69325916aa96 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTest.java @@ -36,10 +36,14 @@ public class ConnectIntegrationTest { private static final Logger log = LoggerFactory.getLogger(ConnectIntegrationTest.class); - private final EmbeddedConnectCluster connect = new EmbeddedConnectCluster(ConnectIntegrationTest.class); + private static final int NUM_RECORDS_PRODUCED = 2000; + private static final int CONSUME_MAX_DURATION_MILLIS = 5000; + + private EmbeddedConnectCluster connect; @Before public void setup() throws IOException { + connect = new EmbeddedConnectCluster(ConnectIntegrationTest.class); connect.start(); } @@ -58,14 +62,14 @@ public void testProduceConsumeConnector() throws Exception { connect.kafka().createTopic("test-topic"); // produce some strings into test topic - for (int i = 0; i < 1000; i++) { + for (int i = 0; i < NUM_RECORDS_PRODUCED / 2; i++) { connect.kafka().produce("test-topic", "hello-" + i); connect.kafka().produce("test-topic", "world-" + i); } // consume all records from test topic or fail log.info("Consuming records from test topic"); - connect.kafka().consumeNRecords(2000, 5000, "test-topic"); + connect.kafka().consume(NUM_RECORDS_PRODUCED, CONSUME_MAX_DURATION_MILLIS, "test-topic"); log.info("Connect endpoint: {}", connect.restUrl()); @@ -83,7 +87,8 @@ public void testProduceConsumeConnector() throws Exception { Thread.sleep(500); } - Assert.assertEquals(2000, MonitorableSinkConnector.COUNTER.get()); + // all records must be consumed + Assert.assertEquals(NUM_RECORDS_PRODUCED, MonitorableSinkConnector.COUNTER.get()); log.info("Connector read {} records from topic", MonitorableSinkConnector.COUNTER.get()); connect.deleteConnector("simple-conn"); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/DeadLetterQueueIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/DeadLetterQueueIntegrationTest.java index b70a10b60b149..a3a071245cbad 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/DeadLetterQueueIntegrationTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/DeadLetterQueueIntegrationTest.java @@ -42,11 +42,16 @@ public class DeadLetterQueueIntegrationTest { private static final Logger log = LoggerFactory.getLogger(DeadLetterQueueIntegrationTest.class); private static final String DLQ_TOPIC = "my-connector-errors"; + private static final int NUM_RECORDS_PRODUCED = 20; + private static final int EXPECTED_CORRECT_RECORDS = 18; + private static final int EXPECTED_INCORRECT_RECORDS = 2; + private static final int CONSUME_MAX_DURATION_MILLIS = 5000; - private final EmbeddedConnectCluster connect = new EmbeddedConnectCluster(ConnectIntegrationTest.class); + private EmbeddedConnectCluster connect; @Before public void setup() throws IOException { + connect = new EmbeddedConnectCluster(ConnectIntegrationTest.class); connect.start(); } @@ -61,47 +66,47 @@ public void testTransformationErrorHandling() throws Exception { connect.kafka().createTopic("test-topic"); // produce some strings into test topic - for (int i = 0; i < 10; i++) { + for (int i = 0; i < NUM_RECORDS_PRODUCED / 2; i++) { connect.kafka().produce("test-topic", "hello-" + i, String.valueOf(i)); connect.kafka().produce("test-topic", "world-" + i, String.valueOf(i)); } // consume all records from test topic log.info("Consuming records from test topic"); - for (ConsumerRecord recs : connect.kafka().consumeNRecords(20, 5000, "test-topic")) { + for (ConsumerRecord recs : connect.kafka().consume(NUM_RECORDS_PRODUCED, CONSUME_MAX_DURATION_MILLIS, "test-topic")) { log.debug("Consumed record ({}, {}) from topic {}", new String(recs.key()), new String(recs.value()), recs.topic()); } - Map confs = new HashMap<>(); - confs.put("connector.class", "MonitorableSink"); - confs.put("task.max", "2"); - confs.put("topics", "test-topic"); - confs.put("key.converter", StringConverter.class.getName()); - confs.put("value.converter", StringConverter.class.getName()); + Map props = new HashMap<>(); + props.put("connector.class", "MonitorableSink"); + props.put("task.max", "2"); + props.put("topics", "test-topic"); + props.put("key.converter", StringConverter.class.getName()); + props.put("value.converter", StringConverter.class.getName()); - confs.put("errors.log.enable", "true"); - confs.put("errors.log.include.messages", "false"); + props.put("errors.log.enable", "true"); + props.put("errors.log.include.messages", "false"); - confs.put("errors.deadletterqueue.topic.name", DLQ_TOPIC); - confs.put("errors.deadletterqueue.topic.replication.factor", "1"); - confs.put("errors.tolerance", "all"); - confs.put("transforms", "failing_transform"); - confs.put("transforms.failing_transform.type", FaultyPassthrough.class.getName()); + props.put("errors.deadletterqueue.topic.name", DLQ_TOPIC); + props.put("errors.deadletterqueue.topic.replication.factor", "1"); + props.put("errors.tolerance", "all"); + props.put("transforms", "failing_transform"); + props.put("transforms.failing_transform.type", FaultyPassthrough.class.getName()); - connect.startConnector("simple-conn", confs); + connect.startConnector("simple-conn", props); int attempts = 0; - while (attempts++ < 5 && MonitorableSinkConnector.COUNTER.get() < 18) { + while (attempts++ < 5 && MonitorableSinkConnector.COUNTER.get() < EXPECTED_CORRECT_RECORDS) { log.info("Received only {} records. Waiting .. ", MonitorableSinkConnector.COUNTER.get()); Thread.sleep(500); } - Assert.assertEquals(18, MonitorableSinkConnector.COUNTER.get()); + Assert.assertEquals(EXPECTED_CORRECT_RECORDS, MonitorableSinkConnector.COUNTER.get()); // consume failed records from dead letter queue topic log.info("Consuming records from test topic"); - for (ConsumerRecord recs : connect.kafka().consumeNRecords(2, 5000, DLQ_TOPIC)) { - log.info("Consumed record ({}, {}) from dead letter queue topic {}", new String(recs.key()), new String(recs.value()), DLQ_TOPIC); + for (ConsumerRecord recs : connect.kafka().consume(EXPECTED_INCORRECT_RECORDS, CONSUME_MAX_DURATION_MILLIS, DLQ_TOPIC)) { + log.info("Consumed record (key={}, value={}) from dead letter queue topic {}", new String(recs.key()), new String(recs.value()), DLQ_TOPIC); } connect.deleteConnector("simple-conn"); @@ -110,18 +115,17 @@ public void testTransformationErrorHandling() throws Exception { public static class FaultyPassthrough> implements Transformation { - private static final Logger log = LoggerFactory.getLogger(FaultyPassthrough.class); - - public static final ConfigDef CONFIG_DEF = new ConfigDef(); - - private int invocations = 0; + static final ConfigDef CONFIG_DEF = new ConfigDef(); + /** + * an arbitrary id which causes this transformation to fail with a {@link RetriableException}. + */ + private static final int BAD_RECORD_VAL = 7; @Override public R apply(R record) { long val = Long.parseLong(String.valueOf(record.value())); - if (val == 7) { - log.debug("Failing record: {} at invocations={}", record, invocations); - throw new RetriableException("Bad invocations " + invocations + " for val = " + val); + if (val == BAD_RECORD_VAL) { + throw new RetriableException("Error when when val=" + val); } return record; } @@ -133,13 +137,10 @@ public ConfigDef config() { @Override public void close() { - log.info("Shutting down transform"); } @Override public void configure(Map configs) { - log.info("Configuring {}.", this.getClass()); } } - } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/MonitorableSinkConnector.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/MonitorableSinkConnector.java index fa4dd791418d1..6b5f8678c4f7b 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/MonitorableSinkConnector.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/MonitorableSinkConnector.java @@ -29,6 +29,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicInteger; public class MonitorableSinkConnector extends TestSinkConnector { @@ -45,7 +46,7 @@ public void start(Map props) { @Override public Class taskClass() { - return SimpleTestSinkTask.class; + return MonitorableSinkTask.class; } @Override @@ -69,11 +70,11 @@ public ConfigDef config() { return new ConfigDef(); } - public static class SimpleTestSinkTask extends SinkTask { + public static class MonitorableSinkTask extends SinkTask { @Override public String version() { - return null; + return "unknown"; } @Override 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 249be08468382..4e2ec2f59260f 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 @@ -17,9 +17,12 @@ package org.apache.kafka.connect.util.clusters; import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.connect.cli.ConnectDistributed; import org.apache.kafka.connect.runtime.Connect; import org.apache.kafka.connect.runtime.Herder; import org.apache.kafka.connect.runtime.Worker; +import org.apache.kafka.connect.runtime.WorkerConfig; import org.apache.kafka.connect.runtime.WorkerConfigTransformer; import org.apache.kafka.connect.runtime.distributed.DistributedConfig; import org.apache.kafka.connect.runtime.distributed.DistributedHerder; @@ -36,6 +39,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.servlet.http.HttpServletResponse; import java.io.IOException; import java.io.InputStream; import java.io.OutputStreamWriter; @@ -50,10 +54,11 @@ public class EmbeddedConnectCluster extends ExternalResource { private static final Logger log = LoggerFactory.getLogger(EmbeddedConnectCluster.class); - private static final int NUM_BROKERS = 1; - private static final Properties BROKER_CONFIG = new Properties(); + private static final int DEFAULT_NUM_BROKERS = 1; + private static final Properties DEFAULT_BROKER_CONFIG = new Properties(); + private static final String REST_HOST_NAME= "localhost"; - private final EmbeddedKafkaCluster kafkaCluster = new EmbeddedKafkaCluster(NUM_BROKERS, BROKER_CONFIG); + private final EmbeddedKafkaCluster kafkaCluster; private final Map workerProps; private final String clusterName; @@ -71,8 +76,13 @@ public EmbeddedConnectCluster(String name) { } public EmbeddedConnectCluster(String name, Map workerProps) { + this(name, workerProps, DEFAULT_NUM_BROKERS, DEFAULT_BROKER_CONFIG); + } + + public EmbeddedConnectCluster(String name, Map workerProps, int numBrokers, Properties brokerProps) { this.workerProps = workerProps; this.clusterName = name; + kafkaCluster = new EmbeddedKafkaCluster(numBrokers, brokerProps); } public void start() throws IOException { @@ -90,7 +100,7 @@ public void stop() { protected void before() throws IOException { kafkaCluster.before(); startConnect(); - log.info("Started connect at {} with kafka cluster at {}", restUrl(), kafka().bootstrapServers()); + log.info("Started Connect at {} with Kafka cluster at {}", restUrl(), kafka().bootstrapServers()); } @Override @@ -109,62 +119,31 @@ protected void after() { } public void startConnect() { - log.info("Starting standalone connect cluster.."); - workerProps.put("bootstrap.servers", kafka().bootstrapServers()); - workerProps.put("rest.host.name", "localhost"); - - putIfAbsent(workerProps, "group.id", "connect-integration-test-" + clusterName); - putIfAbsent(workerProps, "offset.storage.topic", "connect-offset-topic-" + clusterName); - putIfAbsent(workerProps, "offset.storage.replication.factor", "1"); - putIfAbsent(workerProps, "config.storage.topic", "connect-config-topic-" + clusterName); - putIfAbsent(workerProps, "config.storage.replication.factor", "1"); - putIfAbsent(workerProps, "status.storage.topic", "connect-storage-topic-" + clusterName); - putIfAbsent(workerProps, "status.storage.replication.factor", "1"); - putIfAbsent(workerProps, "key.converter", "org.apache.kafka.connect.json.JsonConverter"); - putIfAbsent(workerProps, "value.converter", "org.apache.kafka.connect.json.JsonConverter"); - putIfAbsent(workerProps, "key.converter.schemas.enable", "false"); - putIfAbsent(workerProps, "value.converter.schemas.enable", "false"); - - log.info("Scanning for plugins..."); - Plugins plugins = new Plugins(workerProps); - plugins.compareAndSwapWithDelegatingLoader(); - - DistributedConfig config = new DistributedConfig(workerProps); - - RestServer rest = new RestServer(config); - advertisedUrl = rest.advertisedUrl(); - String workerId = advertisedUrl.getHost() + ":" + advertisedUrl.getPort(); - - KafkaOffsetBackingStore offsetBackingStore = new KafkaOffsetBackingStore(); - offsetBackingStore.configure(config); - - Worker worker = new Worker(workerId, kafkaCluster.time(), plugins, config, offsetBackingStore); - - WorkerConfigTransformer configTransformer = worker.configTransformer(); - Converter internalValueConverter = worker.getInternalValueConverter(); - - StatusBackingStore statusBackingStore = new KafkaStatusBackingStore(kafkaCluster.time(), internalValueConverter); - statusBackingStore.configure(config); - - ConfigBackingStore configBackingStore = new KafkaConfigBackingStore( - internalValueConverter, - config, - configTransformer); - - Herder herder = new DistributedHerder(config, kafkaCluster.time(), worker, - ConnectUtils.lookupKafkaClusterId(config), statusBackingStore, configBackingStore, - advertisedUrl.toString()); - connect = new Connect(herder, rest); - connect.start(); + log.info("Starting Connect cluster with one worker."); + + workerProps.put(WorkerConfig.BOOTSTRAP_SERVERS_CONFIG, kafka().bootstrapServers()); + workerProps.put(WorkerConfig.REST_HOST_NAME_CONFIG, REST_HOST_NAME); + + putIfAbsent(workerProps, ConsumerConfig.GROUP_ID_CONFIG, "connect-integration-test-" + clusterName); + putIfAbsent(workerProps, DistributedConfig.OFFSET_STORAGE_TOPIC_CONFIG, "connect-offset-topic-" + clusterName); + putIfAbsent(workerProps, DistributedConfig.OFFSET_STORAGE_REPLICATION_FACTOR_CONFIG, "1"); + putIfAbsent(workerProps, DistributedConfig.CONFIG_TOPIC_CONFIG, "connect-config-topic-" + clusterName); + putIfAbsent(workerProps, DistributedConfig.CONFIG_STORAGE_REPLICATION_FACTOR_CONFIG, "1"); + putIfAbsent(workerProps, DistributedConfig.STATUS_STORAGE_TOPIC_CONFIG, "connect-storage-topic-" + clusterName); + putIfAbsent(workerProps, DistributedConfig.STATUS_STORAGE_REPLICATION_FACTOR_CONFIG, "1"); + putIfAbsent(workerProps, DistributedConfig.KEY_CONVERTER_CLASS_CONFIG, "org.apache.kafka.connect.storage.StringConverter"); + putIfAbsent(workerProps, DistributedConfig.VALUE_CONVERTER_CLASS_CONFIG, "org.apache.kafka.connect.storage.StringConverter"); + + connect = new ConnectDistributed().startConnect(workerProps); } public void startConnector(String connName, Map connConfig) throws IOException { - String url = String.format("http://localhost:8083/connectors/%s/config", connName); + String url = String.format("http://%s:%s/connectors/%s/config", REST_HOST_NAME, WorkerConfig.REST_PORT_DEFAULT, connName); ObjectMapper mapper = new ObjectMapper(); try { String content = mapper.writeValueAsString(connConfig); int status = executePut(url, content); - if (status >= 400) { + if (status >= HttpServletResponse.SC_BAD_REQUEST) { throw new IOException("Could not execute PUT request. status=" + status); } } catch (IOException e) { @@ -174,13 +153,13 @@ public void startConnector(String connName, Map connConfig) thro } public void deleteConnector(String connName) throws IOException { - int status = executeDelete(String.format("http://localhost:8083/connectors/%s", connName)); - if (status >= 400) { + int status = executeDelete(String.format("http://%s:%s/connectors/%s", REST_HOST_NAME, WorkerConfig.REST_PORT_DEFAULT, connName)); + if (status >= HttpServletResponse.SC_BAD_REQUEST) { throw new IOException("Could not execute DELETE request. status=" + status); } } - private void putIfAbsent(Map props, String propertyKey, String propertyValue) { + private static void putIfAbsent(Map props, String propertyKey, String propertyValue) { if (!props.containsKey(propertyKey)) { props.put(propertyKey, propertyValue); } @@ -200,9 +179,9 @@ public int executePut(String url, String body) throws IOException { httpCon.setDoOutput(true); httpCon.setRequestProperty("Content-Type", "application/json"); httpCon.setRequestMethod("PUT"); - OutputStreamWriter out = new OutputStreamWriter(httpCon.getOutputStream()); - out.write(body); - out.close(); + try (OutputStreamWriter out = new OutputStreamWriter(httpCon.getOutputStream())) { + out.write(body); + } try (InputStream is = httpCon.getInputStream()) { int c; StringBuilder response = new StringBuilder(); @@ -223,4 +202,4 @@ public int executeDelete(String url) throws IOException { return httpCon.getResponseCode(); } -} \ No newline at end of file +} diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java b/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java index b6f9daedaa436..a5dbea9435eaa 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java @@ -57,6 +57,10 @@ import java.util.UUID; import java.util.concurrent.ExecutionException; +/** + * Setup an embedded Kafka cluster with specified number of brokers and specified broker properties. To be used for + * integration tests. + */ public class EmbeddedKafkaCluster extends ExternalResource { private static final Logger log = LoggerFactory.getLogger(EmbeddedKafkaCluster.class); @@ -94,7 +98,7 @@ public void start() throws IOException { zookeeper = new EmbeddedZookeeper(); brokerConfig.put(KafkaConfig$.MODULE$.ZkConnectProp(), zKConnectString()); - brokerConfig.put(KafkaConfig$.MODULE$.PortProp(), 9999); // pick a random port + brokerConfig.put(KafkaConfig$.MODULE$.PortProp(), 0); // pick a random port putIfAbsent(brokerConfig, KafkaConfig$.MODULE$.HostNameProp(), "localhost"); putIfAbsent(brokerConfig, KafkaConfig$.MODULE$.DeleteTopicEnableProp(), true); @@ -152,7 +156,7 @@ private void stop() { } } - private void putIfAbsent(final Properties props, final String propertyKey, final Object propertyValue) { + private static void putIfAbsent(final Properties props, final String propertyKey, final Object propertyValue) { if (!props.containsKey(propertyKey)) { props.put(propertyKey, propertyValue); } @@ -235,7 +239,7 @@ public AdminClient createAdminClient() { * @param topics the topics to subscribe and consume records from. * @return a {@link ConsumerRecords} collection containing at least n records. */ - public ConsumerRecords consumeNRecords(int n, long maxDuration, String... topics) { + public ConsumerRecords consume(int n, long maxDuration, String... topics) { long num = 1; long duration = maxDuration; if (duration > 250) { From aa22a705a1c455ee05189b243e7296148a8de612 Mon Sep 17 00:00:00 2001 From: Arjun Satish Date: Tue, 18 Sep 2018 00:02:28 -0700 Subject: [PATCH 14/52] Reset log4j.properties to original state Signed-off-by: Arjun Satish --- connect/runtime/src/test/resources/log4j.properties | 12 +----------- 1 file changed, 1 insertion(+), 11 deletions(-) diff --git a/connect/runtime/src/test/resources/log4j.properties b/connect/runtime/src/test/resources/log4j.properties index 7d54ae4ab9567..d5e90fe788f76 100644 --- a/connect/runtime/src/test/resources/log4j.properties +++ b/connect/runtime/src/test/resources/log4j.properties @@ -14,20 +14,10 @@ # See the License for the specific language governing permissions and # limitations under the License. ## -# Root logger option -log4j.rootLogger=INFO, stdout +log4j.rootLogger=OFF, stdout -# Redirect log messages to console log4j.appender.stdout=org.apache.log4j.ConsoleAppender -log4j.appender.stdout.Target=System.out log4j.appender.stdout.layout=org.apache.log4j.PatternLayout log4j.appender.stdout.layout.ConversionPattern=[%d] %p %m (%c:%L)%n -log4j.logger.org.reflections=ERROR log4j.logger.org.apache.kafka=ERROR -log4j.logger.org.apache.kafka.connect=INFO -log4j.logger.org.eclipse.jetty=ERROR -log4j.logger.org.apache.zookeeper=ERROR -log4j.logger.kafka=ERROR -log4j.logger.org.apache.kafka.connect.util.MonitorableSinkConnector=DEBUG - From c6db65df6fcaae9f2fa70f8e55895a24f8aaaffe Mon Sep 17 00:00:00 2001 From: Arjun Satish Date: Tue, 18 Sep 2018 00:19:15 -0700 Subject: [PATCH 15/52] Clean up imports Signed-off-by: Arjun Satish --- .../integration/ConnectIntegrationTest.java | 2 - .../util/clusters/EmbeddedConnectCluster.java | 56 ++++++++----------- .../util/clusters/EmbeddedKafkaCluster.java | 25 +++++---- 3 files changed, 37 insertions(+), 46 deletions(-) diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTest.java index f69325916aa96..e4c00330dbef1 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTest.java @@ -71,8 +71,6 @@ public void testProduceConsumeConnector() throws Exception { log.info("Consuming records from test topic"); connect.kafka().consume(NUM_RECORDS_PRODUCED, CONSUME_MAX_DURATION_MILLIS, "test-topic"); - log.info("Connect endpoint: {}", connect.restUrl()); - Map confs = new HashMap<>(); confs.put("connector.class", "MonitorableSink"); confs.put("task.max", "2"); 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 4e2ec2f59260f..4df5d3131759b 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 @@ -17,24 +17,9 @@ package org.apache.kafka.connect.util.clusters; import com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.connect.cli.ConnectDistributed; import org.apache.kafka.connect.runtime.Connect; -import org.apache.kafka.connect.runtime.Herder; -import org.apache.kafka.connect.runtime.Worker; import org.apache.kafka.connect.runtime.WorkerConfig; -import org.apache.kafka.connect.runtime.WorkerConfigTransformer; -import org.apache.kafka.connect.runtime.distributed.DistributedConfig; -import org.apache.kafka.connect.runtime.distributed.DistributedHerder; -import org.apache.kafka.connect.runtime.isolation.Plugins; -import org.apache.kafka.connect.runtime.rest.RestServer; -import org.apache.kafka.connect.storage.ConfigBackingStore; -import org.apache.kafka.connect.storage.Converter; -import org.apache.kafka.connect.storage.KafkaConfigBackingStore; -import org.apache.kafka.connect.storage.KafkaOffsetBackingStore; -import org.apache.kafka.connect.storage.KafkaStatusBackingStore; -import org.apache.kafka.connect.storage.StatusBackingStore; -import org.apache.kafka.connect.util.ConnectUtils; import org.junit.rules.ExternalResource; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -44,12 +29,23 @@ import java.io.InputStream; import java.io.OutputStreamWriter; import java.net.HttpURLConnection; -import java.net.URI; import java.net.URL; import java.util.HashMap; import java.util.Map; import java.util.Properties; +import static org.apache.kafka.clients.consumer.ConsumerConfig.GROUP_ID_CONFIG; +import static org.apache.kafka.connect.runtime.ConnectorConfig.KEY_CONVERTER_CLASS_CONFIG; +import static org.apache.kafka.connect.runtime.ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG; +import static org.apache.kafka.connect.runtime.WorkerConfig.BOOTSTRAP_SERVERS_CONFIG; +import static org.apache.kafka.connect.runtime.WorkerConfig.REST_HOST_NAME_CONFIG; +import static org.apache.kafka.connect.runtime.distributed.DistributedConfig.CONFIG_STORAGE_REPLICATION_FACTOR_CONFIG; +import static org.apache.kafka.connect.runtime.distributed.DistributedConfig.CONFIG_TOPIC_CONFIG; +import static org.apache.kafka.connect.runtime.distributed.DistributedConfig.OFFSET_STORAGE_REPLICATION_FACTOR_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_REPLICATION_FACTOR_CONFIG; +import static org.apache.kafka.connect.runtime.distributed.DistributedConfig.STATUS_STORAGE_TOPIC_CONFIG; + public class EmbeddedConnectCluster extends ExternalResource { private static final Logger log = LoggerFactory.getLogger(EmbeddedConnectCluster.class); @@ -64,7 +60,6 @@ public class EmbeddedConnectCluster extends ExternalResource { private final String clusterName; private Connect connect; - private URI advertisedUrl; public EmbeddedConnectCluster(Class klass) { this(klass.getName()); @@ -100,7 +95,6 @@ public void stop() { protected void before() throws IOException { kafkaCluster.before(); startConnect(); - log.info("Started Connect at {} with Kafka cluster at {}", restUrl(), kafka().bootstrapServers()); } @Override @@ -121,18 +115,18 @@ protected void after() { public void startConnect() { log.info("Starting Connect cluster with one worker."); - workerProps.put(WorkerConfig.BOOTSTRAP_SERVERS_CONFIG, kafka().bootstrapServers()); - workerProps.put(WorkerConfig.REST_HOST_NAME_CONFIG, REST_HOST_NAME); + workerProps.put(BOOTSTRAP_SERVERS_CONFIG, kafka().bootstrapServers()); + workerProps.put(REST_HOST_NAME_CONFIG, REST_HOST_NAME); - putIfAbsent(workerProps, ConsumerConfig.GROUP_ID_CONFIG, "connect-integration-test-" + clusterName); - putIfAbsent(workerProps, DistributedConfig.OFFSET_STORAGE_TOPIC_CONFIG, "connect-offset-topic-" + clusterName); - putIfAbsent(workerProps, DistributedConfig.OFFSET_STORAGE_REPLICATION_FACTOR_CONFIG, "1"); - putIfAbsent(workerProps, DistributedConfig.CONFIG_TOPIC_CONFIG, "connect-config-topic-" + clusterName); - putIfAbsent(workerProps, DistributedConfig.CONFIG_STORAGE_REPLICATION_FACTOR_CONFIG, "1"); - putIfAbsent(workerProps, DistributedConfig.STATUS_STORAGE_TOPIC_CONFIG, "connect-storage-topic-" + clusterName); - putIfAbsent(workerProps, DistributedConfig.STATUS_STORAGE_REPLICATION_FACTOR_CONFIG, "1"); - putIfAbsent(workerProps, DistributedConfig.KEY_CONVERTER_CLASS_CONFIG, "org.apache.kafka.connect.storage.StringConverter"); - putIfAbsent(workerProps, DistributedConfig.VALUE_CONVERTER_CLASS_CONFIG, "org.apache.kafka.connect.storage.StringConverter"); + putIfAbsent(workerProps, GROUP_ID_CONFIG, "connect-integration-test-" + clusterName); + putIfAbsent(workerProps, OFFSET_STORAGE_TOPIC_CONFIG, "connect-offset-topic-" + clusterName); + putIfAbsent(workerProps, OFFSET_STORAGE_REPLICATION_FACTOR_CONFIG, "1"); + putIfAbsent(workerProps, CONFIG_TOPIC_CONFIG, "connect-config-topic-" + clusterName); + putIfAbsent(workerProps, CONFIG_STORAGE_REPLICATION_FACTOR_CONFIG, "1"); + putIfAbsent(workerProps, STATUS_STORAGE_TOPIC_CONFIG, "connect-storage-topic-" + clusterName); + putIfAbsent(workerProps, STATUS_STORAGE_REPLICATION_FACTOR_CONFIG, "1"); + putIfAbsent(workerProps, KEY_CONVERTER_CLASS_CONFIG, "org.apache.kafka.connect.storage.StringConverter"); + putIfAbsent(workerProps, VALUE_CONVERTER_CLASS_CONFIG, "org.apache.kafka.connect.storage.StringConverter"); connect = new ConnectDistributed().startConnect(workerProps); } @@ -165,10 +159,6 @@ private static void putIfAbsent(Map props, String propertyKey, S } } - public URI restUrl() { - return advertisedUrl; - } - public EmbeddedKafkaCluster kafka() { return kafkaCluster; } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java b/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java index a5dbea9435eaa..ed83e934935b2 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java @@ -26,7 +26,6 @@ import org.apache.kafka.clients.admin.AdminClient; import org.apache.kafka.clients.admin.AdminClientConfig; import org.apache.kafka.clients.admin.NewTopic; -import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.clients.consumer.KafkaConsumer; @@ -57,6 +56,13 @@ import java.util.UUID; import java.util.concurrent.ExecutionException; +import static org.apache.kafka.clients.consumer.ConsumerConfig.AUTO_OFFSET_RESET_CONFIG; +import static org.apache.kafka.clients.consumer.ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG; +import static org.apache.kafka.clients.consumer.ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG; +import static org.apache.kafka.clients.consumer.ConsumerConfig.GROUP_ID_CONFIG; +import static org.apache.kafka.clients.consumer.ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG; +import static org.apache.kafka.clients.consumer.ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG; + /** * Setup an embedded Kafka cluster with specified number of brokers and specified broker properties. To be used for * integration tests. @@ -90,10 +96,6 @@ protected void after() { stop(); } - Time time() { - return time; - } - public void start() throws IOException { zookeeper = new EmbeddedZookeeper(); @@ -242,6 +244,7 @@ public AdminClient createAdminClient() { public ConsumerRecords consume(int n, long maxDuration, String... topics) { long num = 1; long duration = maxDuration; + // poll in intervals of 250millis. if (duration > 250) { num = 1 + (maxDuration / 250); duration = 250; @@ -274,12 +277,12 @@ public KafkaConsumer createConsumer(String... topics) { // and through to the task Map props = new HashMap<>(); - props.put(ConsumerConfig.GROUP_ID_CONFIG, UUID.randomUUID().toString()); - props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers()); - props.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false"); - props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); - props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArrayDeserializer"); - props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArrayDeserializer"); + props.put(GROUP_ID_CONFIG, UUID.randomUUID().toString()); + props.put(BOOTSTRAP_SERVERS_CONFIG, bootstrapServers()); + props.put(ENABLE_AUTO_COMMIT_CONFIG, "false"); + props.put(AUTO_OFFSET_RESET_CONFIG, "earliest"); + props.put(KEY_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArrayDeserializer"); + props.put(VALUE_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArrayDeserializer"); KafkaConsumer consumer; try { From 50b916667c5f51c3db557aeaa8f1e9051f23b1e2 Mon Sep 17 00:00:00 2001 From: Arjun Satish Date: Wed, 19 Sep 2018 18:16:28 -0700 Subject: [PATCH 16/52] Align variable name to convention Signed-off-by: Arjun Satish --- checkstyle/import-control.xml | 1 + .../kafka/connect/integration/ConnectIntegrationTest.java | 4 ++-- .../connect/integration/DeadLetterQueueIntegrationTest.java | 6 +++--- .../kafka/connect/integration/MonitorableSinkConnector.java | 1 - .../kafka/connect/util/clusters/EmbeddedConnectCluster.java | 2 +- 5 files changed, 7 insertions(+), 7 deletions(-) diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index bd48dedff86f4..8c98f8d447865 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -369,6 +369,7 @@ + diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTest.java index e4c00330dbef1..597a622e9ab25 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTest.java @@ -37,7 +37,7 @@ public class ConnectIntegrationTest { private static final Logger log = LoggerFactory.getLogger(ConnectIntegrationTest.class); private static final int NUM_RECORDS_PRODUCED = 2000; - private static final int CONSUME_MAX_DURATION_MILLIS = 5000; + private static final int CONSUME_MAX_DURATION_MS = 5000; private EmbeddedConnectCluster connect; @@ -69,7 +69,7 @@ public void testProduceConsumeConnector() throws Exception { // consume all records from test topic or fail log.info("Consuming records from test topic"); - connect.kafka().consume(NUM_RECORDS_PRODUCED, CONSUME_MAX_DURATION_MILLIS, "test-topic"); + connect.kafka().consume(NUM_RECORDS_PRODUCED, CONSUME_MAX_DURATION_MS, "test-topic"); Map confs = new HashMap<>(); confs.put("connector.class", "MonitorableSink"); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/DeadLetterQueueIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/DeadLetterQueueIntegrationTest.java index a3a071245cbad..9e762c8bd9677 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/DeadLetterQueueIntegrationTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/DeadLetterQueueIntegrationTest.java @@ -45,7 +45,7 @@ public class DeadLetterQueueIntegrationTest { private static final int NUM_RECORDS_PRODUCED = 20; private static final int EXPECTED_CORRECT_RECORDS = 18; private static final int EXPECTED_INCORRECT_RECORDS = 2; - private static final int CONSUME_MAX_DURATION_MILLIS = 5000; + private static final int CONSUME_MAX_DURATION_MS = 5000; private EmbeddedConnectCluster connect; @@ -73,7 +73,7 @@ public void testTransformationErrorHandling() throws Exception { // consume all records from test topic log.info("Consuming records from test topic"); - for (ConsumerRecord recs : connect.kafka().consume(NUM_RECORDS_PRODUCED, CONSUME_MAX_DURATION_MILLIS, "test-topic")) { + for (ConsumerRecord recs : connect.kafka().consume(NUM_RECORDS_PRODUCED, CONSUME_MAX_DURATION_MS, "test-topic")) { log.debug("Consumed record ({}, {}) from topic {}", new String(recs.key()), new String(recs.value()), recs.topic()); } @@ -105,7 +105,7 @@ public void testTransformationErrorHandling() throws Exception { // consume failed records from dead letter queue topic log.info("Consuming records from test topic"); - for (ConsumerRecord recs : connect.kafka().consume(EXPECTED_INCORRECT_RECORDS, CONSUME_MAX_DURATION_MILLIS, DLQ_TOPIC)) { + for (ConsumerRecord recs : connect.kafka().consume(EXPECTED_INCORRECT_RECORDS, CONSUME_MAX_DURATION_MS, DLQ_TOPIC)) { log.info("Consumed record (key={}, value={}) from dead letter queue topic {}", new String(recs.key()), new String(recs.value()), DLQ_TOPIC); } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/MonitorableSinkConnector.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/MonitorableSinkConnector.java index 6b5f8678c4f7b..e843b7775b5b5 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/MonitorableSinkConnector.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/MonitorableSinkConnector.java @@ -29,7 +29,6 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicInteger; public class MonitorableSinkConnector extends TestSinkConnector { 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 4df5d3131759b..dfde009aefe63 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 @@ -52,7 +52,7 @@ public class EmbeddedConnectCluster extends ExternalResource { private static final int DEFAULT_NUM_BROKERS = 1; private static final Properties DEFAULT_BROKER_CONFIG = new Properties(); - private static final String REST_HOST_NAME= "localhost"; + private static final String REST_HOST_NAME = "localhost"; private final EmbeddedKafkaCluster kafkaCluster; From 708fd2dba87dd62b6f1ffc48418869eaea46d9d4 Mon Sep 17 00:00:00 2001 From: Arjun Satish Date: Wed, 19 Sep 2018 19:22:56 -0700 Subject: [PATCH 17/52] Use a latch to wait for connector to sink records Signed-off-by: Arjun Satish --- .../integration/ConnectIntegrationTest.java | 37 +++++++++-------- .../DeadLetterQueueIntegrationTest.java | 18 +++++---- .../integration/MonitorableSinkConnector.java | 40 ++++++++++++++++--- 3 files changed, 66 insertions(+), 29 deletions(-) diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTest.java index 597a622e9ab25..f3ff3c6a615a2 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTest.java @@ -16,11 +16,11 @@ */ package org.apache.kafka.connect.integration; +import org.apache.kafka.connect.errors.ConnectException; 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.Assert; import org.junit.Before; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -71,24 +71,29 @@ public void testProduceConsumeConnector() throws Exception { log.info("Consuming records from test topic"); connect.kafka().consume(NUM_RECORDS_PRODUCED, CONSUME_MAX_DURATION_MS, "test-topic"); - Map confs = new HashMap<>(); - confs.put("connector.class", "MonitorableSink"); - confs.put("task.max", "2"); - confs.put("topics", "test-topic"); - confs.put("key.converter", StringConverter.class.getName()); - confs.put("value.converter", StringConverter.class.getName()); - - connect.startConnector("simple-conn", confs); - - int attempts = 0; - while (attempts++ < 5 && MonitorableSinkConnector.COUNTER.get() < 2000) { - Thread.sleep(500); + Map props = new HashMap<>(); + props.put("connector.class", "MonitorableSink"); + props.put("task.max", "2"); + props.put("topics", "test-topic"); + props.put("key.converter", StringConverter.class.getName()); + props.put("value.converter", StringConverter.class.getName()); + props.put(MonitorableSinkConnector.EXPECTED_RECORDS, String.valueOf(NUM_RECORDS_PRODUCED)); + + connect.startConnector("simple-conn", props); + + MonitorableSinkConnector.MonitorableSinkTask task = MonitorableSinkConnector.TASKS.get("simple-conn-0"); + for (int i = 0; i < 10 && task == null; i++) { + log.debug("Sleeping for 200 before looking up task"); + Thread.sleep(200); + task = MonitorableSinkConnector.TASKS.get("simple-conn-0"); + } + if (task == null) { + throw new ConnectException("Connector took too long to start"); } - // all records must be consumed - Assert.assertEquals(NUM_RECORDS_PRODUCED, MonitorableSinkConnector.COUNTER.get()); + task.awaitRecords(CONSUME_MAX_DURATION_MS); - log.info("Connector read {} records from topic", MonitorableSinkConnector.COUNTER.get()); + log.info("Connector read {} records from topic", NUM_RECORDS_PRODUCED); connect.deleteConnector("simple-conn"); } } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/DeadLetterQueueIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/DeadLetterQueueIntegrationTest.java index 9e762c8bd9677..3ff3560245ce5 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/DeadLetterQueueIntegrationTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/DeadLetterQueueIntegrationTest.java @@ -19,13 +19,13 @@ import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.connect.connector.ConnectRecord; +import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.errors.RetriableException; import org.apache.kafka.connect.storage.StringConverter; import org.apache.kafka.connect.transforms.Transformation; import org.apache.kafka.connect.util.clusters.EmbeddedConnectCluster; import org.apache.kafka.test.IntegrationTest; import org.junit.After; -import org.junit.Assert; import org.junit.Before; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -83,6 +83,7 @@ public void testTransformationErrorHandling() throws Exception { props.put("topics", "test-topic"); props.put("key.converter", StringConverter.class.getName()); props.put("value.converter", StringConverter.class.getName()); + props.put(MonitorableSinkConnector.EXPECTED_RECORDS, String.valueOf(EXPECTED_CORRECT_RECORDS)); props.put("errors.log.enable", "true"); props.put("errors.log.include.messages", "false"); @@ -95,13 +96,16 @@ public void testTransformationErrorHandling() throws Exception { connect.startConnector("simple-conn", props); - int attempts = 0; - while (attempts++ < 5 && MonitorableSinkConnector.COUNTER.get() < EXPECTED_CORRECT_RECORDS) { - log.info("Received only {} records. Waiting .. ", MonitorableSinkConnector.COUNTER.get()); - Thread.sleep(500); + MonitorableSinkConnector.MonitorableSinkTask task = MonitorableSinkConnector.TASKS.get("simple-conn-0"); + for (int i = 0; i < 10 && task == null; i++) { + log.debug("Sleeping for 200 before looking up task"); + Thread.sleep(200); + task = MonitorableSinkConnector.TASKS.get("simple-conn-0"); } - - Assert.assertEquals(EXPECTED_CORRECT_RECORDS, MonitorableSinkConnector.COUNTER.get()); + if (task == null) { + throw new ConnectException("Connector took too long to start"); + } + task.awaitRecords(CONSUME_MAX_DURATION_MS); // consume failed records from dead letter queue topic log.info("Consuming records from test topic"); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/MonitorableSinkConnector.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/MonitorableSinkConnector.java index e843b7775b5b5..02b5a6bcaceb7 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/MonitorableSinkConnector.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/MonitorableSinkConnector.java @@ -18,6 +18,7 @@ import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.connect.connector.Task; +import org.apache.kafka.connect.errors.DataException; import org.apache.kafka.connect.runtime.TestSinkConnector; import org.apache.kafka.connect.sink.SinkRecord; import org.apache.kafka.connect.sink.SinkTask; @@ -29,18 +30,25 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; public class MonitorableSinkConnector extends TestSinkConnector { + public static final String EXPECTED_RECORDS = "expected_records"; private static final Logger log = LoggerFactory.getLogger(MonitorableSinkConnector.class); - public static final AtomicInteger COUNTER = new AtomicInteger(); + public static final Map TASKS = new ConcurrentHashMap<>(); + + private String connectorName; + private String expectedRecordsStr; @Override public void start(Map props) { - log.info("Starting connector"); - COUNTER.set(0); + connectorName = props.get("name"); + expectedRecordsStr = props.get(EXPECTED_RECORDS); + log.info("Starting connector {}", props.get("name")); } @Override @@ -53,7 +61,9 @@ public List> taskConfigs(int maxTasks) { List> configs = new ArrayList<>(); for (int i = 0; i < maxTasks; i++) { Map config = new HashMap<>(); + config.put("task.id", connectorName + "-" + i); config.put("required", "dummy-val"); + config.put(EXPECTED_RECORDS, expectedRecordsStr); configs.add(config); } return configs; @@ -61,7 +71,7 @@ public List> taskConfigs(int maxTasks) { @Override public void stop() { - COUNTER.set(0); + } @Override @@ -71,6 +81,10 @@ public ConfigDef config() { public static class MonitorableSinkTask extends SinkTask { + private String taskId; + private int expectedRecords; + private CountDownLatch latch; + @Override public String version() { return "unknown"; @@ -79,20 +93,34 @@ public String version() { @Override public void start(Map props) { log.debug("Starting task {}", context); + taskId = props.get("task.id"); + expectedRecords = Integer.parseInt(props.get(EXPECTED_RECORDS)); + TASKS.put(taskId, this); + latch = new CountDownLatch(expectedRecords); } @Override public void put(Collection records) { for (SinkRecord rec : records) { - COUNTER.incrementAndGet(); + latch.countDown(); log.debug("Obtained record: {} at {}", rec.value(), context); } } @Override public void stop() { + log.info("Removing {}", TASKS.remove(taskId)); } + public void awaitRecords(int consumeMaxDurationMs) throws InterruptedException { + if (latch == null) { + throw new IllegalStateException("latch was not initialized"); + } else { + if (!latch.await(consumeMaxDurationMs, TimeUnit.MILLISECONDS)) { + throw new DataException("Insufficient records seen by task"); + } + } + } } } From 63bfbb40a664098d2d2115be38fe40876f0cb5eb Mon Sep 17 00:00:00 2001 From: Arjun Satish Date: Tue, 25 Sep 2018 11:31:28 -0700 Subject: [PATCH 18/52] Test can wait till task is created instead of busy loop Signed-off-by: Arjun Satish --- .../integration/ConnectIntegrationTest.java | 14 +----- .../DeadLetterQueueIntegrationTest.java | 14 +----- .../integration/MonitorableSinkConnector.java | 45 +++++++++++++++++-- 3 files changed, 46 insertions(+), 27 deletions(-) diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTest.java index f3ff3c6a615a2..a1e2bff9c6c0b 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTest.java @@ -81,19 +81,9 @@ public void testProduceConsumeConnector() throws Exception { connect.startConnector("simple-conn", props); - MonitorableSinkConnector.MonitorableSinkTask task = MonitorableSinkConnector.TASKS.get("simple-conn-0"); - for (int i = 0; i < 10 && task == null; i++) { - log.debug("Sleeping for 200 before looking up task"); - Thread.sleep(200); - task = MonitorableSinkConnector.TASKS.get("simple-conn-0"); - } - if (task == null) { - throw new ConnectException("Connector took too long to start"); - } - - task.awaitRecords(CONSUME_MAX_DURATION_MS); + MonitorableSinkConnector.taskInstances("simple-conn-0").task().awaitRecords(CONSUME_MAX_DURATION_MS); - log.info("Connector read {} records from topic", NUM_RECORDS_PRODUCED); + log.debug("Connector read {} records from topic", NUM_RECORDS_PRODUCED); connect.deleteConnector("simple-conn"); } } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/DeadLetterQueueIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/DeadLetterQueueIntegrationTest.java index 3ff3560245ce5..e4ec22d2bf92d 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/DeadLetterQueueIntegrationTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/DeadLetterQueueIntegrationTest.java @@ -96,25 +96,15 @@ public void testTransformationErrorHandling() throws Exception { connect.startConnector("simple-conn", props); - MonitorableSinkConnector.MonitorableSinkTask task = MonitorableSinkConnector.TASKS.get("simple-conn-0"); - for (int i = 0; i < 10 && task == null; i++) { - log.debug("Sleeping for 200 before looking up task"); - Thread.sleep(200); - task = MonitorableSinkConnector.TASKS.get("simple-conn-0"); - } - if (task == null) { - throw new ConnectException("Connector took too long to start"); - } - task.awaitRecords(CONSUME_MAX_DURATION_MS); + MonitorableSinkConnector.taskInstances("simple-conn-0").task().awaitRecords(CONSUME_MAX_DURATION_MS); // consume failed records from dead letter queue topic log.info("Consuming records from test topic"); for (ConsumerRecord recs : connect.kafka().consume(EXPECTED_INCORRECT_RECORDS, CONSUME_MAX_DURATION_MS, DLQ_TOPIC)) { - log.info("Consumed record (key={}, value={}) from dead letter queue topic {}", new String(recs.key()), new String(recs.value()), DLQ_TOPIC); + log.debug("Consumed record (key={}, value={}) from dead letter queue topic {}", new String(recs.key()), new String(recs.value()), DLQ_TOPIC); } connect.deleteConnector("simple-conn"); - } public static class FaultyPassthrough> implements Transformation { diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/MonitorableSinkConnector.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/MonitorableSinkConnector.java index 02b5a6bcaceb7..9d92d661ef766 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/MonitorableSinkConnector.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/MonitorableSinkConnector.java @@ -18,6 +18,7 @@ import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.connect.connector.Task; +import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.errors.DataException; import org.apache.kafka.connect.runtime.TestSinkConnector; import org.apache.kafka.connect.sink.SinkRecord; @@ -33,17 +34,52 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; public class MonitorableSinkConnector extends TestSinkConnector { public static final String EXPECTED_RECORDS = "expected_records"; private static final Logger log = LoggerFactory.getLogger(MonitorableSinkConnector.class); - public static final Map TASKS = new ConcurrentHashMap<>(); + private static final Map HANDLES = new ConcurrentHashMap<>(); private String connectorName; private String expectedRecordsStr; + public static class Handle { + private static final int MAX_WAIT_FOR_TASK_DURATION_MS = 60_000; + + private final String taskName; + private final AtomicReference taskRef = new AtomicReference<>(); + private final CountDownLatch taskAvailable = new CountDownLatch(1); + + public Handle(String taskName) { + this.taskName = taskName; + } + + public void task(MonitorableSinkTask task) { + if (this.taskRef.compareAndSet(null, task)) { + taskAvailable.countDown(); + } + } + + public MonitorableSinkTask task() { + try { + log.debug("Waiting on task {}", taskName); + taskAvailable.await(MAX_WAIT_FOR_TASK_DURATION_MS, TimeUnit.MILLISECONDS); + log.debug("Found task!"); + } catch (InterruptedException e) { + throw new ConnectException("Could not find task for " + taskName, e); + } + + return taskRef.get(); + } + } + + public static Handle taskInstances(String taskId) { + return HANDLES.computeIfAbsent(taskId, connName -> new Handle(taskId)); + } + @Override public void start(Map props) { connectorName = props.get("name"); @@ -69,6 +105,9 @@ public List> taskConfigs(int maxTasks) { return configs; } + // [2018-09-25 11:23:34,562] DEBUG Waiting on task simple-conn-0 (org.apache.kafka.connect.integration.MonitorableSinkConnector:68) + // [2018-09-25 11:23:36,058] DEBUG Found task! (org.apache.kafka.connect.integration.MonitorableSinkConnector:70) + @Override public void stop() { @@ -95,7 +134,7 @@ public void start(Map props) { log.debug("Starting task {}", context); taskId = props.get("task.id"); expectedRecords = Integer.parseInt(props.get(EXPECTED_RECORDS)); - TASKS.put(taskId, this); + taskInstances(taskId).task(this); latch = new CountDownLatch(expectedRecords); } @@ -109,7 +148,7 @@ public void put(Collection records) { @Override public void stop() { - log.info("Removing {}", TASKS.remove(taskId)); + log.info("Removing {}", HANDLES.remove(taskId)); } public void awaitRecords(int consumeMaxDurationMs) throws InterruptedException { From b9a1f19db0033ed59851e424210b1bff8f08956c Mon Sep 17 00:00:00 2001 From: Arjun Satish Date: Tue, 25 Sep 2018 11:42:43 -0700 Subject: [PATCH 19/52] Address other review comments Signed-off-by: Arjun Satish --- .../integration/ConnectIntegrationTest.java | 5 ++--- .../DeadLetterQueueIntegrationTest.java | 15 +++++++-------- .../integration/MonitorableSinkConnector.java | 10 +++++++--- 3 files changed, 16 insertions(+), 14 deletions(-) diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTest.java index a1e2bff9c6c0b..564bd6872cc8f 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTest.java @@ -62,9 +62,8 @@ public void testProduceConsumeConnector() throws Exception { connect.kafka().createTopic("test-topic"); // produce some strings into test topic - for (int i = 0; i < NUM_RECORDS_PRODUCED / 2; i++) { - connect.kafka().produce("test-topic", "hello-" + i); - connect.kafka().produce("test-topic", "world-" + i); + for (int i = 0; i < NUM_RECORDS_PRODUCED; i++) { + connect.kafka().produce("test-topic", "simple-message-value-" + i); } // consume all records from test topic or fail diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/DeadLetterQueueIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/DeadLetterQueueIntegrationTest.java index e4ec22d2bf92d..36c3613a30305 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/DeadLetterQueueIntegrationTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/DeadLetterQueueIntegrationTest.java @@ -43,8 +43,8 @@ public class DeadLetterQueueIntegrationTest { private static final String DLQ_TOPIC = "my-connector-errors"; private static final int NUM_RECORDS_PRODUCED = 20; - private static final int EXPECTED_CORRECT_RECORDS = 18; - private static final int EXPECTED_INCORRECT_RECORDS = 2; + private static final int EXPECTED_CORRECT_RECORDS = 19; + private static final int EXPECTED_INCORRECT_RECORDS = 1; private static final int CONSUME_MAX_DURATION_MS = 5000; private EmbeddedConnectCluster connect; @@ -66,9 +66,8 @@ public void testTransformationErrorHandling() throws Exception { connect.kafka().createTopic("test-topic"); // produce some strings into test topic - for (int i = 0; i < NUM_RECORDS_PRODUCED / 2; i++) { - connect.kafka().produce("test-topic", "hello-" + i, String.valueOf(i)); - connect.kafka().produce("test-topic", "world-" + i, String.valueOf(i)); + for (int i = 0; i < NUM_RECORDS_PRODUCED; i++) { + connect.kafka().produce("test-topic", "key-" + i, "value-" + String.valueOf(i)); } // consume all records from test topic @@ -117,9 +116,9 @@ public static class FaultyPassthrough> implements Tra @Override public R apply(R record) { - long val = Long.parseLong(String.valueOf(record.value())); - if (val == BAD_RECORD_VAL) { - throw new RetriableException("Error when when val=" + val); + String badVal = "value-" + BAD_RECORD_VAL; + if (badVal.equals(record.value())) { + throw new RetriableException("Error when when val=" + badVal); } return record; } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/MonitorableSinkConnector.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/MonitorableSinkConnector.java index 9d92d661ef766..7d5647c0cc94e 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/MonitorableSinkConnector.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/MonitorableSinkConnector.java @@ -153,13 +153,17 @@ public void stop() { public void awaitRecords(int consumeMaxDurationMs) throws InterruptedException { if (latch == null) { - throw new IllegalStateException("latch was not initialized"); + throw new IllegalStateException("Illegal state encountered. Maybe this task was not started by the framework?"); } else { if (!latch.await(consumeMaxDurationMs, TimeUnit.MILLISECONDS)) { - throw new DataException("Insufficient records seen by task"); + String msg = String.format("Insufficient records seen by task %s in %d millis. Records expected=%d, actual=%d", + taskId, + consumeMaxDurationMs, + expectedRecords, + expectedRecords - latch.getCount()); + throw new DataException(msg); } } } } - } From bf72fc68a8d380950811b3ab1a9832e2ec08875a Mon Sep 17 00:00:00 2001 From: Arjun Satish Date: Tue, 25 Sep 2018 11:45:53 -0700 Subject: [PATCH 20/52] Make variables final Signed-off-by: Arjun Satish --- .../java/org/apache/kafka/connect/cli/ConnectDistributed.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/cli/ConnectDistributed.java b/connect/runtime/src/main/java/org/apache/kafka/connect/cli/ConnectDistributed.java index 4aa0b2798ffc9..9be27a0bf4d8a 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/cli/ConnectDistributed.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/cli/ConnectDistributed.java @@ -54,8 +54,8 @@ public class ConnectDistributed { private static final Logger log = LoggerFactory.getLogger(ConnectDistributed.class); - private Time time = Time.SYSTEM; - private long initStart = time.hiResClockMs(); + private final Time time = Time.SYSTEM; + private final long initStart = time.hiResClockMs(); public static void main(String[] args) { From 1f0e9f2b85d4a5784c1ebd941d963d8bba4cf448 Mon Sep 17 00:00:00 2001 From: Arjun Satish Date: Tue, 25 Sep 2018 11:47:35 -0700 Subject: [PATCH 21/52] Remove unused imports Signed-off-by: Arjun Satish --- .../apache/kafka/connect/integration/ConnectIntegrationTest.java | 1 - .../connect/integration/DeadLetterQueueIntegrationTest.java | 1 - 2 files changed, 2 deletions(-) diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTest.java index 564bd6872cc8f..02543d7d3af56 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTest.java @@ -16,7 +16,6 @@ */ package org.apache.kafka.connect.integration; -import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.storage.StringConverter; import org.apache.kafka.connect.util.clusters.EmbeddedConnectCluster; import org.apache.kafka.test.IntegrationTest; diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/DeadLetterQueueIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/DeadLetterQueueIntegrationTest.java index 36c3613a30305..d763ef139b09d 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/DeadLetterQueueIntegrationTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/DeadLetterQueueIntegrationTest.java @@ -19,7 +19,6 @@ import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.connect.connector.ConnectRecord; -import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.errors.RetriableException; import org.apache.kafka.connect.storage.StringConverter; import org.apache.kafka.connect.transforms.Transformation; From b1def66eb1a814ec16f0984b5c2a8d4266454845 Mon Sep 17 00:00:00 2001 From: Arjun Satish Date: Thu, 4 Oct 2018 16:27:28 -0700 Subject: [PATCH 22/52] Add some javadoc and count down on latch when task exits Signed-off-by: Arjun Satish --- .../integration/ConnectIntegrationTest.java | 11 ++++-- .../DeadLetterQueueIntegrationTest.java | 9 +++-- .../integration/MonitorableSinkConnector.java | 30 +++++++++++----- .../util/clusters/EmbeddedConnectCluster.java | 4 +++ .../util/clusters/EmbeddedKafkaCluster.java | 36 ++++++++++++------- 5 files changed, 65 insertions(+), 25 deletions(-) diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTest.java index 02543d7d3af56..5556d0276311f 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTest.java @@ -30,6 +30,13 @@ import java.util.HashMap; import java.util.Map; +import static org.apache.kafka.connect.runtime.ConnectorConfig.CONNECTOR_CLASS_CONFIG; +import static org.apache.kafka.connect.runtime.ConnectorConfig.TASKS_MAX_CONFIG; + +/** + * A simple integration test which brings up Connect, Kafka and ZooKeeper and produces messages + * into a topic, and sinks them with a {@link MonitorableSinkConnector}. + */ @Category(IntegrationTest.class) public class ConnectIntegrationTest { @@ -70,8 +77,8 @@ public void testProduceConsumeConnector() throws Exception { connect.kafka().consume(NUM_RECORDS_PRODUCED, CONSUME_MAX_DURATION_MS, "test-topic"); Map props = new HashMap<>(); - props.put("connector.class", "MonitorableSink"); - props.put("task.max", "2"); + props.put(CONNECTOR_CLASS_CONFIG, "MonitorableSink"); + props.put(TASKS_MAX_CONFIG, "1"); props.put("topics", "test-topic"); props.put("key.converter", StringConverter.class.getName()); props.put("value.converter", StringConverter.class.getName()); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/DeadLetterQueueIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/DeadLetterQueueIntegrationTest.java index d763ef139b09d..93f2a7f2c068c 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/DeadLetterQueueIntegrationTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/DeadLetterQueueIntegrationTest.java @@ -35,6 +35,10 @@ import java.util.HashMap; import java.util.Map; +/** + * Start a connector and attempt to sink records, some of which cause one or more stages in + * the connector pipeline to fail. The bad records must be found in the dead letter queue. + */ @Category(IntegrationTest.class) public class DeadLetterQueueIntegrationTest { @@ -72,12 +76,13 @@ public void testTransformationErrorHandling() throws Exception { // consume all records from test topic log.info("Consuming records from test topic"); for (ConsumerRecord recs : connect.kafka().consume(NUM_RECORDS_PRODUCED, CONSUME_MAX_DURATION_MS, "test-topic")) { - log.debug("Consumed record ({}, {}) from topic {}", new String(recs.key()), new String(recs.value()), recs.topic()); + log.debug("Consumed record (key='{}', value='{}') from topic {}", + new String(recs.key()), new String(recs.value()), recs.topic()); } Map props = new HashMap<>(); props.put("connector.class", "MonitorableSink"); - props.put("task.max", "2"); + props.put("task.max", "1"); props.put("topics", "test-topic"); props.put("key.converter", StringConverter.class.getName()); props.put("value.converter", StringConverter.class.getName()); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/MonitorableSinkConnector.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/MonitorableSinkConnector.java index 7d5647c0cc94e..af747e3c6761e 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/MonitorableSinkConnector.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/MonitorableSinkConnector.java @@ -36,6 +36,11 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; +/** + * A connector to be used in integration tests. This class provides methods to find task instances + * which are initiated by the embedded connector, and wait for them to consume a desired number of + * messages. + */ public class MonitorableSinkConnector extends TestSinkConnector { public static final String EXPECTED_RECORDS = "expected_records"; @@ -49,12 +54,12 @@ public class MonitorableSinkConnector extends TestSinkConnector { public static class Handle { private static final int MAX_WAIT_FOR_TASK_DURATION_MS = 60_000; - private final String taskName; + private final String taskId; private final AtomicReference taskRef = new AtomicReference<>(); private final CountDownLatch taskAvailable = new CountDownLatch(1); - public Handle(String taskName) { - this.taskName = taskName; + public Handle(String taskId) { + this.taskId = taskId; } public void task(MonitorableSinkTask task) { @@ -65,15 +70,23 @@ public void task(MonitorableSinkTask task) { public MonitorableSinkTask task() { try { - log.debug("Waiting on task {}", taskName); + log.debug("Waiting on task {}", taskId); taskAvailable.await(MAX_WAIT_FOR_TASK_DURATION_MS, TimeUnit.MILLISECONDS); log.debug("Found task!"); } catch (InterruptedException e) { - throw new ConnectException("Could not find task for " + taskName, e); + throw new ConnectException("Could not find task for " + taskId, e); } return taskRef.get(); } + + @Override + public String toString() { + return "Handle{" + + "taskId='" + taskId + '\'' + + ", taskAvailable=" + taskAvailable.getCount() + + '}'; + } } public static Handle taskInstances(String taskId) { @@ -105,9 +118,6 @@ public List> taskConfigs(int maxTasks) { return configs; } - // [2018-09-25 11:23:34,562] DEBUG Waiting on task simple-conn-0 (org.apache.kafka.connect.integration.MonitorableSinkConnector:68) - // [2018-09-25 11:23:36,058] DEBUG Found task! (org.apache.kafka.connect.integration.MonitorableSinkConnector:70) - @Override public void stop() { @@ -148,7 +158,9 @@ public void put(Collection records) { @Override public void stop() { - log.info("Removing {}", HANDLES.remove(taskId)); + Handle handle = HANDLES.remove(taskId); + handle.taskAvailable.countDown(); + log.info("Removing {} for taskId {}", handle, taskId); } public void awaitRecords(int consumeMaxDurationMs) throws InterruptedException { 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 dfde009aefe63..7990e53ce6694 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 @@ -46,6 +46,10 @@ import static org.apache.kafka.connect.runtime.distributed.DistributedConfig.STATUS_STORAGE_REPLICATION_FACTOR_CONFIG; import static org.apache.kafka.connect.runtime.distributed.DistributedConfig.STATUS_STORAGE_TOPIC_CONFIG; +/** + * Start an embedded connect worker. Internally, this class will spin up a Kafka and Zk cluster, setup any tmp + * directories and clean up them on them. + */ public class EmbeddedConnectCluster extends ExternalResource { private static final Logger log = LoggerFactory.getLogger(EmbeddedConnectCluster.class); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java b/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java index ed83e934935b2..fa2f4ae5c7289 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java @@ -96,7 +96,7 @@ protected void after() { stop(); } - public void start() throws IOException { + private void start() throws IOException { zookeeper = new EmbeddedZookeeper(); brokerConfig.put(KafkaConfig$.MODULE$.ZkConnectProp(), zKConnectString()); @@ -252,7 +252,7 @@ public ConsumerRecords consume(int n, long maxDuration, String.. Map>> records = new HashMap<>(); int consumedRecords = 0; - try (KafkaConsumer consumer = createConsumer(topics)) { + try (KafkaConsumer consumer = createConsumerAndSubscribeTo(topics)) { for (int i = 0; i < num; i++) { ConsumerRecords rec = consumer.poll(Duration.ofMillis(duration)); if (rec.isEmpty()) { @@ -272,17 +272,16 @@ public ConsumerRecords consume(int n, long maxDuration, String.. throw new RuntimeException("Could not find enough records. found " + consumedRecords + ", expected " + n); } - public KafkaConsumer createConsumer(String... topics) { - // Include any unknown worker configs so consumer configs can be set globally on the worker - // and through to the task - Map props = new HashMap<>(); + public KafkaConsumer createConsumer(Map consumerProps) { - props.put(GROUP_ID_CONFIG, UUID.randomUUID().toString()); - props.put(BOOTSTRAP_SERVERS_CONFIG, bootstrapServers()); - props.put(ENABLE_AUTO_COMMIT_CONFIG, "false"); - props.put(AUTO_OFFSET_RESET_CONFIG, "earliest"); - props.put(KEY_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArrayDeserializer"); - props.put(VALUE_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArrayDeserializer"); + Map props = new HashMap<>(consumerProps); + + putIfAbsent(props, GROUP_ID_CONFIG, UUID.randomUUID().toString()); + putIfAbsent(props, BOOTSTRAP_SERVERS_CONFIG, bootstrapServers()); + putIfAbsent(props, ENABLE_AUTO_COMMIT_CONFIG, "false"); + putIfAbsent(props, AUTO_OFFSET_RESET_CONFIG, "earliest"); + putIfAbsent(props, KEY_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArrayDeserializer"); + putIfAbsent(props, VALUE_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArrayDeserializer"); KafkaConsumer consumer; try { @@ -290,9 +289,22 @@ public KafkaConsumer createConsumer(String... topics) { } catch (Throwable t) { throw new ConnectException("Failed to create consumer", t); } + return consumer; + } + + public KafkaConsumer createConsumer() { + return createConsumer(Collections.emptyMap()); + } + public KafkaConsumer createConsumerAndSubscribeTo(String... topics) { + KafkaConsumer consumer = createConsumer(); consumer.subscribe(Arrays.asList(topics)); return consumer; } + private static void putIfAbsent(final Map props, final String propertyKey, final Object propertyValue) { + if (!props.containsKey(propertyKey)) { + props.put(propertyKey, propertyValue); + } + } } From 1fb10b86d78614309b3e9667acc1c660f86f3c6f Mon Sep 17 00:00:00 2001 From: Arjun Satish Date: Thu, 4 Oct 2018 17:16:53 -0700 Subject: [PATCH 23/52] Update ConnectIntegrationTest to use multiple topic partitions and tasks Signed-off-by: Arjun Satish --- .../connect/integration/ConnectIntegrationTest.java | 13 ++++++++----- .../integration/MonitorableSinkConnector.java | 2 +- .../connect/util/clusters/EmbeddedKafkaCluster.java | 8 ++++++-- 3 files changed, 15 insertions(+), 8 deletions(-) diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTest.java index 5556d0276311f..6e3442d6873b3 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTest.java @@ -27,6 +27,7 @@ import org.slf4j.LoggerFactory; import java.io.IOException; +import java.util.Collections; import java.util.HashMap; import java.util.Map; @@ -35,7 +36,7 @@ /** * A simple integration test which brings up Connect, Kafka and ZooKeeper and produces messages - * into a topic, and sinks them with a {@link MonitorableSinkConnector}. + * into multiple topic-partitions, and sinks them with multiple tasks of a {@link MonitorableSinkConnector}. */ @Category(IntegrationTest.class) public class ConnectIntegrationTest { @@ -43,6 +44,7 @@ public class ConnectIntegrationTest { private static final Logger log = LoggerFactory.getLogger(ConnectIntegrationTest.class); private static final int NUM_RECORDS_PRODUCED = 2000; + private static final int NUM_TOPIC_PARTITIONS = 2; private static final int CONSUME_MAX_DURATION_MS = 5000; private EmbeddedConnectCluster connect; @@ -65,11 +67,11 @@ public void close() { @Test public void testProduceConsumeConnector() throws Exception { // create test topic - connect.kafka().createTopic("test-topic"); + connect.kafka().createTopic("test-topic", NUM_TOPIC_PARTITIONS, 1, Collections.emptyMap()); // produce some strings into test topic for (int i = 0; i < NUM_RECORDS_PRODUCED; i++) { - connect.kafka().produce("test-topic", "simple-message-value-" + i); + connect.kafka().produce("test-topic", i % NUM_TOPIC_PARTITIONS, "key", "simple-message-value-" + i); } // consume all records from test topic or fail @@ -78,15 +80,16 @@ public void testProduceConsumeConnector() throws Exception { Map props = new HashMap<>(); props.put(CONNECTOR_CLASS_CONFIG, "MonitorableSink"); - props.put(TASKS_MAX_CONFIG, "1"); + props.put(TASKS_MAX_CONFIG, "2"); props.put("topics", "test-topic"); props.put("key.converter", StringConverter.class.getName()); props.put("value.converter", StringConverter.class.getName()); - props.put(MonitorableSinkConnector.EXPECTED_RECORDS, String.valueOf(NUM_RECORDS_PRODUCED)); + props.put(MonitorableSinkConnector.EXPECTED_RECORDS, String.valueOf(NUM_RECORDS_PRODUCED/NUM_TOPIC_PARTITIONS)); connect.startConnector("simple-conn", props); MonitorableSinkConnector.taskInstances("simple-conn-0").task().awaitRecords(CONSUME_MAX_DURATION_MS); + MonitorableSinkConnector.taskInstances("simple-conn-1").task().awaitRecords(CONSUME_MAX_DURATION_MS); log.debug("Connector read {} records from topic", NUM_RECORDS_PRODUCED); connect.deleteConnector("simple-conn"); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/MonitorableSinkConnector.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/MonitorableSinkConnector.java index af747e3c6761e..18f3c0d23c1e8 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/MonitorableSinkConnector.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/MonitorableSinkConnector.java @@ -152,7 +152,7 @@ public void start(Map props) { public void put(Collection records) { for (SinkRecord rec : records) { latch.countDown(); - log.debug("Obtained record: {} at {}", rec.value(), context); + log.debug("Obtained record (key='{}' value='{}') at task {}", rec.key(), rec.value(), context); } } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java b/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java index fa2f4ae5c7289..eac1544d737d5 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java @@ -214,11 +214,15 @@ public void createTopic(String topic, int partitions, int replication, Map(topic, key == null ? null : key.getBytes(), value == null ? null : value.getBytes())); + produce(topic, null, key, value); + } + + public void produce(String topic, Integer partition, String key, String value) { + producer.send(new ProducerRecord<>(topic, partition, key == null ? null : key.getBytes(), value == null ? null : value.getBytes())); } public AdminClient createAdminClient() { From cdcfb9f4095cb7db5f5ba13e9c48a62ced027b94 Mon Sep 17 00:00:00 2001 From: Arjun Satish Date: Fri, 5 Oct 2018 15:55:27 -0700 Subject: [PATCH 24/52] Changes to startConnector method Signed-off-by: Arjun Satish --- .../integration/ConnectIntegrationTest.java | 4 +-- .../DeadLetterQueueIntegrationTest.java | 2 +- .../util/clusters/EmbeddedConnectCluster.java | 31 ++++++++++++++----- 3 files changed, 27 insertions(+), 10 deletions(-) diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTest.java index 6e3442d6873b3..a6f030eecdfed 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTest.java @@ -84,9 +84,9 @@ public void testProduceConsumeConnector() throws Exception { props.put("topics", "test-topic"); props.put("key.converter", StringConverter.class.getName()); props.put("value.converter", StringConverter.class.getName()); - props.put(MonitorableSinkConnector.EXPECTED_RECORDS, String.valueOf(NUM_RECORDS_PRODUCED/NUM_TOPIC_PARTITIONS)); + props.put(MonitorableSinkConnector.EXPECTED_RECORDS, String.valueOf(NUM_RECORDS_PRODUCED / NUM_TOPIC_PARTITIONS)); - connect.startConnector("simple-conn", props); + connect.configureConnector("simple-conn", props); MonitorableSinkConnector.taskInstances("simple-conn-0").task().awaitRecords(CONSUME_MAX_DURATION_MS); MonitorableSinkConnector.taskInstances("simple-conn-1").task().awaitRecords(CONSUME_MAX_DURATION_MS); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/DeadLetterQueueIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/DeadLetterQueueIntegrationTest.java index 93f2a7f2c068c..9d175ef015bbf 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/DeadLetterQueueIntegrationTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/DeadLetterQueueIntegrationTest.java @@ -97,7 +97,7 @@ public void testTransformationErrorHandling() throws Exception { props.put("transforms", "failing_transform"); props.put("transforms.failing_transform.type", FaultyPassthrough.class.getName()); - connect.startConnector("simple-conn", props); + connect.configureConnector("simple-conn", props); MonitorableSinkConnector.taskInstances("simple-conn-0").task().awaitRecords(CONSUME_MAX_DURATION_MS); 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 7990e53ce6694..787184f72f8c2 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 @@ -20,6 +20,7 @@ import org.apache.kafka.connect.cli.ConnectDistributed; import org.apache.kafka.connect.runtime.Connect; import org.apache.kafka.connect.runtime.WorkerConfig; +import org.apache.kafka.connect.runtime.rest.errors.ConnectRestException; import org.junit.rules.ExternalResource; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -135,25 +136,41 @@ public void startConnect() { connect = new ConnectDistributed().startConnect(workerProps); } - public void startConnector(String connName, Map connConfig) throws IOException { + /** + * Configure a connector. If the connector does not already exist, a new one will be created and + * the given configuration will be applied to it. + * + * @param connName the name of the connector + * @param connConfig the intended configuration + * @throws IOException if call to the REST api fails. + * @throws ConnectRestException if REST api returns error status + */ + public void configureConnector(String connName, Map connConfig) throws IOException { String url = String.format("http://%s:%s/connectors/%s/config", REST_HOST_NAME, WorkerConfig.REST_PORT_DEFAULT, connName); ObjectMapper mapper = new ObjectMapper(); + int status; try { String content = mapper.writeValueAsString(connConfig); - int status = executePut(url, content); - if (status >= HttpServletResponse.SC_BAD_REQUEST) { - throw new IOException("Could not execute PUT request. status=" + status); - } + status = executePut(url, content); } catch (IOException e) { log.error("Could not serialize config", e); - throw new IOException(e); + throw e; + } + if (status >= HttpServletResponse.SC_BAD_REQUEST) { + throw new ConnectRestException(status, "Could not execute PUT request"); } } + /** + * Delete an existing connector. + * + * @param connName name of the connector to be deleted + * @throws IOException if call to the REST api fails. + */ public void deleteConnector(String connName) throws IOException { int status = executeDelete(String.format("http://%s:%s/connectors/%s", REST_HOST_NAME, WorkerConfig.REST_PORT_DEFAULT, connName)); if (status >= HttpServletResponse.SC_BAD_REQUEST) { - throw new IOException("Could not execute DELETE request. status=" + status); + throw new ConnectRestException(status, "Could not execute DELETE request."); } } From 213b31c42d80275833a8b9fa60fe826ce5473fb0 Mon Sep 17 00:00:00 2001 From: Arjun Satish Date: Fri, 5 Oct 2018 16:38:52 -0700 Subject: [PATCH 25/52] Remove before() and after() methods from EmbeddedConnectCluster Signed-off-by: Arjun Satish --- .../util/clusters/EmbeddedConnectCluster.java | 22 +++++++------------ 1 file changed, 8 insertions(+), 14 deletions(-) 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 787184f72f8c2..82ebea735cb77 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 @@ -51,7 +51,7 @@ * Start an embedded connect worker. Internally, this class will spin up a Kafka and Zk cluster, setup any tmp * directories and clean up them on them. */ -public class EmbeddedConnectCluster extends ExternalResource { +public class EmbeddedConnectCluster { private static final Logger log = LoggerFactory.getLogger(EmbeddedConnectCluster.class); @@ -85,25 +85,19 @@ public EmbeddedConnectCluster(String name, Map workerProps, int kafkaCluster = new EmbeddedKafkaCluster(numBrokers, brokerProps); } - public void start() throws IOException { - before(); - } - /** - * Stop the connect cluster and the embedded Kafka and + * Start the connect cluster and the embedded Kafka and Zookeeper cluster. */ - public void stop() { - after(); - } - - @Override - protected void before() throws IOException { + public void start() throws IOException { kafkaCluster.before(); startConnect(); } - @Override - protected void after() { + /** + * Stop the connect cluster and the embedded Kafka and Zookeeper cluster. + * Clean up any temp directories created locally. + */ + public void stop() { try { connect.stop(); } catch (Exception e) { From 2f235c08ba9836b76973be4e7b9ef13b7345c602 Mon Sep 17 00:00:00 2001 From: Arjun Satish Date: Fri, 5 Oct 2018 16:43:45 -0700 Subject: [PATCH 26/52] Generate clusterName randomly instead of using argument Signed-off-by: Arjun Satish --- .../kafka/connect/integration/ConnectIntegrationTest.java | 2 +- .../integration/DeadLetterQueueIntegrationTest.java | 2 +- .../connect/util/clusters/EmbeddedConnectCluster.java | 8 ++++---- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTest.java index a6f030eecdfed..5695bb6f5c4ea 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTest.java @@ -51,7 +51,7 @@ public class ConnectIntegrationTest { @Before public void setup() throws IOException { - connect = new EmbeddedConnectCluster(ConnectIntegrationTest.class); + connect = new EmbeddedConnectCluster(); connect.start(); } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/DeadLetterQueueIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/DeadLetterQueueIntegrationTest.java index 9d175ef015bbf..a40d1c36f0bc5 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/DeadLetterQueueIntegrationTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/DeadLetterQueueIntegrationTest.java @@ -54,7 +54,7 @@ public class DeadLetterQueueIntegrationTest { @Before public void setup() throws IOException { - connect = new EmbeddedConnectCluster(ConnectIntegrationTest.class); + connect = new EmbeddedConnectCluster(); connect.start(); } 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 82ebea735cb77..b28d3fc0fd936 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 @@ -21,7 +21,6 @@ import org.apache.kafka.connect.runtime.Connect; import org.apache.kafka.connect.runtime.WorkerConfig; import org.apache.kafka.connect.runtime.rest.errors.ConnectRestException; -import org.junit.rules.ExternalResource; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -34,6 +33,7 @@ import java.util.HashMap; import java.util.Map; import java.util.Properties; +import java.util.UUID; import static org.apache.kafka.clients.consumer.ConsumerConfig.GROUP_ID_CONFIG; import static org.apache.kafka.connect.runtime.ConnectorConfig.KEY_CONVERTER_CLASS_CONFIG; @@ -66,8 +66,8 @@ public class EmbeddedConnectCluster { private Connect connect; - public EmbeddedConnectCluster(Class klass) { - this(klass.getName()); + public EmbeddedConnectCluster() { + this(UUID.randomUUID().toString()); } public EmbeddedConnectCluster(String name) { @@ -112,7 +112,7 @@ public void stop() { } public void startConnect() { - log.info("Starting Connect cluster with one worker."); + log.info("Starting Connect cluster with one worker. clusterName=" + clusterName); workerProps.put(BOOTSTRAP_SERVERS_CONFIG, kafka().bootstrapServers()); workerProps.put(REST_HOST_NAME_CONFIG, REST_HOST_NAME); From 676721cd8861ebd8a02bf2a19174c7e01662e445 Mon Sep 17 00:00:00 2001 From: Arjun Satish Date: Fri, 5 Oct 2018 16:55:45 -0700 Subject: [PATCH 27/52] bootstrapServers() returns all brokers Signed-off-by: Arjun Satish --- .../kafka/connect/util/clusters/EmbeddedKafkaCluster.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java b/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java index eac1544d737d5..41e06efa4222f 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java @@ -38,6 +38,7 @@ import org.apache.kafka.common.network.ListenerName; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Utils; import org.apache.kafka.connect.errors.ConnectException; import org.junit.rules.ExternalResource; import org.junit.rules.TemporaryFolder; @@ -55,6 +56,7 @@ import java.util.Properties; import java.util.UUID; import java.util.concurrent.ExecutionException; +import java.util.stream.Collectors; import static org.apache.kafka.clients.consumer.ConsumerConfig.AUTO_OFFSET_RESET_CONFIG; import static org.apache.kafka.clients.consumer.ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG; @@ -171,7 +173,9 @@ private String createLogDir() throws IOException { } public String bootstrapServers() { - return address(brokers[0]); + return Arrays.stream(brokers) + .map(this::address) + .collect(Collectors.joining(",")); } public String address(KafkaServer server) { From 4cc9a73da787f78c9a560b204b9a5a40406e3db3 Mon Sep 17 00:00:00 2001 From: Arjun Satish Date: Fri, 5 Oct 2018 16:59:22 -0700 Subject: [PATCH 28/52] Throw exception if desired replicator is unachievable Signed-off-by: Arjun Satish --- .../kafka/connect/util/clusters/EmbeddedKafkaCluster.java | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java b/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java index 41e06efa4222f..de6ec911a123c 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java @@ -35,10 +35,10 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.config.SslConfigs; import org.apache.kafka.common.config.types.Password; +import org.apache.kafka.common.errors.InvalidReplicationFactorException; import org.apache.kafka.common.network.ListenerName; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Time; -import org.apache.kafka.common.utils.Utils; import org.apache.kafka.connect.errors.ConnectException; import org.junit.rules.ExternalResource; import org.junit.rules.TemporaryFolder; @@ -204,7 +204,11 @@ public void createTopic(String topic) { * @param topicConfig Additional topic-level configuration settings. */ public void createTopic(String topic, int partitions, int replication, Map topicConfig) { - replication = Math.min(replication, brokers.length); + if (replication > brokers.length) { + throw new InvalidReplicationFactorException("Insufficient brokers (" + + brokers.length + ") for desired replication (" + replication + ")"); + } + log.debug("Creating topic { name: {}, partitions: {}, replication: {}, config: {} }", topic, partitions, replication, topicConfig); final NewTopic newTopic = new NewTopic(topic, partitions, (short) replication); From 7380c1d334008648b8d0e140f1fd48a943dafd7f Mon Sep 17 00:00:00 2001 From: Arjun Satish Date: Fri, 5 Oct 2018 17:19:43 -0700 Subject: [PATCH 29/52] Refactor logic for consuming up to n records in maxDuration Signed-off-by: Arjun Satish --- .../util/clusters/EmbeddedKafkaCluster.java | 17 ++++++----------- 1 file changed, 6 insertions(+), 11 deletions(-) diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java b/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java index de6ec911a123c..e585094d51f0d 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java @@ -249,24 +249,18 @@ public AdminClient createAdminClient() { * Consume at least n records in a given duration or throw an exception. * * @param n the number of expected records in this topic. - * @param maxDuration the max duration to wait for these records. + * @param maxDuration the max duration to wait for these records (in milliseconds). * @param topics the topics to subscribe and consume records from. * @return a {@link ConsumerRecords} collection containing at least n records. */ public ConsumerRecords consume(int n, long maxDuration, String... topics) { - long num = 1; - long duration = maxDuration; - // poll in intervals of 250millis. - if (duration > 250) { - num = 1 + (maxDuration / 250); - duration = 250; - } - Map>> records = new HashMap<>(); int consumedRecords = 0; try (KafkaConsumer consumer = createConsumerAndSubscribeTo(topics)) { - for (int i = 0; i < num; i++) { - ConsumerRecords rec = consumer.poll(Duration.ofMillis(duration)); + final long startMillis = System.currentTimeMillis(); + long current = startMillis; + while (current - startMillis < maxDuration) { + ConsumerRecords rec = consumer.poll(Duration.ofMillis(maxDuration)); if (rec.isEmpty()) { continue; } @@ -278,6 +272,7 @@ public ConsumerRecords consume(int n, long maxDuration, String.. if (consumedRecords >= n) { return new ConsumerRecords<>(records); } + current = System.currentTimeMillis(); } } From b3cf19a019c5336faf451816e9639aa3dd7005f0 Mon Sep 17 00:00:00 2001 From: Arjun Satish Date: Mon, 8 Oct 2018 08:18:27 -0700 Subject: [PATCH 30/52] Remove unused methods Signed-off-by: Arjun Satish --- .../connect/util/clusters/EmbeddedKafkaCluster.java | 11 +++-------- 1 file changed, 3 insertions(+), 8 deletions(-) diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java b/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java index e585094d51f0d..2b2ffb947f26b 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java @@ -256,7 +256,7 @@ public AdminClient createAdminClient() { public ConsumerRecords consume(int n, long maxDuration, String... topics) { Map>> records = new HashMap<>(); int consumedRecords = 0; - try (KafkaConsumer consumer = createConsumerAndSubscribeTo(topics)) { + try (KafkaConsumer consumer = createConsumerAndSubscribeTo(Collections.emptyMap(), topics)) { final long startMillis = System.currentTimeMillis(); long current = startMillis; while (current - startMillis < maxDuration) { @@ -280,7 +280,6 @@ public ConsumerRecords consume(int n, long maxDuration, String.. } public KafkaConsumer createConsumer(Map consumerProps) { - Map props = new HashMap<>(consumerProps); putIfAbsent(props, GROUP_ID_CONFIG, UUID.randomUUID().toString()); @@ -299,12 +298,8 @@ public KafkaConsumer createConsumer(Map consumer return consumer; } - public KafkaConsumer createConsumer() { - return createConsumer(Collections.emptyMap()); - } - - public KafkaConsumer createConsumerAndSubscribeTo(String... topics) { - KafkaConsumer consumer = createConsumer(); + public KafkaConsumer createConsumerAndSubscribeTo(Map consumerProps, String... topics) { + KafkaConsumer consumer = createConsumer(consumerProps); consumer.subscribe(Arrays.asList(topics)); return consumer; } From 00d91774dec7310120f00858e6066038acddacda Mon Sep 17 00:00:00 2001 From: Arjun Satish Date: Mon, 8 Oct 2018 15:33:06 -0700 Subject: [PATCH 31/52] Rename tests and broaden the scope of error handling test Signed-off-by: Arjun Satish --- .../DeadLetterQueueIntegrationTest.java | 143 ------------- .../ErrorHandlingIntegrationTest.java | 202 ++++++++++++++++++ ...ava => ExampleConnectIntegrationTest.java} | 35 +-- 3 files changed, 221 insertions(+), 159 deletions(-) delete mode 100644 connect/runtime/src/test/java/org/apache/kafka/connect/integration/DeadLetterQueueIntegrationTest.java create mode 100644 connect/runtime/src/test/java/org/apache/kafka/connect/integration/ErrorHandlingIntegrationTest.java rename connect/runtime/src/test/java/org/apache/kafka/connect/integration/{ConnectIntegrationTest.java => ExampleConnectIntegrationTest.java} (68%) diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/DeadLetterQueueIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/DeadLetterQueueIntegrationTest.java deleted file mode 100644 index a40d1c36f0bc5..0000000000000 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/DeadLetterQueueIntegrationTest.java +++ /dev/null @@ -1,143 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.kafka.connect.integration; - -import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.common.config.ConfigDef; -import org.apache.kafka.connect.connector.ConnectRecord; -import org.apache.kafka.connect.errors.RetriableException; -import org.apache.kafka.connect.storage.StringConverter; -import org.apache.kafka.connect.transforms.Transformation; -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 org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; -import java.util.HashMap; -import java.util.Map; - -/** - * Start a connector and attempt to sink records, some of which cause one or more stages in - * the connector pipeline to fail. The bad records must be found in the dead letter queue. - */ -@Category(IntegrationTest.class) -public class DeadLetterQueueIntegrationTest { - - private static final Logger log = LoggerFactory.getLogger(DeadLetterQueueIntegrationTest.class); - - private static final String DLQ_TOPIC = "my-connector-errors"; - private static final int NUM_RECORDS_PRODUCED = 20; - private static final int EXPECTED_CORRECT_RECORDS = 19; - private static final int EXPECTED_INCORRECT_RECORDS = 1; - private static final int CONSUME_MAX_DURATION_MS = 5000; - - private EmbeddedConnectCluster connect; - - @Before - public void setup() throws IOException { - connect = new EmbeddedConnectCluster(); - connect.start(); - } - - @After - public void close() { - connect.stop(); - } - - @Test - public void testTransformationErrorHandling() throws Exception { - // create test topic - connect.kafka().createTopic("test-topic"); - - // produce some strings into test topic - for (int i = 0; i < NUM_RECORDS_PRODUCED; i++) { - connect.kafka().produce("test-topic", "key-" + i, "value-" + String.valueOf(i)); - } - - // consume all records from test topic - log.info("Consuming records from test topic"); - for (ConsumerRecord recs : connect.kafka().consume(NUM_RECORDS_PRODUCED, CONSUME_MAX_DURATION_MS, "test-topic")) { - log.debug("Consumed record (key='{}', value='{}') from topic {}", - new String(recs.key()), new String(recs.value()), recs.topic()); - } - - Map props = new HashMap<>(); - props.put("connector.class", "MonitorableSink"); - props.put("task.max", "1"); - props.put("topics", "test-topic"); - props.put("key.converter", StringConverter.class.getName()); - props.put("value.converter", StringConverter.class.getName()); - props.put(MonitorableSinkConnector.EXPECTED_RECORDS, String.valueOf(EXPECTED_CORRECT_RECORDS)); - - props.put("errors.log.enable", "true"); - props.put("errors.log.include.messages", "false"); - - props.put("errors.deadletterqueue.topic.name", DLQ_TOPIC); - props.put("errors.deadletterqueue.topic.replication.factor", "1"); - props.put("errors.tolerance", "all"); - props.put("transforms", "failing_transform"); - props.put("transforms.failing_transform.type", FaultyPassthrough.class.getName()); - - connect.configureConnector("simple-conn", props); - - MonitorableSinkConnector.taskInstances("simple-conn-0").task().awaitRecords(CONSUME_MAX_DURATION_MS); - - // consume failed records from dead letter queue topic - log.info("Consuming records from test topic"); - for (ConsumerRecord recs : connect.kafka().consume(EXPECTED_INCORRECT_RECORDS, CONSUME_MAX_DURATION_MS, DLQ_TOPIC)) { - log.debug("Consumed record (key={}, value={}) from dead letter queue topic {}", new String(recs.key()), new String(recs.value()), DLQ_TOPIC); - } - - connect.deleteConnector("simple-conn"); - } - - public static class FaultyPassthrough> implements Transformation { - - static final ConfigDef CONFIG_DEF = new ConfigDef(); - /** - * an arbitrary id which causes this transformation to fail with a {@link RetriableException}. - */ - private static final int BAD_RECORD_VAL = 7; - - @Override - public R apply(R record) { - String badVal = "value-" + BAD_RECORD_VAL; - if (badVal.equals(record.value())) { - throw new RetriableException("Error when when val=" + badVal); - } - return record; - } - - @Override - public ConfigDef config() { - return CONFIG_DEF; - } - - @Override - public void close() { - } - - @Override - public void configure(Map configs) { - } - } -} diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ErrorHandlingIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ErrorHandlingIntegrationTest.java new file mode 100644 index 0000000000000..89ff98c153db4 --- /dev/null +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ErrorHandlingIntegrationTest.java @@ -0,0 +1,202 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.connect.integration; + +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.common.header.Headers; +import org.apache.kafka.connect.connector.ConnectRecord; +import org.apache.kafka.connect.errors.RetriableException; +import org.apache.kafka.connect.storage.StringConverter; +import org.apache.kafka.connect.transforms.Transformation; +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 org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +import static org.apache.kafka.connect.runtime.ConnectorConfig.CONNECTOR_CLASS_CONFIG; +import static org.apache.kafka.connect.runtime.ConnectorConfig.ERRORS_LOG_ENABLE_CONFIG; +import static org.apache.kafka.connect.runtime.ConnectorConfig.ERRORS_LOG_INCLUDE_MESSAGES_CONFIG; +import static org.apache.kafka.connect.runtime.ConnectorConfig.ERRORS_RETRY_TIMEOUT_CONFIG; +import static org.apache.kafka.connect.runtime.ConnectorConfig.ERRORS_TOLERANCE_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.TRANSFORMS_CONFIG; +import static org.apache.kafka.connect.runtime.ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG; +import static org.apache.kafka.connect.runtime.SinkConnectorConfig.DLQ_CONTEXT_HEADERS_ENABLE_CONFIG; +import static org.apache.kafka.connect.runtime.SinkConnectorConfig.DLQ_TOPIC_NAME_CONFIG; +import static org.apache.kafka.connect.runtime.SinkConnectorConfig.DLQ_TOPIC_REPLICATION_FACTOR_CONFIG; +import static org.apache.kafka.connect.runtime.SinkConnectorConfig.TOPICS_CONFIG; +import static org.apache.kafka.connect.runtime.errors.DeadLetterQueueReporter.ERROR_HEADER_EXCEPTION; +import static org.apache.kafka.connect.runtime.errors.DeadLetterQueueReporter.ERROR_HEADER_EXCEPTION_MESSAGE; +import static org.apache.kafka.connect.runtime.errors.DeadLetterQueueReporter.ERROR_HEADER_ORIG_TOPIC; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +/** + * Integration test for the different error handling policies in Connect (namely, retry policies, skipping bad records, + * and dead letter queues). + */ +@Category(IntegrationTest.class) +public class ErrorHandlingIntegrationTest { + + private static final Logger log = LoggerFactory.getLogger(ErrorHandlingIntegrationTest.class); + + private static final String DLQ_TOPIC = "my-connector-errors"; + private static final int NUM_RECORDS_PRODUCED = 20; + private static final int EXPECTED_CORRECT_RECORDS = 19; + private static final int EXPECTED_INCORRECT_RECORDS = 1; + private static final int CONSUME_MAX_DURATION_MS = 5000; + + private EmbeddedConnectCluster connect; + + @Before + public void setup() throws IOException { + connect = new EmbeddedConnectCluster(); + connect.start(); + } + + @After + public void close() { + connect.stop(); + } + + @Test + public void testSkipRetryAndDLQWithHeaders() throws Exception { + // create test topic + connect.kafka().createTopic("test-topic"); + + // produce some strings into test topic + for (int i = 0; i < NUM_RECORDS_PRODUCED; i++) { + connect.kafka().produce("test-topic", "key-" + i, "value-" + String.valueOf(i)); + } + + // consume all records from test topic + log.info("Consuming records from test topic"); + for (ConsumerRecord recs : connect.kafka().consume(NUM_RECORDS_PRODUCED, CONSUME_MAX_DURATION_MS, "test-topic")) { + log.debug("Consumed record (key='{}', value='{}') from topic {}", + new String(recs.key()), new String(recs.value()), recs.topic()); + } + + Map props = new HashMap<>(); + props.put(CONNECTOR_CLASS_CONFIG, "MonitorableSink"); + props.put(TASKS_MAX_CONFIG, "1"); + props.put(TOPICS_CONFIG, "test-topic"); + props.put(KEY_CONVERTER_CLASS_CONFIG, StringConverter.class.getName()); + props.put(VALUE_CONVERTER_CLASS_CONFIG, StringConverter.class.getName()); + props.put(MonitorableSinkConnector.EXPECTED_RECORDS, String.valueOf(EXPECTED_CORRECT_RECORDS)); + + props.put(ERRORS_LOG_ENABLE_CONFIG, "true"); + props.put(ERRORS_LOG_INCLUDE_MESSAGES_CONFIG, "false"); + + // produce bad messages into dead letter queue + props.put(DLQ_TOPIC_NAME_CONFIG, DLQ_TOPIC); + props.put(DLQ_CONTEXT_HEADERS_ENABLE_CONFIG, "true"); + props.put(DLQ_TOPIC_REPLICATION_FACTOR_CONFIG, "1"); + // tolerate all erros + props.put(ERRORS_TOLERANCE_CONFIG, "all"); + props.put(TRANSFORMS_CONFIG, "failing_transform"); + // retry for up to one second + props.put(ERRORS_RETRY_TIMEOUT_CONFIG, "1000"); + props.put("transforms.failing_transform.type", FaultyPassthrough.class.getName()); + + connect.configureConnector("simple-conn", props); + + MonitorableSinkConnector.taskInstances("simple-conn-0").task().awaitRecords(CONSUME_MAX_DURATION_MS); + + // consume failed records from dead letter queue topic + log.info("Consuming records from test topic"); + ConsumerRecords messages = connect.kafka().consume(EXPECTED_INCORRECT_RECORDS, CONSUME_MAX_DURATION_MS, DLQ_TOPIC); + for (ConsumerRecord recs : messages) { + log.debug("Consumed record (key={}, value={}) from dead letter queue topic {}", + new String(recs.key()), new String(recs.value()), DLQ_TOPIC); + assertTrue(recs.headers().toArray().length > 0); + assertValue("test-topic", recs.headers(), ERROR_HEADER_ORIG_TOPIC); + assertValue(RetriableException.class.getName(), recs.headers(), ERROR_HEADER_EXCEPTION); + assertValue("Error when when value='value-7'", recs.headers(), ERROR_HEADER_EXCEPTION_MESSAGE); + } + + connect.deleteConnector("simple-conn"); + } + + private void assertValue(String expected, Headers headers, String headerKey) { + byte[] actual = headers.lastHeader(headerKey).value(); + if (expected == null && actual == null) { + return; + } + if (expected == null || actual == null) { + fail(); + } + assertEquals(expected, new String(actual)); + } + + public static class FaultyPassthrough> implements Transformation { + + static final ConfigDef CONFIG_DEF = new ConfigDef(); + + /** + * an arbitrary id which causes this transformation to fail with a {@link RetriableException}, but succeeds + * on subsequent attempt. + */ + static final int BAD_RECORD_VAL_RETRIABLE = 4; + + /** + * an arbitrary id which causes this transformation to fail with a {@link RetriableException}. + */ + static final int BAD_RECORD_VAL = 7; + + private boolean shouldFail = true; + + @Override + public R apply(R record) { + String badValRetriable = "val-" + BAD_RECORD_VAL_RETRIABLE; + if (badValRetriable.equals(record.value()) && shouldFail) { + shouldFail = false; + throw new RetriableException("Error when when value='" + badValRetriable + + "'. A reattempt with this record will succeed."); + } + String badVal = "value-" + BAD_RECORD_VAL; + if (badVal.equals(record.value())) { + throw new RetriableException("Error when when value='" + badVal + "'"); + } + return record; + } + + @Override + public ConfigDef config() { + return CONFIG_DEF; + } + + @Override + public void close() { + } + + @Override + public void configure(Map configs) { + } + } +} diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ExampleConnectIntegrationTest.java similarity index 68% rename from connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTest.java rename to connect/runtime/src/test/java/org/apache/kafka/connect/integration/ExampleConnectIntegrationTest.java index 5695bb6f5c4ea..623cf78455e34 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ExampleConnectIntegrationTest.java @@ -23,8 +23,6 @@ import org.junit.Before; import org.junit.Test; import org.junit.experimental.categories.Category; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.io.IOException; import java.util.Collections; @@ -32,16 +30,19 @@ import java.util.Map; import static org.apache.kafka.connect.runtime.ConnectorConfig.CONNECTOR_CLASS_CONFIG; +import static org.apache.kafka.connect.runtime.ConnectorConfig.KEY_CONVERTER_CLASS_CONFIG; import static org.apache.kafka.connect.runtime.ConnectorConfig.TASKS_MAX_CONFIG; +import static org.apache.kafka.connect.runtime.ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG; +import static org.apache.kafka.connect.runtime.SinkConnectorConfig.TOPICS_CONFIG; /** - * A simple integration test which brings up Connect, Kafka and ZooKeeper and produces messages - * into multiple topic-partitions, and sinks them with multiple tasks of a {@link MonitorableSinkConnector}. + * An example integration test that demonstrates how to setup an integration test for Connect. + *

+ * The following test configures and executes up a sink connector pipeline in a worker, produces messages into + * the source topic-partitions, and demonstrates how to check the overall behavior of the pipeline. */ @Category(IntegrationTest.class) -public class ConnectIntegrationTest { - - private static final Logger log = LoggerFactory.getLogger(ConnectIntegrationTest.class); +public class ExampleConnectIntegrationTest { private static final int NUM_RECORDS_PRODUCED = 2000; private static final int NUM_TOPIC_PARTITIONS = 2; @@ -61,37 +62,39 @@ public void close() { } /** - * Simple test case to bring up an embedded Connect cluster along a backing Kafka and Zk process. - * The test will produce and consume records, and start up a sink connector which will consume these records. + * Simple test case to configure and execute an embedded Connect cluster. The test will produce and consume + * records, and start up a sink connector which will consume these records. */ @Test public void testProduceConsumeConnector() throws Exception { // create test topic connect.kafka().createTopic("test-topic", NUM_TOPIC_PARTITIONS, 1, Collections.emptyMap()); - // produce some strings into test topic + // produce some messages into source topic partitions for (int i = 0; i < NUM_RECORDS_PRODUCED; i++) { connect.kafka().produce("test-topic", i % NUM_TOPIC_PARTITIONS, "key", "simple-message-value-" + i); } - // consume all records from test topic or fail - log.info("Consuming records from test topic"); + // consume all records from the source topic or fail, to ensure that they were correctly produced. connect.kafka().consume(NUM_RECORDS_PRODUCED, CONSUME_MAX_DURATION_MS, "test-topic"); + // setup up props for the sink connector Map props = new HashMap<>(); props.put(CONNECTOR_CLASS_CONFIG, "MonitorableSink"); props.put(TASKS_MAX_CONFIG, "2"); - props.put("topics", "test-topic"); - props.put("key.converter", StringConverter.class.getName()); - props.put("value.converter", StringConverter.class.getName()); + props.put(TOPICS_CONFIG, "test-topic"); + props.put(KEY_CONVERTER_CLASS_CONFIG, StringConverter.class.getName()); + props.put(VALUE_CONVERTER_CLASS_CONFIG, StringConverter.class.getName()); props.put(MonitorableSinkConnector.EXPECTED_RECORDS, String.valueOf(NUM_RECORDS_PRODUCED / NUM_TOPIC_PARTITIONS)); + // start a sink connector connect.configureConnector("simple-conn", props); + // wait for the connector tasks to consume desired number of records. MonitorableSinkConnector.taskInstances("simple-conn-0").task().awaitRecords(CONSUME_MAX_DURATION_MS); MonitorableSinkConnector.taskInstances("simple-conn-1").task().awaitRecords(CONSUME_MAX_DURATION_MS); - log.debug("Connector read {} records from topic", NUM_RECORDS_PRODUCED); + // delete connector connect.deleteConnector("simple-conn"); } } From 1cc92a15a2e2968330b5cc261a1ba03adb260d8a Mon Sep 17 00:00:00 2001 From: Arjun Satish Date: Mon, 8 Oct 2018 16:12:02 -0700 Subject: [PATCH 32/52] Clean up handles for tasks before starting the test Signed-off-by: Arjun Satish --- .../ErrorHandlingIntegrationTest.java | 12 +++++++++--- .../ExampleConnectIntegrationTest.java | 4 ++++ .../integration/MonitorableSinkConnector.java | 17 +++++++++++++---- 3 files changed, 26 insertions(+), 7 deletions(-) diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ErrorHandlingIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ErrorHandlingIntegrationTest.java index 89ff98c153db4..342cf2bf1f2b1 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ErrorHandlingIntegrationTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ErrorHandlingIntegrationTest.java @@ -76,6 +76,8 @@ public class ErrorHandlingIntegrationTest { @Before public void setup() throws IOException { + // clean up connector status before starting test. + MonitorableSinkConnector.cleanHandle("simple-conn-0"); connect = new EmbeddedConnectCluster(); connect.start(); } @@ -108,21 +110,25 @@ public void testSkipRetryAndDLQWithHeaders() throws Exception { props.put(TOPICS_CONFIG, "test-topic"); props.put(KEY_CONVERTER_CLASS_CONFIG, StringConverter.class.getName()); props.put(VALUE_CONVERTER_CLASS_CONFIG, StringConverter.class.getName()); + props.put(TRANSFORMS_CONFIG, "failing_transform"); + props.put("transforms.failing_transform.type", FaultyPassthrough.class.getName()); + props.put(MonitorableSinkConnector.EXPECTED_RECORDS, String.valueOf(EXPECTED_CORRECT_RECORDS)); + // log all errors, along with message metadata props.put(ERRORS_LOG_ENABLE_CONFIG, "true"); - props.put(ERRORS_LOG_INCLUDE_MESSAGES_CONFIG, "false"); + props.put(ERRORS_LOG_INCLUDE_MESSAGES_CONFIG, "true"); // produce bad messages into dead letter queue props.put(DLQ_TOPIC_NAME_CONFIG, DLQ_TOPIC); props.put(DLQ_CONTEXT_HEADERS_ENABLE_CONFIG, "true"); props.put(DLQ_TOPIC_REPLICATION_FACTOR_CONFIG, "1"); + // tolerate all erros props.put(ERRORS_TOLERANCE_CONFIG, "all"); - props.put(TRANSFORMS_CONFIG, "failing_transform"); + // retry for up to one second props.put(ERRORS_RETRY_TIMEOUT_CONFIG, "1000"); - props.put("transforms.failing_transform.type", FaultyPassthrough.class.getName()); connect.configureConnector("simple-conn", 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 623cf78455e34..54dc84ccd2742 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 @@ -52,6 +52,10 @@ public class ExampleConnectIntegrationTest { @Before public void setup() throws IOException { + // clean up task status before starting test. + MonitorableSinkConnector.cleanHandle("simple-conn-0"); + MonitorableSinkConnector.cleanHandle("simple-conn-1"); + connect = new EmbeddedConnectCluster(); connect.start(); } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/MonitorableSinkConnector.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/MonitorableSinkConnector.java index 18f3c0d23c1e8..23860edbafff8 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/MonitorableSinkConnector.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/MonitorableSinkConnector.java @@ -35,6 +35,7 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Function; /** * A connector to be used in integration tests. This class provides methods to find task instances @@ -71,7 +72,9 @@ public void task(MonitorableSinkTask task) { public MonitorableSinkTask task() { try { log.debug("Waiting on task {}", taskId); - taskAvailable.await(MAX_WAIT_FOR_TASK_DURATION_MS, TimeUnit.MILLISECONDS); + if (!taskAvailable.await(MAX_WAIT_FOR_TASK_DURATION_MS, TimeUnit.MILLISECONDS)) { + throw new ConnectException("Could not find task '" + taskId + "'."); + } log.debug("Found task!"); } catch (InterruptedException e) { throw new ConnectException("Could not find task for " + taskId, e); @@ -93,6 +96,13 @@ public static Handle taskInstances(String taskId) { return HANDLES.computeIfAbsent(taskId, connName -> new Handle(taskId)); } + public static void cleanHandle(String taskId) { + HANDLES.computeIfPresent(taskId, (k, handle) -> { + handle.taskAvailable.countDown(); + return null; + }); + } + @Override public void start(Map props) { connectorName = props.get("name"); @@ -158,9 +168,8 @@ public void put(Collection records) { @Override public void stop() { - Handle handle = HANDLES.remove(taskId); - handle.taskAvailable.countDown(); - log.info("Removing {} for taskId {}", handle, taskId); + cleanHandle(taskId); + log.info("Removing handle for taskId {}", taskId); } public void awaitRecords(int consumeMaxDurationMs) throws InterruptedException { From af5ccd8a469d9b56bdd338a8aeadae1e9b5cb11b Mon Sep 17 00:00:00 2001 From: Arjun Satish Date: Mon, 8 Oct 2018 16:27:05 -0700 Subject: [PATCH 33/52] Correct some typos Signed-off-by: Arjun Satish --- .../connect/integration/ErrorHandlingIntegrationTest.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ErrorHandlingIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ErrorHandlingIntegrationTest.java index 342cf2bf1f2b1..233971291764c 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ErrorHandlingIntegrationTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ErrorHandlingIntegrationTest.java @@ -165,13 +165,13 @@ public static class FaultyPassthrough> implements Tra static final ConfigDef CONFIG_DEF = new ConfigDef(); /** - * an arbitrary id which causes this transformation to fail with a {@link RetriableException}, but succeeds + * An arbitrary id which causes this transformation to fail with a {@link RetriableException}, but succeeds * on subsequent attempt. */ static final int BAD_RECORD_VAL_RETRIABLE = 4; /** - * an arbitrary id which causes this transformation to fail with a {@link RetriableException}. + * An arbitrary id which causes this transformation to fail with a {@link RetriableException}. */ static final int BAD_RECORD_VAL = 7; @@ -179,7 +179,7 @@ public static class FaultyPassthrough> implements Tra @Override public R apply(R record) { - String badValRetriable = "val-" + BAD_RECORD_VAL_RETRIABLE; + String badValRetriable = "value-" + BAD_RECORD_VAL_RETRIABLE; if (badValRetriable.equals(record.value()) && shouldFail) { shouldFail = false; throw new RetriableException("Error when when value='" + badValRetriable From 56e504f197cf7871ced6699b5f72681227077f72 Mon Sep 17 00:00:00 2001 From: Arjun Satish Date: Thu, 25 Oct 2018 13:44:11 -0700 Subject: [PATCH 34/52] KAFKA-7503: Fix typos and remove unnecessary task props Signed-off-by: Arjun Satish --- .../connect/integration/ErrorHandlingIntegrationTest.java | 6 +++--- .../kafka/connect/integration/MonitorableSinkConnector.java | 1 - 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ErrorHandlingIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ErrorHandlingIntegrationTest.java index 233971291764c..b3005a782df03 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ErrorHandlingIntegrationTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ErrorHandlingIntegrationTest.java @@ -143,7 +143,7 @@ public void testSkipRetryAndDLQWithHeaders() throws Exception { assertTrue(recs.headers().toArray().length > 0); assertValue("test-topic", recs.headers(), ERROR_HEADER_ORIG_TOPIC); assertValue(RetriableException.class.getName(), recs.headers(), ERROR_HEADER_EXCEPTION); - assertValue("Error when when value='value-7'", recs.headers(), ERROR_HEADER_EXCEPTION_MESSAGE); + assertValue("Error when value='value-7'", recs.headers(), ERROR_HEADER_EXCEPTION_MESSAGE); } connect.deleteConnector("simple-conn"); @@ -182,12 +182,12 @@ public R apply(R record) { String badValRetriable = "value-" + BAD_RECORD_VAL_RETRIABLE; if (badValRetriable.equals(record.value()) && shouldFail) { shouldFail = false; - throw new RetriableException("Error when when value='" + badValRetriable + throw new RetriableException("Error when value='" + badValRetriable + "'. A reattempt with this record will succeed."); } String badVal = "value-" + BAD_RECORD_VAL; if (badVal.equals(record.value())) { - throw new RetriableException("Error when when value='" + badVal + "'"); + throw new RetriableException("Error when value='" + badVal + "'"); } return record; } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/MonitorableSinkConnector.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/MonitorableSinkConnector.java index 23860edbafff8..593f1f745037e 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/MonitorableSinkConnector.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/MonitorableSinkConnector.java @@ -121,7 +121,6 @@ public List> taskConfigs(int maxTasks) { for (int i = 0; i < maxTasks; i++) { Map config = new HashMap<>(); config.put("task.id", connectorName + "-" + i); - config.put("required", "dummy-val"); config.put(EXPECTED_RECORDS, expectedRecordsStr); configs.add(config); } From fa68baaf9c141f51412846545e2859c8d3946cca Mon Sep 17 00:00:00 2001 From: Arjun Satish Date: Thu, 25 Oct 2018 14:59:20 -0700 Subject: [PATCH 35/52] KAFKA-7503: Rename taskInstances method Signed-off-by: Arjun Satish --- .../integration/ErrorHandlingIntegrationTest.java | 2 +- .../integration/ExampleConnectIntegrationTest.java | 4 ++-- .../connect/integration/MonitorableSinkConnector.java | 11 +++++++---- 3 files changed, 10 insertions(+), 7 deletions(-) diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ErrorHandlingIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ErrorHandlingIntegrationTest.java index b3005a782df03..81ba9616c4d5d 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ErrorHandlingIntegrationTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ErrorHandlingIntegrationTest.java @@ -132,7 +132,7 @@ public void testSkipRetryAndDLQWithHeaders() throws Exception { connect.configureConnector("simple-conn", props); - MonitorableSinkConnector.taskInstances("simple-conn-0").task().awaitRecords(CONSUME_MAX_DURATION_MS); + MonitorableSinkConnector.task("simple-conn-0").awaitRecords(CONSUME_MAX_DURATION_MS); // consume failed records from dead letter queue topic log.info("Consuming records from test topic"); 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 54dc84ccd2742..3f834c7739b94 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 @@ -95,8 +95,8 @@ public void testProduceConsumeConnector() throws Exception { connect.configureConnector("simple-conn", props); // wait for the connector tasks to consume desired number of records. - MonitorableSinkConnector.taskInstances("simple-conn-0").task().awaitRecords(CONSUME_MAX_DURATION_MS); - MonitorableSinkConnector.taskInstances("simple-conn-1").task().awaitRecords(CONSUME_MAX_DURATION_MS); + MonitorableSinkConnector.task("simple-conn-0").awaitRecords(CONSUME_MAX_DURATION_MS); + MonitorableSinkConnector.task("simple-conn-1").awaitRecords(CONSUME_MAX_DURATION_MS); // delete connector connect.deleteConnector("simple-conn"); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/MonitorableSinkConnector.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/MonitorableSinkConnector.java index 593f1f745037e..6637f7598e1c1 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/MonitorableSinkConnector.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/MonitorableSinkConnector.java @@ -35,7 +35,6 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; -import java.util.function.Function; /** * A connector to be used in integration tests. This class provides methods to find task instances @@ -92,8 +91,12 @@ public String toString() { } } - public static Handle taskInstances(String taskId) { - return HANDLES.computeIfAbsent(taskId, connName -> new Handle(taskId)); + public static MonitorableSinkTask task(String taskId) { + return HANDLES.computeIfAbsent(taskId, connName -> new Handle(taskId)).task(); + } + + public static void task(String taskId, MonitorableSinkTask task) { + HANDLES.computeIfAbsent(taskId, connName -> new Handle(taskId)).task(task); } public static void cleanHandle(String taskId) { @@ -153,7 +156,7 @@ public void start(Map props) { log.debug("Starting task {}", context); taskId = props.get("task.id"); expectedRecords = Integer.parseInt(props.get(EXPECTED_RECORDS)); - taskInstances(taskId).task(this); + task(taskId, this); latch = new CountDownLatch(expectedRecords); } From 3d91b8e018a6acba61c40083365f5bab2f68bf75 Mon Sep 17 00:00:00 2001 From: Arjun Satish Date: Thu, 25 Oct 2018 15:21:55 -0700 Subject: [PATCH 36/52] KAFKA-7503: Add documentation to clarify taskAvailable's contracts Signed-off-by: Arjun Satish --- .../connect/integration/MonitorableSinkConnector.java | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/MonitorableSinkConnector.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/MonitorableSinkConnector.java index 6637f7598e1c1..87737d367e041 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/MonitorableSinkConnector.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/MonitorableSinkConnector.java @@ -56,6 +56,7 @@ public static class Handle { private final String taskId; private final AtomicReference taskRef = new AtomicReference<>(); + // a value of 1 in the latch means that there is no task available. private final CountDownLatch taskAvailable = new CountDownLatch(1); public Handle(String taskId) { @@ -64,6 +65,7 @@ public Handle(String taskId) { public void task(MonitorableSinkTask task) { if (this.taskRef.compareAndSet(null, task)) { + // we have a task, set latch to zero. taskAvailable.countDown(); } } @@ -71,6 +73,8 @@ public void task(MonitorableSinkTask task) { public MonitorableSinkTask task() { try { log.debug("Waiting on task {}", taskId); + // wait for limited duration for a task to be initialized and set to taskRef, + // if we already have a task (i.e., taskAvailable is set to zero), await will immediately return if (!taskAvailable.await(MAX_WAIT_FOR_TASK_DURATION_MS, TimeUnit.MILLISECONDS)) { throw new ConnectException("Could not find task '" + taskId + "'."); } @@ -86,7 +90,7 @@ public MonitorableSinkTask task() { public String toString() { return "Handle{" + "taskId='" + taskId + '\'' + - ", taskAvailable=" + taskAvailable.getCount() + + ", taskAvailable=" + (taskAvailable.getCount() == 0) + '}'; } } @@ -101,7 +105,9 @@ public static void task(String taskId, MonitorableSinkTask task) { public static void cleanHandle(String taskId) { HANDLES.computeIfPresent(taskId, (k, handle) -> { + // unblock any waiting threads handle.taskAvailable.countDown(); + // remove this key from HANDLES return null; }); } From 91acd0550cf0dc638701a7a71050169d88ae02e5 Mon Sep 17 00:00:00 2001 From: Arjun Satish Date: Thu, 25 Oct 2018 16:18:00 -0700 Subject: [PATCH 37/52] KAFKA-7503: Let Jetty pick any available port for integration tests Signed-off-by: Arjun Satish --- .../apache/kafka/connect/runtime/Connect.java | 6 ++++++ .../kafka/connect/runtime/rest/RestServer.java | 4 +++- .../util/clusters/EmbeddedConnectCluster.java | 18 +++++++++++++++--- 3 files changed, 24 insertions(+), 4 deletions(-) diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Connect.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Connect.java index 846ed1a883570..db363c2d71637 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Connect.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Connect.java @@ -20,6 +20,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.net.URI; import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicBoolean; @@ -82,6 +83,11 @@ public void awaitStop() { } } + // Visible for testing + public URI restUrl() { + return rest.serverUrl(); + } + private class ShutdownHook extends Thread { @Override public void run() { diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServer.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServer.java index c0d83f291039e..28b8a06214afe 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServer.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServer.java @@ -220,7 +220,9 @@ public void start(Herder herder) { log.info("REST server listening at " + jettyServer.getURI() + ", advertising URL " + advertisedUrl()); } - + public URI serverUrl() { + return jettyServer.getURI(); + } public void stop() { log.info("Stopping REST server"); 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 b28d3fc0fd936..ef15175700279 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 @@ -19,7 +19,6 @@ import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.kafka.connect.cli.ConnectDistributed; import org.apache.kafka.connect.runtime.Connect; -import org.apache.kafka.connect.runtime.WorkerConfig; import org.apache.kafka.connect.runtime.rest.errors.ConnectRestException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -40,6 +39,7 @@ import static org.apache.kafka.connect.runtime.ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG; import static org.apache.kafka.connect.runtime.WorkerConfig.BOOTSTRAP_SERVERS_CONFIG; import static org.apache.kafka.connect.runtime.WorkerConfig.REST_HOST_NAME_CONFIG; +import static org.apache.kafka.connect.runtime.WorkerConfig.REST_PORT_CONFIG; import static org.apache.kafka.connect.runtime.distributed.DistributedConfig.CONFIG_STORAGE_REPLICATION_FACTOR_CONFIG; import static org.apache.kafka.connect.runtime.distributed.DistributedConfig.CONFIG_TOPIC_CONFIG; import static org.apache.kafka.connect.runtime.distributed.DistributedConfig.OFFSET_STORAGE_REPLICATION_FACTOR_CONFIG; @@ -70,6 +70,11 @@ public EmbeddedConnectCluster() { this(UUID.randomUUID().toString()); } + public EmbeddedConnectCluster(Map workerProps) { + // this empty map will be populated with defaults before starting Connect. + this(UUID.randomUUID().toString(), workerProps); + } + public EmbeddedConnectCluster(String name) { // this empty map will be populated with defaults before starting Connect. this(name, new HashMap<>()); @@ -116,6 +121,7 @@ public void startConnect() { workerProps.put(BOOTSTRAP_SERVERS_CONFIG, kafka().bootstrapServers()); workerProps.put(REST_HOST_NAME_CONFIG, REST_HOST_NAME); + workerProps.put(REST_PORT_CONFIG, "0"); // set this to zero so we pick a random port putIfAbsent(workerProps, GROUP_ID_CONFIG, "connect-integration-test-" + clusterName); putIfAbsent(workerProps, OFFSET_STORAGE_TOPIC_CONFIG, "connect-offset-topic-" + clusterName); @@ -140,7 +146,7 @@ public void startConnect() { * @throws ConnectRestException if REST api returns error status */ public void configureConnector(String connName, Map connConfig) throws IOException { - String url = String.format("http://%s:%s/connectors/%s/config", REST_HOST_NAME, WorkerConfig.REST_PORT_DEFAULT, connName); + String url = endpointForResource(String.format("connectors/%s/config", connName)); ObjectMapper mapper = new ObjectMapper(); int status; try { @@ -162,12 +168,18 @@ public void configureConnector(String connName, Map connConfig) * @throws IOException if call to the REST api fails. */ public void deleteConnector(String connName) throws IOException { - int status = executeDelete(String.format("http://%s:%s/connectors/%s", REST_HOST_NAME, WorkerConfig.REST_PORT_DEFAULT, connName)); + String url = endpointForResource(String.format("connectors/%s", connName)); + int status = executeDelete(url); if (status >= HttpServletResponse.SC_BAD_REQUEST) { throw new ConnectRestException(status, "Could not execute DELETE request."); } } + public String endpointForResource(String resource) { + String url = String.valueOf(connect.restUrl()); + return url + resource; + } + private static void putIfAbsent(Map props, String propertyKey, String propertyValue) { if (!props.containsKey(propertyKey)) { props.put(propertyKey, propertyValue); From 591f8defabac4a17f4d9396712331db03dfe59c1 Mon Sep 17 00:00:00 2001 From: Arjun Satish Date: Sun, 28 Oct 2018 13:35:27 -0700 Subject: [PATCH 38/52] KAFKA-7503: Multi worker connect cluster in integration tests Signed-off-by: Arjun Satish --- .../rest/entities/ConnectorStateInfo.java | 11 +- .../connect/integration/ConnectorHandle.java | 60 +++++++++ .../ErrorHandlingIntegrationTest.java | 20 ++- .../ExampleConnectIntegrationTest.java | 82 +++++++++--- .../integration/MonitorableSinkConnector.java | 115 +++------------- .../connect/integration/RuntimeHandles.java | 63 +++++++++ .../kafka/connect/integration/TaskHandle.java | 118 +++++++++++++++++ .../util/clusters/EmbeddedConnectCluster.java | 123 +++++++++++++----- .../util/clusters/EmbeddedKafkaCluster.java | 11 +- 9 files changed, 438 insertions(+), 165 deletions(-) create mode 100644 connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectorHandle.java create mode 100644 connect/runtime/src/test/java/org/apache/kafka/connect/integration/RuntimeHandles.java create mode 100644 connect/runtime/src/test/java/org/apache/kafka/connect/integration/TaskHandle.java diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConnectorStateInfo.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConnectorStateInfo.java index 80192ca064489..6280473af964d 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConnectorStateInfo.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConnectorStateInfo.java @@ -90,7 +90,10 @@ public String trace() { } public static class ConnectorState extends AbstractState { - public ConnectorState(String state, String worker, String msg) { + @JsonCreator + public ConnectorState(@JsonProperty("state") String state, + @JsonProperty("worker_id") String worker, + @JsonProperty("msg") String msg) { super(state, worker, msg); } } @@ -98,7 +101,11 @@ public ConnectorState(String state, String worker, String msg) { public static class TaskState extends AbstractState implements Comparable { private final int id; - public TaskState(int id, String state, String worker, String msg) { + @JsonCreator + public TaskState(@JsonProperty("id") int id, + @JsonProperty("state") String state, + @JsonProperty("worker_id") String worker, + @JsonProperty("msg") String msg) { super(state, worker, msg); this.id = id; } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectorHandle.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectorHandle.java new file mode 100644 index 0000000000000..cc8e14853f121 --- /dev/null +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectorHandle.java @@ -0,0 +1,60 @@ +/* + * 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.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +/** + * A handle to a connector executing in a Connect cluster. + */ +public class ConnectorHandle { + + private static final Logger log = LoggerFactory.getLogger(ConnectorHandle.class); + private final String connectorName; + + private Map taskHandles = new ConcurrentHashMap<>(); + + public ConnectorHandle(String connectorName) { + this.connectorName = connectorName; + } + + /** + * Get or create a task handle for a given task id. The task need not be created when this method is called. If the + * handle is called before the task is created, the task will bind to the handle once it starts (or restarts). + * + * @param taskId the task id + * @return a non-null {@link TaskHandle} + */ + public TaskHandle taskHandle(String taskId) { + return taskHandles.computeIfAbsent(taskId, k -> new TaskHandle(taskId)); + } + + /** + * Delete the task handle for this task id. + * + * @param taskId the task id. + */ + public void deleteTask(String taskId) { + log.info("Removing handle for {} task in connector {}", taskId, connectorName); + taskHandles.remove(taskId); + } + +} diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ErrorHandlingIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ErrorHandlingIntegrationTest.java index 81ba9616c4d5d..514841afaf9d7 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ErrorHandlingIntegrationTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ErrorHandlingIntegrationTest.java @@ -67,23 +67,29 @@ public class ErrorHandlingIntegrationTest { private static final Logger log = LoggerFactory.getLogger(ErrorHandlingIntegrationTest.class); private static final String DLQ_TOPIC = "my-connector-errors"; + private static final String CONNECTOR_NAME = "error-conn"; + private static final String TASK_ID = "error-conn-0"; private static final int NUM_RECORDS_PRODUCED = 20; private static final int EXPECTED_CORRECT_RECORDS = 19; private static final int EXPECTED_INCORRECT_RECORDS = 1; private static final int CONSUME_MAX_DURATION_MS = 5000; private EmbeddedConnectCluster connect; + private ConnectorHandle connectorHandle; @Before public void setup() throws IOException { // clean up connector status before starting test. - MonitorableSinkConnector.cleanHandle("simple-conn-0"); - connect = new EmbeddedConnectCluster(); + connectorHandle = RuntimeHandles.get().connectorHandle(CONNECTOR_NAME); + connectorHandle.deleteTask(TASK_ID); + connect = new EmbeddedConnectCluster.Builder().build(); connect.start(); } @After public void close() { + connectorHandle.deleteTask(TASK_ID); + RuntimeHandles.get().deleteConnector(CONNECTOR_NAME); connect.stop(); } @@ -113,8 +119,6 @@ public void testSkipRetryAndDLQWithHeaders() throws Exception { props.put(TRANSFORMS_CONFIG, "failing_transform"); props.put("transforms.failing_transform.type", FaultyPassthrough.class.getName()); - props.put(MonitorableSinkConnector.EXPECTED_RECORDS, String.valueOf(EXPECTED_CORRECT_RECORDS)); - // log all errors, along with message metadata props.put(ERRORS_LOG_ENABLE_CONFIG, "true"); props.put(ERRORS_LOG_INCLUDE_MESSAGES_CONFIG, "true"); @@ -130,9 +134,11 @@ public void testSkipRetryAndDLQWithHeaders() throws Exception { // retry for up to one second props.put(ERRORS_RETRY_TIMEOUT_CONFIG, "1000"); - connect.configureConnector("simple-conn", props); + connectorHandle.taskHandle(TASK_ID).expectedRecords(EXPECTED_CORRECT_RECORDS); + + connect.configureConnector("error-conn", props); - MonitorableSinkConnector.task("simple-conn-0").awaitRecords(CONSUME_MAX_DURATION_MS); + connectorHandle.taskHandle(TASK_ID).awaitRecords(CONSUME_MAX_DURATION_MS); // consume failed records from dead letter queue topic log.info("Consuming records from test topic"); @@ -146,7 +152,7 @@ public void testSkipRetryAndDLQWithHeaders() throws Exception { assertValue("Error when value='value-7'", recs.headers(), ERROR_HEADER_EXCEPTION_MESSAGE); } - connect.deleteConnector("simple-conn"); + connect.deleteConnector("error-conn"); } private void assertValue(String expected, Headers headers, String headerKey) { 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 3f834c7739b94..c1737c90ddf7a 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 @@ -25,15 +25,18 @@ import org.junit.experimental.categories.Category; import java.io.IOException; -import java.util.Collections; import java.util.HashMap; import java.util.Map; +import java.util.Properties; import static org.apache.kafka.connect.runtime.ConnectorConfig.CONNECTOR_CLASS_CONFIG; import static org.apache.kafka.connect.runtime.ConnectorConfig.KEY_CONVERTER_CLASS_CONFIG; import static org.apache.kafka.connect.runtime.ConnectorConfig.TASKS_MAX_CONFIG; import static org.apache.kafka.connect.runtime.ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG; import static org.apache.kafka.connect.runtime.SinkConnectorConfig.TOPICS_CONFIG; +import static org.apache.kafka.connect.runtime.WorkerConfig.REST_ADVERTISED_HOST_NAME_CONFIG; +import static org.apache.kafka.connect.runtime.WorkerConfig.REST_ADVERTISED_PORT_CONFIG; +import static org.junit.Assert.assertEquals; /** * An example integration test that demonstrates how to setup an integration test for Connect. @@ -47,21 +50,50 @@ public class ExampleConnectIntegrationTest { private static final int NUM_RECORDS_PRODUCED = 2000; private static final int NUM_TOPIC_PARTITIONS = 2; private static final int CONSUME_MAX_DURATION_MS = 5000; + private static final String CONNECTOR_NAME = "simple-conn"; + private static final String TASK_1_ID = "simple-conn-0"; + private static final String TASK_2_ID = "simple-conn-1"; private EmbeddedConnectCluster connect; + private ConnectorHandle connectorHandle; @Before public void setup() throws IOException { - // clean up task status before starting test. - MonitorableSinkConnector.cleanHandle("simple-conn-0"); - MonitorableSinkConnector.cleanHandle("simple-conn-1"); - - connect = new EmbeddedConnectCluster(); + // setup Connect worker properties + Map exampleWorkerProps = new HashMap<>(); + exampleWorkerProps.put(REST_ADVERTISED_HOST_NAME_CONFIG, "integration.test.host.io"); + exampleWorkerProps.put(REST_ADVERTISED_PORT_CONFIG, "8083"); + + // setup Kafka broker properties + Properties exampleBrokerProps = new Properties(); + exampleBrokerProps.put("auto.create.topics.enable", "false"); + + // build a Connect cluster backed by Kakfa and Zk + connect = new EmbeddedConnectCluster.Builder() + .name("example-cluster") + .numWorkers(1) + .numBrokers(1) + .workerProps(exampleWorkerProps) + .brokerProps(exampleBrokerProps) + .build(); + + // start the clusters connect.start(); + + // get a handle to the connector + connectorHandle = RuntimeHandles.get().connectorHandle(CONNECTOR_NAME); } @After public void close() { + // delete used tasks + connectorHandle.deleteTask(TASK_1_ID); + connectorHandle.deleteTask(TASK_2_ID); + + // delete connector handle + RuntimeHandles.get().deleteConnector(CONNECTOR_NAME); + + // stop all Connect, Kakfa and Zk threads. connect.stop(); } @@ -72,15 +104,7 @@ public void close() { @Test public void testProduceConsumeConnector() throws Exception { // create test topic - connect.kafka().createTopic("test-topic", NUM_TOPIC_PARTITIONS, 1, Collections.emptyMap()); - - // produce some messages into source topic partitions - for (int i = 0; i < NUM_RECORDS_PRODUCED; i++) { - connect.kafka().produce("test-topic", i % NUM_TOPIC_PARTITIONS, "key", "simple-message-value-" + i); - } - - // consume all records from the source topic or fail, to ensure that they were correctly produced. - connect.kafka().consume(NUM_RECORDS_PRODUCED, CONSUME_MAX_DURATION_MS, "test-topic"); + connect.kafka().createTopic("test-topic", NUM_TOPIC_PARTITIONS); // setup up props for the sink connector Map props = new HashMap<>(); @@ -89,16 +113,34 @@ public void testProduceConsumeConnector() throws Exception { props.put(TOPICS_CONFIG, "test-topic"); props.put(KEY_CONVERTER_CLASS_CONFIG, StringConverter.class.getName()); props.put(VALUE_CONVERTER_CLASS_CONFIG, StringConverter.class.getName()); - props.put(MonitorableSinkConnector.EXPECTED_RECORDS, String.valueOf(NUM_RECORDS_PRODUCED / NUM_TOPIC_PARTITIONS)); // start a sink connector - connect.configureConnector("simple-conn", props); + connect.configureConnector(CONNECTOR_NAME, props); + + // expect equal number of records for both tasks + connectorHandle.taskHandle(TASK_1_ID).expectedRecords(NUM_RECORDS_PRODUCED / NUM_TOPIC_PARTITIONS); + connectorHandle.taskHandle(TASK_2_ID).expectedRecords(NUM_RECORDS_PRODUCED / NUM_TOPIC_PARTITIONS); + + // wait for partition assignment + connectorHandle.taskHandle(TASK_1_ID).awaitPartitionAssignment(CONSUME_MAX_DURATION_MS); + connectorHandle.taskHandle(TASK_2_ID).awaitPartitionAssignment(CONSUME_MAX_DURATION_MS); + + // check that the REST API returns two tasks + assertEquals("Incorrect task count in connector", 2, connect.getConnectorStatus(CONNECTOR_NAME).tasks().size()); + + // produce some messages into source topic partitions + for (int i = 0; i < NUM_RECORDS_PRODUCED; i++) { + connect.kafka().produce("test-topic", i % NUM_TOPIC_PARTITIONS, "key", "simple-message-value-" + i); + } + + // consume all records from the source topic or fail, to ensure that they were correctly produced. + connect.kafka().consume(NUM_RECORDS_PRODUCED, CONSUME_MAX_DURATION_MS, "test-topic"); // wait for the connector tasks to consume desired number of records. - MonitorableSinkConnector.task("simple-conn-0").awaitRecords(CONSUME_MAX_DURATION_MS); - MonitorableSinkConnector.task("simple-conn-1").awaitRecords(CONSUME_MAX_DURATION_MS); + connectorHandle.taskHandle(TASK_1_ID).awaitRecords(CONSUME_MAX_DURATION_MS); + connectorHandle.taskHandle(TASK_2_ID).awaitRecords(CONSUME_MAX_DURATION_MS); // delete connector - connect.deleteConnector("simple-conn"); + connect.deleteConnector(CONNECTOR_NAME); } } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/MonitorableSinkConnector.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/MonitorableSinkConnector.java index 87737d367e041..23a8d99e84edc 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/MonitorableSinkConnector.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/MonitorableSinkConnector.java @@ -16,10 +16,9 @@ */ package org.apache.kafka.connect.integration; +import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.connect.connector.Task; -import org.apache.kafka.connect.errors.ConnectException; -import org.apache.kafka.connect.errors.DataException; import org.apache.kafka.connect.runtime.TestSinkConnector; import org.apache.kafka.connect.sink.SinkRecord; import org.apache.kafka.connect.sink.SinkTask; @@ -31,10 +30,6 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicReference; /** * A connector to be used in integration tests. This class provides methods to find task instances @@ -43,79 +38,13 @@ */ public class MonitorableSinkConnector extends TestSinkConnector { - public static final String EXPECTED_RECORDS = "expected_records"; private static final Logger log = LoggerFactory.getLogger(MonitorableSinkConnector.class); - private static final Map HANDLES = new ConcurrentHashMap<>(); - private String connectorName; - private String expectedRecordsStr; - - public static class Handle { - private static final int MAX_WAIT_FOR_TASK_DURATION_MS = 60_000; - - private final String taskId; - private final AtomicReference taskRef = new AtomicReference<>(); - // a value of 1 in the latch means that there is no task available. - private final CountDownLatch taskAvailable = new CountDownLatch(1); - - public Handle(String taskId) { - this.taskId = taskId; - } - - public void task(MonitorableSinkTask task) { - if (this.taskRef.compareAndSet(null, task)) { - // we have a task, set latch to zero. - taskAvailable.countDown(); - } - } - - public MonitorableSinkTask task() { - try { - log.debug("Waiting on task {}", taskId); - // wait for limited duration for a task to be initialized and set to taskRef, - // if we already have a task (i.e., taskAvailable is set to zero), await will immediately return - if (!taskAvailable.await(MAX_WAIT_FOR_TASK_DURATION_MS, TimeUnit.MILLISECONDS)) { - throw new ConnectException("Could not find task '" + taskId + "'."); - } - log.debug("Found task!"); - } catch (InterruptedException e) { - throw new ConnectException("Could not find task for " + taskId, e); - } - - return taskRef.get(); - } - - @Override - public String toString() { - return "Handle{" + - "taskId='" + taskId + '\'' + - ", taskAvailable=" + (taskAvailable.getCount() == 0) + - '}'; - } - } - - public static MonitorableSinkTask task(String taskId) { - return HANDLES.computeIfAbsent(taskId, connName -> new Handle(taskId)).task(); - } - - public static void task(String taskId, MonitorableSinkTask task) { - HANDLES.computeIfAbsent(taskId, connName -> new Handle(taskId)).task(task); - } - - public static void cleanHandle(String taskId) { - HANDLES.computeIfPresent(taskId, (k, handle) -> { - // unblock any waiting threads - handle.taskAvailable.countDown(); - // remove this key from HANDLES - return null; - }); - } @Override public void start(Map props) { connectorName = props.get("name"); - expectedRecordsStr = props.get(EXPECTED_RECORDS); log.info("Starting connector {}", props.get("name")); } @@ -129,8 +58,8 @@ public List> taskConfigs(int maxTasks) { List> configs = new ArrayList<>(); for (int i = 0; i < maxTasks; i++) { Map config = new HashMap<>(); + config.put("connector.name", connectorName); config.put("task.id", connectorName + "-" + i); - config.put(EXPECTED_RECORDS, expectedRecordsStr); configs.add(config); } return configs; @@ -138,7 +67,6 @@ public List> taskConfigs(int maxTasks) { @Override public void stop() { - } @Override @@ -148,9 +76,9 @@ public ConfigDef config() { public static class MonitorableSinkTask extends SinkTask { + private String connectorName; private String taskId; - private int expectedRecords; - private CountDownLatch latch; + private TaskHandle taskHandle; @Override public String version() { @@ -159,40 +87,29 @@ public String version() { @Override public void start(Map props) { - log.debug("Starting task {}", context); taskId = props.get("task.id"); - expectedRecords = Integer.parseInt(props.get(EXPECTED_RECORDS)); - task(taskId, this); - latch = new CountDownLatch(expectedRecords); + connectorName = props.get("connector.name"); + taskHandle = RuntimeHandles.get().connectorHandle(connectorName).taskHandle(taskId); + log.debug("Starting task {}", taskId); + } + + @Override + public void open(Collection partitions) { + log.debug("Opening {} partitions", partitions.size()); + super.open(partitions); + taskHandle.partitionsAssigned(partitions.size()); } @Override public void put(Collection records) { for (SinkRecord rec : records) { - latch.countDown(); - log.debug("Obtained record (key='{}' value='{}') at task {}", rec.key(), rec.value(), context); + taskHandle.record(); + log.trace("Task {} obtained record (key='{}' value='{}')", taskId, rec.key(), rec.value()); } } @Override public void stop() { - cleanHandle(taskId); - log.info("Removing handle for taskId {}", taskId); - } - - public void awaitRecords(int consumeMaxDurationMs) throws InterruptedException { - if (latch == null) { - throw new IllegalStateException("Illegal state encountered. Maybe this task was not started by the framework?"); - } else { - if (!latch.await(consumeMaxDurationMs, TimeUnit.MILLISECONDS)) { - String msg = String.format("Insufficient records seen by task %s in %d millis. Records expected=%d, actual=%d", - taskId, - consumeMaxDurationMs, - expectedRecords, - expectedRecords - latch.getCount()); - throw new DataException(msg); - } - } } } } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/RuntimeHandles.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/RuntimeHandles.java new file mode 100644 index 0000000000000..c9900f3a7fb62 --- /dev/null +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/RuntimeHandles.java @@ -0,0 +1,63 @@ +/* + * 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 java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +/** + * A singleton class which provides a shared class for {@link ConnectorHandle}s and {@link TaskHandle}s that are + * required for integration tests. + */ +public class RuntimeHandles { + + private static final RuntimeHandles INSTANCE = new RuntimeHandles(); + + private final Map connectorHandles = new ConcurrentHashMap<>(); + + private RuntimeHandles() { + } + + /** + * @return the shared {@link RuntimeHandles} instance. + */ + public static RuntimeHandles get() { + return INSTANCE; + } + + /** + * Get or create a connector handle for a given connector name. The connector need not be running at the time + * this method is called. Once the connector is created, it will bind to this handle. Binding happens with the + * connectorName. + * + * @param connectorName the name of the connector + * @return a non-null {@link ConnectorHandle} + */ + public ConnectorHandle connectorHandle(String connectorName) { + return connectorHandles.computeIfAbsent(connectorName, k -> new ConnectorHandle(connectorName)); + } + + /** + * Delete the connector handle for this connector name. + * + * @param connectorName name of the connector + */ + public void deleteConnector(String connectorName) { + connectorHandles.remove(connectorName); + } + +} diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/TaskHandle.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/TaskHandle.java new file mode 100644 index 0000000000000..428d5b1bc4e88 --- /dev/null +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/TaskHandle.java @@ -0,0 +1,118 @@ +/* + * 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.errors.DataException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; + +/** + * A handle to an executing task in a worker. Use this class to record progress, for example: number of records seen + * by the task using so far, or waiting for partitions to be assigned to the task. + */ +public class TaskHandle { + + private static final Logger log = LoggerFactory.getLogger(TaskHandle.class); + + private final String taskId; + private final CountDownLatch expectedPartitionsLatch; + + private CountDownLatch recordsRemainingLatch; + private int expectedRecords = -1; + + public TaskHandle(String taskId) { + this.taskId = taskId; + this.expectedPartitionsLatch = new CountDownLatch(1); + } + + /** + * Decrement the number of records seen by this task. + */ + public void record() { + if (recordsRemainingLatch != null) { + recordsRemainingLatch.countDown(); + } + } + + /** + * Set the number of expected records for this task. + * + * @param expectedRecords number of records + */ + public void expectedRecords(int expectedRecords) { + this.expectedRecords = expectedRecords; + this.recordsRemainingLatch = new CountDownLatch(expectedRecords); + } + + /** + * Set the number of partitions assigned to this task. + * + * @param numPartitions number of partitions + */ + public void partitionsAssigned(int numPartitions) { + for (int i = 0; i < numPartitions; i++) { + expectedPartitionsLatch.countDown(); + } + } + + /** + * Wait for this task to be assigned partitions. + * + * @param consumeMaxDurationMs max duration to wait for partition assignment. + * @throws InterruptedException if another threads interrupts this one while waiting for partitions to be assigned + */ + public void awaitPartitionAssignment(int consumeMaxDurationMs) throws InterruptedException { + if (!expectedPartitionsLatch.await(consumeMaxDurationMs, TimeUnit.MILLISECONDS)) { + String msg = String.format("No partitions were assigned to task %s in %d millis.", + taskId, + consumeMaxDurationMs); + throw new DataException(msg); + } + log.debug("Task {} saw {} records, expected {} records", taskId, expectedRecords - recordsRemainingLatch.getCount(), expectedRecords); + } + + /** + * Wait for this task to receive the expected number of records. + * + * @param consumeMaxDurationMs max duration to wait for records + * @throws InterruptedException if another threads interrupts this one while waiting for records + */ + public void awaitRecords(int consumeMaxDurationMs) throws InterruptedException { + if (recordsRemainingLatch == null) { + throw new IllegalStateException("Illegal state encountered. expectedRecords() was not set for this task?"); + } + if (!recordsRemainingLatch.await(consumeMaxDurationMs, TimeUnit.MILLISECONDS)) { + String msg = String.format("Insufficient records seen by task %s in %d millis. Records expected=%d, actual=%d", + taskId, + consumeMaxDurationMs, + expectedRecords, + expectedRecords - recordsRemainingLatch.getCount()); + throw new DataException(msg); + } + log.debug("Task {} saw {} records, expected {} records", taskId, expectedRecords - recordsRemainingLatch.getCount(), expectedRecords); + } + + @Override + public String toString() { + return "Handle{" + + "taskId='" + taskId + '\'' + + '}'; + } +} 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 ef15175700279..1ed2ae3611747 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 @@ -19,6 +19,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.kafka.connect.cli.ConnectDistributed; import org.apache.kafka.connect.runtime.Connect; +import org.apache.kafka.connect.runtime.rest.entities.ConnectorStateInfo; import org.apache.kafka.connect.runtime.rest.errors.ConnectRestException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -56,38 +57,20 @@ 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; private static final Properties DEFAULT_BROKER_CONFIG = new Properties(); private static final String REST_HOST_NAME = "localhost"; + private final Connect[] connectCluster; private final EmbeddedKafkaCluster kafkaCluster; - private final Map workerProps; - private final String clusterName; - - private Connect connect; - - public EmbeddedConnectCluster() { - this(UUID.randomUUID().toString()); - } - - public EmbeddedConnectCluster(Map workerProps) { - // this empty map will be populated with defaults before starting Connect. - this(UUID.randomUUID().toString(), workerProps); - } + private final String connectClusterName; - public EmbeddedConnectCluster(String name) { - // this empty map will be populated with defaults before starting Connect. - this(name, new HashMap<>()); - } - - public EmbeddedConnectCluster(String name, Map workerProps) { - this(name, workerProps, DEFAULT_NUM_BROKERS, DEFAULT_BROKER_CONFIG); - } - - public EmbeddedConnectCluster(String name, Map workerProps, int numBrokers, Properties brokerProps) { + private EmbeddedConnectCluster(String name, Map workerProps, int numWorkers, int numBrokers, Properties brokerProps) { this.workerProps = workerProps; - this.clusterName = name; - kafkaCluster = new EmbeddedKafkaCluster(numBrokers, brokerProps); + this.connectClusterName = name; + this.kafkaCluster = new EmbeddedKafkaCluster(numBrokers, brokerProps); + this.connectCluster = new Connect[numWorkers]; } /** @@ -103,10 +86,12 @@ public void start() throws IOException { * Clean up any temp directories created locally. */ public void stop() { - try { - connect.stop(); - } catch (Exception e) { - log.error("Could not stop connect", e); + for (Connect worker : this.connectCluster) { + try { + worker.stop(); + } catch (Exception e) { + log.error("Could not stop connect", e); + } } try { @@ -117,23 +102,25 @@ public void stop() { } public void startConnect() { - log.info("Starting Connect cluster with one worker. clusterName=" + clusterName); + log.info("Starting Connect cluster with {} workers. clusterName {}", connectCluster.length, connectClusterName); workerProps.put(BOOTSTRAP_SERVERS_CONFIG, kafka().bootstrapServers()); workerProps.put(REST_HOST_NAME_CONFIG, REST_HOST_NAME); workerProps.put(REST_PORT_CONFIG, "0"); // set this to zero so we pick a random port - putIfAbsent(workerProps, GROUP_ID_CONFIG, "connect-integration-test-" + clusterName); - putIfAbsent(workerProps, OFFSET_STORAGE_TOPIC_CONFIG, "connect-offset-topic-" + clusterName); + putIfAbsent(workerProps, GROUP_ID_CONFIG, "connect-integration-test-" + connectClusterName); + putIfAbsent(workerProps, OFFSET_STORAGE_TOPIC_CONFIG, "connect-offset-topic-" + connectClusterName); putIfAbsent(workerProps, OFFSET_STORAGE_REPLICATION_FACTOR_CONFIG, "1"); - putIfAbsent(workerProps, CONFIG_TOPIC_CONFIG, "connect-config-topic-" + clusterName); + putIfAbsent(workerProps, CONFIG_TOPIC_CONFIG, "connect-config-topic-" + connectClusterName); putIfAbsent(workerProps, CONFIG_STORAGE_REPLICATION_FACTOR_CONFIG, "1"); - putIfAbsent(workerProps, STATUS_STORAGE_TOPIC_CONFIG, "connect-storage-topic-" + clusterName); + putIfAbsent(workerProps, STATUS_STORAGE_TOPIC_CONFIG, "connect-storage-topic-" + connectClusterName); putIfAbsent(workerProps, STATUS_STORAGE_REPLICATION_FACTOR_CONFIG, "1"); putIfAbsent(workerProps, KEY_CONVERTER_CLASS_CONFIG, "org.apache.kafka.connect.storage.StringConverter"); putIfAbsent(workerProps, VALUE_CONVERTER_CLASS_CONFIG, "org.apache.kafka.connect.storage.StringConverter"); - connect = new ConnectDistributed().startConnect(workerProps); + for (int i = 0; i < connectCluster.length; i++) { + connectCluster[i] = new ConnectDistributed().startConnect(workerProps); + } } /** @@ -175,8 +162,19 @@ public void deleteConnector(String connName) throws IOException { } } + public ConnectorStateInfo getConnectorStatus(String connectorName) { + ObjectMapper mapper = new ObjectMapper(); + String url = endpointForResource(String.format("connectors/%s/status", connectorName)); + try { + return mapper.readerFor(ConnectorStateInfo.class).readValue(executeGet(url)); + } catch (IOException e) { + log.error("Could not read connector state", e); + return null; + } + } + public String endpointForResource(String resource) { - String url = String.valueOf(connect.restUrl()); + String url = String.valueOf(connectCluster[0].restUrl()); return url + resource; } @@ -210,6 +208,22 @@ public int executePut(String url, String body) throws IOException { return httpCon.getResponseCode(); } + public String executeGet(String url) throws IOException { + log.debug("Executing GET request to URL={}.", url); + HttpURLConnection httpCon = (HttpURLConnection) new URL(url).openConnection(); + httpCon.setDoOutput(true); + httpCon.setRequestMethod("GET"); + try (InputStream is = httpCon.getInputStream()) { + int c; + StringBuilder response = new StringBuilder(); + while ((c = is.read()) != -1) { + response.append((char) c); + } + log.debug("Get response for URL={} is {}", url, response); + return response.toString(); + } + } + public int executeDelete(String url) throws IOException { log.debug("Executing DELETE request to URL={}", url); HttpURLConnection httpCon = (HttpURLConnection) new URL(url).openConnection(); @@ -219,4 +233,41 @@ public int executeDelete(String url) throws IOException { return httpCon.getResponseCode(); } + public static class Builder { + private String name = UUID.randomUUID().toString(); + private Map workerProps = new HashMap<>(); + private int numWorkers = DEFAULT_NUM_WORKERS; + private int numBrokers = DEFAULT_NUM_BROKERS; + private Properties brokerProps = DEFAULT_BROKER_CONFIG; + + public Builder name(String name) { + this.name = name; + return this; + } + + public Builder workerProps(Map workerProps) { + this.workerProps = workerProps; + return this; + } + + public Builder numWorkers(int numWorkers) { + this.numWorkers = numWorkers; + return this; + } + + public Builder numBrokers(int numBrokers) { + this.numBrokers = numBrokers; + return this; + } + + public Builder brokerProps(Properties brokerProps) { + this.brokerProps = brokerProps; + return this; + } + + public EmbeddedConnectCluster build() { + return new EmbeddedConnectCluster(name, workerProps, numWorkers, numBrokers, brokerProps); + } + } + } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java b/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java index 2b2ffb947f26b..a23e79e9b12c2 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java @@ -192,7 +192,16 @@ public String zKConnectString() { * @param topic The name of the topic. */ public void createTopic(String topic) { - createTopic(topic, 1, 1, new HashMap<>()); + createTopic(topic, 1); + } + + /** + * Create a Kafka topic with given partition and a replication factor of 1. + * + * @param topic The name of the topic. + */ + public void createTopic(String topic, int partitions) { + createTopic(topic, partitions, 1, new HashMap<>()); } /** From 1d86332c2b401d14f069c569daf038543a8e369f Mon Sep 17 00:00:00 2001 From: Arjun Satish Date: Sun, 28 Oct 2018 14:17:35 -0700 Subject: [PATCH 39/52] KAFKA-7503: Add counters in connector handle This is useful when the exact expectations cannot be set for tasks (for the purposes of integration tests, tasks can be a bit unreliable since a rebalance can delay when a task comes up and gets ready to consume records). Signed-off-by: Arjun Satish --- .../connect/integration/ConnectorHandle.java | 47 ++++++++++++++++++- .../ErrorHandlingIntegrationTest.java | 27 ++++++----- .../ExampleConnectIntegrationTest.java | 22 ++++----- .../kafka/connect/integration/TaskHandle.java | 9 ++-- 4 files changed, 75 insertions(+), 30 deletions(-) diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectorHandle.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectorHandle.java index cc8e14853f121..104d6c1e09a27 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectorHandle.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectorHandle.java @@ -16,11 +16,14 @@ */ package org.apache.kafka.connect.integration; +import org.apache.kafka.connect.errors.DataException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; /** * A handle to a connector executing in a Connect cluster. @@ -32,6 +35,9 @@ public class ConnectorHandle { private Map taskHandles = new ConcurrentHashMap<>(); + private CountDownLatch recordsRemainingLatch; + private int expectedRecords = -1; + public ConnectorHandle(String connectorName) { this.connectorName = connectorName; } @@ -44,7 +50,7 @@ public ConnectorHandle(String connectorName) { * @return a non-null {@link TaskHandle} */ public TaskHandle taskHandle(String taskId) { - return taskHandles.computeIfAbsent(taskId, k -> new TaskHandle(taskId)); + return taskHandles.computeIfAbsent(taskId, k -> new TaskHandle(this, taskId)); } /** @@ -57,4 +63,43 @@ public void deleteTask(String taskId) { taskHandles.remove(taskId); } + /** + * Set the number of expected records for this task. + * + * @param expectedRecords number of records + */ + public void expectedRecords(int expectedRecords) { + this.expectedRecords = expectedRecords; + this.recordsRemainingLatch = new CountDownLatch(expectedRecords); + } + + /** + * Record a message arrival at the connector. + */ + public void record() { + if (recordsRemainingLatch != null) { + recordsRemainingLatch.countDown(); + } + } + + /** + * Wait for this task to receive the expected number of records. + * + * @param consumeMaxDurationMs max duration to wait for records + * @throws InterruptedException if another threads interrupts this one while waiting for records + */ + public void awaitRecords(int consumeMaxDurationMs) throws InterruptedException { + if (recordsRemainingLatch == null) { + throw new IllegalStateException("Illegal state encountered. expectedRecords() was not set for this task?"); + } + if (!recordsRemainingLatch.await(consumeMaxDurationMs, TimeUnit.MILLISECONDS)) { + String msg = String.format("Insufficient records seen by connector %s in %d millis. Records expected=%d, actual=%d", + connectorName, + consumeMaxDurationMs, + expectedRecords, + expectedRecords - recordsRemainingLatch.getCount()); + throw new DataException(msg); + } + } + } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ErrorHandlingIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ErrorHandlingIntegrationTest.java index 514841afaf9d7..c3c4b7ab5b519 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ErrorHandlingIntegrationTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ErrorHandlingIntegrationTest.java @@ -98,18 +98,6 @@ public void testSkipRetryAndDLQWithHeaders() throws Exception { // create test topic connect.kafka().createTopic("test-topic"); - // produce some strings into test topic - for (int i = 0; i < NUM_RECORDS_PRODUCED; i++) { - connect.kafka().produce("test-topic", "key-" + i, "value-" + String.valueOf(i)); - } - - // consume all records from test topic - log.info("Consuming records from test topic"); - for (ConsumerRecord recs : connect.kafka().consume(NUM_RECORDS_PRODUCED, CONSUME_MAX_DURATION_MS, "test-topic")) { - log.debug("Consumed record (key='{}', value='{}') from topic {}", - new String(recs.key()), new String(recs.value()), recs.topic()); - } - Map props = new HashMap<>(); props.put(CONNECTOR_CLASS_CONFIG, "MonitorableSink"); props.put(TASKS_MAX_CONFIG, "1"); @@ -138,6 +126,21 @@ public void testSkipRetryAndDLQWithHeaders() throws Exception { connect.configureConnector("error-conn", props); + // wait for partition assignment + connectorHandle.taskHandle(TASK_ID).awaitPartitionAssignment(CONSUME_MAX_DURATION_MS); + + // produce some strings into test topic + for (int i = 0; i < NUM_RECORDS_PRODUCED; i++) { + connect.kafka().produce("test-topic", "key-" + i, "value-" + String.valueOf(i)); + } + + // consume all records from test topic + log.info("Consuming records from test topic"); + for (ConsumerRecord recs : connect.kafka().consume(NUM_RECORDS_PRODUCED, CONSUME_MAX_DURATION_MS, "test-topic")) { + log.debug("Consumed record (key='{}', value='{}') from topic {}", + new String(recs.key()), new String(recs.value()), recs.topic()); + } + connectorHandle.taskHandle(TASK_ID).awaitRecords(CONSUME_MAX_DURATION_MS); // consume failed records from dead letter queue topic 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 c1737c90ddf7a..be1be625ae6b4 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 @@ -36,7 +36,7 @@ import static org.apache.kafka.connect.runtime.SinkConnectorConfig.TOPICS_CONFIG; import static org.apache.kafka.connect.runtime.WorkerConfig.REST_ADVERTISED_HOST_NAME_CONFIG; import static org.apache.kafka.connect.runtime.WorkerConfig.REST_ADVERTISED_PORT_CONFIG; -import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; /** * An example integration test that demonstrates how to setup an integration test for Connect. @@ -71,7 +71,7 @@ public void setup() throws IOException { // build a Connect cluster backed by Kakfa and Zk connect = new EmbeddedConnectCluster.Builder() .name("example-cluster") - .numWorkers(1) + .numWorkers(3) .numBrokers(1) .workerProps(exampleWorkerProps) .brokerProps(exampleBrokerProps) @@ -118,15 +118,7 @@ public void testProduceConsumeConnector() throws Exception { connect.configureConnector(CONNECTOR_NAME, props); // expect equal number of records for both tasks - connectorHandle.taskHandle(TASK_1_ID).expectedRecords(NUM_RECORDS_PRODUCED / NUM_TOPIC_PARTITIONS); - connectorHandle.taskHandle(TASK_2_ID).expectedRecords(NUM_RECORDS_PRODUCED / NUM_TOPIC_PARTITIONS); - - // wait for partition assignment - connectorHandle.taskHandle(TASK_1_ID).awaitPartitionAssignment(CONSUME_MAX_DURATION_MS); - connectorHandle.taskHandle(TASK_2_ID).awaitPartitionAssignment(CONSUME_MAX_DURATION_MS); - - // check that the REST API returns two tasks - assertEquals("Incorrect task count in connector", 2, connect.getConnectorStatus(CONNECTOR_NAME).tasks().size()); + connectorHandle.expectedRecords(NUM_RECORDS_PRODUCED); // produce some messages into source topic partitions for (int i = 0; i < NUM_RECORDS_PRODUCED; i++) { @@ -136,9 +128,11 @@ public void testProduceConsumeConnector() throws Exception { // consume all records from the source topic or fail, to ensure that they were correctly produced. connect.kafka().consume(NUM_RECORDS_PRODUCED, CONSUME_MAX_DURATION_MS, "test-topic"); - // wait for the connector tasks to consume desired number of records. - connectorHandle.taskHandle(TASK_1_ID).awaitRecords(CONSUME_MAX_DURATION_MS); - connectorHandle.taskHandle(TASK_2_ID).awaitRecords(CONSUME_MAX_DURATION_MS); + // wait for the connector tasks to consume all records. + connectorHandle.awaitRecords(CONSUME_MAX_DURATION_MS); + + // at least one task should have initalized and executed + assertTrue("Incorrect task count in connector", connect.getConnectorStatus(CONNECTOR_NAME).tasks().size() > 0); // delete connector connect.deleteConnector(CONNECTOR_NAME); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/TaskHandle.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/TaskHandle.java index 428d5b1bc4e88..82b56000c37b9 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/TaskHandle.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/TaskHandle.java @@ -33,22 +33,25 @@ public class TaskHandle { private final String taskId; private final CountDownLatch expectedPartitionsLatch; + private final ConnectorHandle connectorHandle; private CountDownLatch recordsRemainingLatch; private int expectedRecords = -1; - public TaskHandle(String taskId) { + public TaskHandle(ConnectorHandle connectorHandle, String taskId) { this.taskId = taskId; + this.connectorHandle = connectorHandle; this.expectedPartitionsLatch = new CountDownLatch(1); } /** - * Decrement the number of records seen by this task. + * Record a message arrival at the task. */ public void record() { if (recordsRemainingLatch != null) { recordsRemainingLatch.countDown(); } + connectorHandle.record(); } /** @@ -85,7 +88,7 @@ public void awaitPartitionAssignment(int consumeMaxDurationMs) throws Interrupte consumeMaxDurationMs); throw new DataException(msg); } - log.debug("Task {} saw {} records, expected {} records", taskId, expectedRecords - recordsRemainingLatch.getCount(), expectedRecords); + log.debug("Task {} was assigned partitions", taskId); } /** From dab22a18a4bc89d6cbb318dd0431cfd1fea62d47 Mon Sep 17 00:00:00 2001 From: Arjun Satish Date: Sun, 28 Oct 2018 20:12:08 -0700 Subject: [PATCH 40/52] KAFKA-7503: Remove unnecessary task handle cleanup requests Signed-off-by: Arjun Satish --- .../connect/integration/ErrorHandlingIntegrationTest.java | 8 +++++--- .../integration/ExampleConnectIntegrationTest.java | 8 +------- 2 files changed, 6 insertions(+), 10 deletions(-) diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ErrorHandlingIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ErrorHandlingIntegrationTest.java index c3c4b7ab5b519..ea6cb6def2c26 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ErrorHandlingIntegrationTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ErrorHandlingIntegrationTest.java @@ -79,16 +79,18 @@ public class ErrorHandlingIntegrationTest { @Before public void setup() throws IOException { - // clean up connector status before starting test. + // get connector handles before starting test. connectorHandle = RuntimeHandles.get().connectorHandle(CONNECTOR_NAME); - connectorHandle.deleteTask(TASK_ID); + + // setup Connect cluster with defaults connect = new EmbeddedConnectCluster.Builder().build(); + + // start Connect cluster connect.start(); } @After public void close() { - connectorHandle.deleteTask(TASK_ID); RuntimeHandles.get().deleteConnector(CONNECTOR_NAME); connect.stop(); } 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 be1be625ae6b4..e0b6d24ae86c7 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 @@ -51,8 +51,6 @@ public class ExampleConnectIntegrationTest { private static final int NUM_TOPIC_PARTITIONS = 2; private static final int CONSUME_MAX_DURATION_MS = 5000; private static final String CONNECTOR_NAME = "simple-conn"; - private static final String TASK_1_ID = "simple-conn-0"; - private static final String TASK_2_ID = "simple-conn-1"; private EmbeddedConnectCluster connect; private ConnectorHandle connectorHandle; @@ -86,10 +84,6 @@ public void setup() throws IOException { @After public void close() { - // delete used tasks - connectorHandle.deleteTask(TASK_1_ID); - connectorHandle.deleteTask(TASK_2_ID); - // delete connector handle RuntimeHandles.get().deleteConnector(CONNECTOR_NAME); @@ -117,7 +111,7 @@ public void testProduceConsumeConnector() throws Exception { // start a sink connector connect.configureConnector(CONNECTOR_NAME, props); - // expect equal number of records for both tasks + // expect all records to be consumed by the connector connectorHandle.expectedRecords(NUM_RECORDS_PRODUCED); // produce some messages into source topic partitions From 96d53f54b7f0642880cb2a233c9d7e089ce33c37 Mon Sep 17 00:00:00 2001 From: Arjun Satish Date: Sun, 4 Nov 2018 17:34:33 -0800 Subject: [PATCH 41/52] MINOR: Change example worker props Signed-off-by: Arjun Satish --- .../connect/integration/ExampleConnectIntegrationTest.java | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) 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 e0b6d24ae86c7..7adf8161e340a 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 @@ -34,8 +34,7 @@ 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.WorkerConfig.REST_ADVERTISED_HOST_NAME_CONFIG; -import static org.apache.kafka.connect.runtime.WorkerConfig.REST_ADVERTISED_PORT_CONFIG; +import static org.apache.kafka.connect.runtime.WorkerConfig.OFFSET_COMMIT_INTERVAL_MS_CONFIG; import static org.junit.Assert.assertTrue; /** @@ -59,8 +58,7 @@ public class ExampleConnectIntegrationTest { public void setup() throws IOException { // setup Connect worker properties Map exampleWorkerProps = new HashMap<>(); - exampleWorkerProps.put(REST_ADVERTISED_HOST_NAME_CONFIG, "integration.test.host.io"); - exampleWorkerProps.put(REST_ADVERTISED_PORT_CONFIG, "8083"); + exampleWorkerProps.put(OFFSET_COMMIT_INTERVAL_MS_CONFIG, "30000"); // setup Kafka broker properties Properties exampleBrokerProps = new Properties(); From 63cc56f32119fa1ccf33e7a9bd5dae6743710428 Mon Sep 17 00:00:00 2001 From: Arjun Satish Date: Mon, 5 Nov 2018 12:19:54 -0800 Subject: [PATCH 42/52] MINOR: Use better error message Signed-off-by: Arjun Satish --- .../kafka/connect/util/clusters/EmbeddedConnectCluster.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 1ed2ae3611747..f89931fb31aeb 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 @@ -140,7 +140,7 @@ public void configureConnector(String connName, Map connConfig) String content = mapper.writeValueAsString(connConfig); status = executePut(url, content); } catch (IOException e) { - log.error("Could not serialize config", e); + log.error("Could not execute PUT request to " + url, e); throw e; } if (status >= HttpServletResponse.SC_BAD_REQUEST) { From d4ba51d3dbf7b835b709c6b97803f338349231ba Mon Sep 17 00:00:00 2001 From: Arjun Satish Date: Mon, 5 Nov 2018 13:21:00 -0800 Subject: [PATCH 43/52] TEMP: Modify log4j.properties to debug failing test Signed-off-by: Arjun Satish --- connect/runtime/src/test/resources/log4j.properties | 13 +++++++++++-- 1 file changed, 11 insertions(+), 2 deletions(-) diff --git a/connect/runtime/src/test/resources/log4j.properties b/connect/runtime/src/test/resources/log4j.properties index d5e90fe788f76..42001176493fa 100644 --- a/connect/runtime/src/test/resources/log4j.properties +++ b/connect/runtime/src/test/resources/log4j.properties @@ -14,10 +14,19 @@ # See the License for the specific language governing permissions and # limitations under the License. ## -log4j.rootLogger=OFF, stdout +# Root logger option +log4j.rootLogger=INFO, stdout +# Redirect log messages to console log4j.appender.stdout=org.apache.log4j.ConsoleAppender +log4j.appender.stdout.Target=System.out log4j.appender.stdout.layout=org.apache.log4j.PatternLayout -log4j.appender.stdout.layout.ConversionPattern=[%d] %p %m (%c:%L)%n +log4j.appender.stdout.layout.ConversionPattern=[%d] (%t) %p %m (%c:%L)%n +log4j.logger.org.reflections=ERROR log4j.logger.org.apache.kafka=ERROR +log4j.logger.org.apache.kafka.connect=DEBUG +log4j.logger.org.eclipse.jetty=ERROR +log4j.logger.org.apache.zookeeper=ERROR +log4j.logger.kafka=ERROR +log4j.logger.org.apache.kafka.connect.integration.MonitorableSinkConnector=DEBUG From 3af72cd328f5afcc2696571ce10cd895ee024a38 Mon Sep 17 00:00:00 2001 From: Arjun Satish Date: Mon, 5 Nov 2018 16:12:55 -0800 Subject: [PATCH 44/52] MINOR: Find free ports using ServerSocket API Signed-off-by: Arjun Satish --- .../util/clusters/EmbeddedConnectCluster.java | 33 ++++++++++++++++++- .../src/test/resources/log4j.properties | 10 +----- 2 files changed, 33 insertions(+), 10 deletions(-) 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 f89931fb31aeb..fc98e78d71da9 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 @@ -29,7 +29,9 @@ import java.io.InputStream; import java.io.OutputStreamWriter; import java.net.HttpURLConnection; +import java.net.ServerSocket; import java.net.URL; +import java.util.Arrays; import java.util.HashMap; import java.util.Map; import java.util.Properties; @@ -106,7 +108,6 @@ public void startConnect() { workerProps.put(BOOTSTRAP_SERVERS_CONFIG, kafka().bootstrapServers()); workerProps.put(REST_HOST_NAME_CONFIG, REST_HOST_NAME); - workerProps.put(REST_PORT_CONFIG, "0"); // set this to zero so we pick a random port putIfAbsent(workerProps, GROUP_ID_CONFIG, "connect-integration-test-" + connectClusterName); putIfAbsent(workerProps, OFFSET_STORAGE_TOPIC_CONFIG, "connect-offset-topic-" + connectClusterName); @@ -118,7 +119,13 @@ public void startConnect() { putIfAbsent(workerProps, KEY_CONVERTER_CLASS_CONFIG, "org.apache.kafka.connect.storage.StringConverter"); putIfAbsent(workerProps, VALUE_CONVERTER_CLASS_CONFIG, "org.apache.kafka.connect.storage.StringConverter"); + int[] ports = new int[connectCluster.length]; + for (int i = 0; i < ports.length; i++) { + ports[i] = findFreePort(); + } + log.debug("Found available ports: {}", Arrays.toString(ports)); for (int i = 0; i < connectCluster.length; i++) { + workerProps.put(REST_PORT_CONFIG, String.valueOf(ports[i])); // set this to zero so we pick a random port connectCluster[i] = new ConnectDistributed().startConnect(workerProps); } } @@ -233,6 +240,30 @@ public int executeDelete(String url) throws IOException { return httpCon.getResponseCode(); } + + private int findFreePort() { + ServerSocket socket = null; + try { + socket = new ServerSocket(0); + // make this port available to bind during timeout state after connection is released + socket.setReuseAddress(true); + int port = socket.getLocalPort(); + try { + socket.close(); + } catch (IOException ignored) { + } + return port; + } catch (Exception e) { + if (socket != null) { + try { + socket.close(); + } catch (IOException ignored) { + } + } + } + throw new IllegalStateException("Could not find a free TCP/IP port"); + } + public static class Builder { private String name = UUID.randomUUID().toString(); private Map workerProps = new HashMap<>(); diff --git a/connect/runtime/src/test/resources/log4j.properties b/connect/runtime/src/test/resources/log4j.properties index 42001176493fa..1feedb89721bc 100644 --- a/connect/runtime/src/test/resources/log4j.properties +++ b/connect/runtime/src/test/resources/log4j.properties @@ -14,19 +14,11 @@ # See the License for the specific language governing permissions and # limitations under the License. ## -# Root logger option -log4j.rootLogger=INFO, stdout +log4j.rootLogger=OFF, stdout -# Redirect log messages to console log4j.appender.stdout=org.apache.log4j.ConsoleAppender -log4j.appender.stdout.Target=System.out log4j.appender.stdout.layout=org.apache.log4j.PatternLayout log4j.appender.stdout.layout.ConversionPattern=[%d] (%t) %p %m (%c:%L)%n log4j.logger.org.reflections=ERROR log4j.logger.org.apache.kafka=ERROR -log4j.logger.org.apache.kafka.connect=DEBUG -log4j.logger.org.eclipse.jetty=ERROR -log4j.logger.org.apache.zookeeper=ERROR -log4j.logger.kafka=ERROR -log4j.logger.org.apache.kafka.connect.integration.MonitorableSinkConnector=DEBUG From a9a38957c853f2b46f59b1083f58b25c398e4285 Mon Sep 17 00:00:00 2001 From: Arjun Satish Date: Mon, 12 Nov 2018 20:32:49 -0800 Subject: [PATCH 45/52] KAFKA-7503: Address reviewer comments Signed-off-by: Arjun Satish --- .../connect/integration/ConnectorHandle.java | 6 ++--- .../ErrorHandlingIntegrationTest.java | 13 +++++++---- .../ExampleConnectIntegrationTest.java | 13 ++++++----- .../kafka/connect/integration/TaskHandle.java | 5 ++-- .../util/clusters/EmbeddedConnectCluster.java | 6 +++-- .../util/clusters/EmbeddedKafkaCluster.java | 23 +++++++++++++------ 6 files changed, 42 insertions(+), 24 deletions(-) diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectorHandle.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectorHandle.java index 104d6c1e09a27..d6b47b227b608 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectorHandle.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectorHandle.java @@ -31,9 +31,9 @@ public class ConnectorHandle { private static final Logger log = LoggerFactory.getLogger(ConnectorHandle.class); - private final String connectorName; - private Map taskHandles = new ConcurrentHashMap<>(); + private final String connectorName; + private final Map taskHandles = new ConcurrentHashMap<>(); private CountDownLatch recordsRemainingLatch; private int expectedRecords = -1; @@ -90,7 +90,7 @@ public void record() { */ public void awaitRecords(int consumeMaxDurationMs) throws InterruptedException { if (recordsRemainingLatch == null) { - throw new IllegalStateException("Illegal state encountered. expectedRecords() was not set for this task?"); + throw new IllegalStateException("expectedRecords() was not set for this task?"); } if (!recordsRemainingLatch.await(consumeMaxDurationMs, TimeUnit.MILLISECONDS)) { String msg = String.format("Insufficient records seen by connector %s in %d millis. Records expected=%d, actual=%d", diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ErrorHandlingIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ErrorHandlingIntegrationTest.java index ea6cb6def2c26..3b15e15a5c671 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ErrorHandlingIntegrationTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ErrorHandlingIntegrationTest.java @@ -133,14 +133,19 @@ public void testSkipRetryAndDLQWithHeaders() throws Exception { // produce some strings into test topic for (int i = 0; i < NUM_RECORDS_PRODUCED; i++) { - connect.kafka().produce("test-topic", "key-" + i, "value-" + String.valueOf(i)); + connect.kafka().produce("test-topic", "key-" + i, "value-" + i); } // consume all records from test topic log.info("Consuming records from test topic"); - for (ConsumerRecord recs : connect.kafka().consume(NUM_RECORDS_PRODUCED, CONSUME_MAX_DURATION_MS, "test-topic")) { - log.debug("Consumed record (key='{}', value='{}') from topic {}", - new String(recs.key()), new String(recs.value()), recs.topic()); + int i = 0; + for (ConsumerRecord rec : connect.kafka().consume(NUM_RECORDS_PRODUCED, CONSUME_MAX_DURATION_MS, "test-topic")) { + String k = new String(rec.key()); + String v = new String(rec.value()); + log.debug("Consumed record (key='{}', value='{}') from topic {}", k, v, rec.topic()); + assertEquals("Unexpected key", k, "key-" + i); + assertEquals("Unexpected value", v, "value-" + i); + i++; } connectorHandle.taskHandle(TASK_ID).awaitRecords(CONSUME_MAX_DURATION_MS); 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 7adf8161e340a..3a529fa6fa76c 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 @@ -35,7 +35,7 @@ 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.WorkerConfig.OFFSET_COMMIT_INTERVAL_MS_CONFIG; -import static org.junit.Assert.assertTrue; +import static org.junit.Assert.assertEquals; /** * An example integration test that demonstrates how to setup an integration test for Connect. @@ -106,25 +106,26 @@ public void testProduceConsumeConnector() throws Exception { props.put(KEY_CONVERTER_CLASS_CONFIG, StringConverter.class.getName()); props.put(VALUE_CONVERTER_CLASS_CONFIG, StringConverter.class.getName()); - // start a sink connector - connect.configureConnector(CONNECTOR_NAME, props); - // expect all records to be consumed by the connector connectorHandle.expectedRecords(NUM_RECORDS_PRODUCED); + // start a sink connector + connect.configureConnector(CONNECTOR_NAME, props); + // produce some messages into source topic partitions for (int i = 0; i < NUM_RECORDS_PRODUCED; i++) { connect.kafka().produce("test-topic", i % NUM_TOPIC_PARTITIONS, "key", "simple-message-value-" + i); } // consume all records from the source topic or fail, to ensure that they were correctly produced. - connect.kafka().consume(NUM_RECORDS_PRODUCED, CONSUME_MAX_DURATION_MS, "test-topic"); + assertEquals("Unexpected number of records consumed", NUM_RECORDS_PRODUCED, + connect.kafka().consume(NUM_RECORDS_PRODUCED, CONSUME_MAX_DURATION_MS, "test-topic").count()); // wait for the connector tasks to consume all records. connectorHandle.awaitRecords(CONSUME_MAX_DURATION_MS); // at least one task should have initalized and executed - assertTrue("Incorrect task count in connector", connect.getConnectorStatus(CONNECTOR_NAME).tasks().size() > 0); + assertEquals("Incorrect task count in connector", 2, connect.getConnectorStatus(CONNECTOR_NAME).tasks().size()); // delete connector connect.deleteConnector(CONNECTOR_NAME); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/TaskHandle.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/TaskHandle.java index 82b56000c37b9..2a402ddebeac0 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/TaskHandle.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/TaskHandle.java @@ -70,9 +70,10 @@ public void expectedRecords(int expectedRecords) { * @param numPartitions number of partitions */ public void partitionsAssigned(int numPartitions) { - for (int i = 0; i < numPartitions; i++) { - expectedPartitionsLatch.countDown(); + if (numPartitions > 1) { + throw new IllegalStateException("Expected only one partition. But, assigned " + numPartitions + "."); } + expectedPartitionsLatch.countDown(); } /** 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 fc98e78d71da9..19a8a167ddc27 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 @@ -93,6 +93,7 @@ public void stop() { worker.stop(); } catch (Exception e) { log.error("Could not stop connect", e); + throw new RuntimeException("Could not stop worker", e); } } @@ -100,6 +101,7 @@ public void stop() { kafkaCluster.after(); } catch (Exception e) { log.error("Could not stop kafka", e); + throw new RuntimeException("Could not stop brokers", e); } } @@ -125,7 +127,7 @@ public void startConnect() { } log.debug("Found available ports: {}", Arrays.toString(ports)); for (int i = 0; i < connectCluster.length; i++) { - workerProps.put(REST_PORT_CONFIG, String.valueOf(ports[i])); // set this to zero so we pick a random port + workerProps.put(REST_PORT_CONFIG, String.valueOf(ports[i])); connectCluster[i] = new ConnectDistributed().startConnect(workerProps); } } @@ -180,7 +182,7 @@ public ConnectorStateInfo getConnectorStatus(String connectorName) { } } - public String endpointForResource(String resource) { + private String endpointForResource(String resource) { String url = String.valueOf(connectCluster[0].restUrl()); return url + resource; } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java b/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java index a23e79e9b12c2..159ea5c649cb8 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java @@ -134,13 +134,16 @@ private void stop() { producer.close(); } catch (Exception e) { log.error("Could not shutdown producer ", e); + throw new RuntimeException("Could not shutdown producer", e); } for (KafkaServer broker : brokers) { try { broker.shutdown(); } catch (Throwable t) { - log.error("Could not shutdown broker at {}", address(broker), t); + String msg = String.format("Could not shutdown broker at %s", address(broker)); + log.error(msg, t); + throw new RuntimeException(msg, t); } } @@ -149,14 +152,18 @@ private void stop() { log.info("Cleaning up kafka log dirs at {}", broker.config().logDirs()); CoreUtils.delete(broker.config().logDirs()); } catch (Throwable t) { - log.error("Could not clean up log dirs for broker at {}", address(broker), t); + String msg = String.format("Could not clean up log dirs for broker at %s", address(broker)); + log.error(msg, t); + throw new RuntimeException(msg, t); } } try { zookeeper.shutdown(); } catch (Throwable t) { - log.error("Could not shutdown zookeeper at {}", zKConnectString(), t); + String msg = String.format("Could not shutdown zookeeper at %s", zKConnectString()); + log.error(msg, t); + throw new RuntimeException(msg, t); } } @@ -267,10 +274,12 @@ public ConsumerRecords consume(int n, long maxDuration, String.. int consumedRecords = 0; try (KafkaConsumer consumer = createConsumerAndSubscribeTo(Collections.emptyMap(), topics)) { final long startMillis = System.currentTimeMillis(); - long current = startMillis; - while (current - startMillis < maxDuration) { - ConsumerRecords rec = consumer.poll(Duration.ofMillis(maxDuration)); + long allowedDuration = maxDuration; + while (allowedDuration > 0) { + log.debug("Consuming from {} for {} millis.", Arrays.toString(topics), allowedDuration); + ConsumerRecords rec = consumer.poll(Duration.ofMillis(allowedDuration)); if (rec.isEmpty()) { + allowedDuration = maxDuration - (System.currentTimeMillis() - startMillis); continue; } for (TopicPartition partition: rec.partitions()) { @@ -281,7 +290,7 @@ public ConsumerRecords consume(int n, long maxDuration, String.. if (consumedRecords >= n) { return new ConsumerRecords<>(records); } - current = System.currentTimeMillis(); + allowedDuration = maxDuration - (System.currentTimeMillis() - startMillis); } } From 85d72fc691df28375bad2b5b3c1fd3f9c6685d55 Mon Sep 17 00:00:00 2001 From: Arjun Satish Date: Fri, 16 Nov 2018 09:21:28 -0800 Subject: [PATCH 46/52] KAFKA-7503: Initialize RestServer before starting herder (#8) This is necessary to keep the advertisedUrl consistent with the host:port bound to by the SocketServer when a REST_PORT_CONFIG of 0 is passed into RestServer from integration tests. Signed-off-by: Arjun Satish --- .../kafka/connect/cli/ConnectDistributed.java | 7 ++ .../apache/kafka/connect/runtime/Connect.java | 1 - .../kafka/connect/runtime/HerderProvider.java | 66 +++++++++++++++++++ .../health/ConnectClusterStateImpl.java | 12 ++-- .../connect/runtime/rest/RestServer.java | 25 ++++--- .../resources/ConnectorPluginsResource.java | 12 ++-- .../rest/resources/ConnectorsResource.java | 39 ++++++----- .../runtime/rest/resources/RootResource.java | 8 +-- .../connect/runtime/rest/RestServerTest.java | 3 +- .../ConnectorPluginsResourceTest.java | 3 +- .../resources/ConnectorsResourceTest.java | 3 +- .../rest/resources/RootResourceTest.java | 3 +- 12 files changed, 131 insertions(+), 51 deletions(-) create mode 100644 connect/runtime/src/main/java/org/apache/kafka/connect/runtime/HerderProvider.java diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/cli/ConnectDistributed.java b/connect/runtime/src/main/java/org/apache/kafka/connect/cli/ConnectDistributed.java index 9be27a0bf4d8a..a6c6d98facacc 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/cli/ConnectDistributed.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/cli/ConnectDistributed.java @@ -20,6 +20,7 @@ import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.connect.runtime.Connect; +import org.apache.kafka.connect.runtime.HerderProvider; import org.apache.kafka.connect.runtime.Worker; import org.apache.kafka.connect.runtime.WorkerConfigTransformer; import org.apache.kafka.connect.runtime.WorkerInfo; @@ -94,6 +95,9 @@ public Connect startConnect(Map workerProps) { log.debug("Kafka cluster ID: {}", kafkaClusterId); RestServer rest = new RestServer(config); + HerderProvider provider = new HerderProvider(); + rest.start(provider, plugins); + URI advertisedUrl = rest.advertisedUrl(); String workerId = advertisedUrl.getHost() + ":" + advertisedUrl.getPort(); @@ -115,10 +119,13 @@ public Connect startConnect(Map workerProps) { DistributedHerder herder = new DistributedHerder(config, time, worker, kafkaClusterId, statusBackingStore, configBackingStore, advertisedUrl.toString()); + final Connect connect = new Connect(herder, rest); log.info("Kafka Connect distributed worker initialization took {}ms", time.hiResClockMs() - initStart); try { connect.start(); + // herder has initialized now, and ready to be used by the RestServer. + provider.setHerder(herder); } catch (Exception e) { log.error("Failed to start Connect", e); connect.stop(); diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Connect.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Connect.java index db363c2d71637..965046cccf003 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Connect.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Connect.java @@ -51,7 +51,6 @@ public void start() { Runtime.getRuntime().addShutdownHook(shutdownHook); herder.start(); - rest.start(herder); log.info("Kafka Connect started"); } finally { diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/HerderProvider.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/HerderProvider.java new file mode 100644 index 0000000000000..50de21c65aead --- /dev/null +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/HerderProvider.java @@ -0,0 +1,66 @@ +/* + * 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.connect.errors.ConnectException; + +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; + +/** + * A supplier for {@link Herder}s. + */ +public class HerderProvider { + + private final CountDownLatch initialized = new CountDownLatch(1); + Herder herder = null; + + public HerderProvider() { + } + + /** + * Create a herder provider with a herder. + * @param herder the herder that will be supplied to threads waiting on this provider + */ + public HerderProvider(Herder herder) { + this.herder = herder; + initialized.countDown(); + } + + /** + * @return the contained herder. + * @throws ConnectException if a herder was not available within a duration of calling this method + */ + public Herder get() { + try { + // wait up to 15 seconds + initialized.await(15, TimeUnit.SECONDS); + } catch (InterruptedException e) { + throw new ConnectException("Timed out waiting for herder to be initialized"); + } + return herder; + } + + /** + * @param herder set a herder, and signal to all threads waiting on get(). + */ + public void setHerder(Herder herder) { + initialized.countDown(); + this.herder = herder; + } + +} diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/health/ConnectClusterStateImpl.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/health/ConnectClusterStateImpl.java index a0f7fdeea9fc8..ea93a72d5006e 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/health/ConnectClusterStateImpl.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/health/ConnectClusterStateImpl.java @@ -22,7 +22,7 @@ import org.apache.kafka.connect.health.ConnectorState; import org.apache.kafka.connect.health.ConnectorType; import org.apache.kafka.connect.health.TaskState; -import org.apache.kafka.connect.runtime.Herder; +import org.apache.kafka.connect.runtime.HerderProvider; import org.apache.kafka.connect.runtime.rest.entities.ConnectorStateInfo; import org.apache.kafka.connect.util.Callback; @@ -34,16 +34,16 @@ public class ConnectClusterStateImpl implements ConnectClusterState { - private Herder herder; + private HerderProvider herderProvider; - public ConnectClusterStateImpl(Herder herder) { - this.herder = herder; + public ConnectClusterStateImpl(HerderProvider herderProvider) { + this.herderProvider = herderProvider; } @Override public Collection connectors() { final Collection connectors = new ArrayList<>(); - herder.connectors(new Callback>() { + herderProvider.get().connectors(new Callback>() { @Override public void onCompletion(Throwable error, Collection result) { connectors.addAll(result); @@ -55,7 +55,7 @@ public void onCompletion(Throwable error, Collection result) { @Override public ConnectorHealth connectorHealth(String connName) { - ConnectorStateInfo state = herder.connectorStatus(connName); + ConnectorStateInfo state = herderProvider.get().connectorStatus(connName); ConnectorState connectorState = new ConnectorState( state.connector().state(), state.connector().workerId(), diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServer.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServer.java index 28b8a06214afe..fe662b5a397e2 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServer.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServer.java @@ -17,14 +17,14 @@ package org.apache.kafka.connect.runtime.rest; import com.fasterxml.jackson.jaxrs.json.JacksonJsonProvider; - import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.rest.ConnectRestExtension; import org.apache.kafka.connect.rest.ConnectRestExtensionContext; -import org.apache.kafka.connect.runtime.Herder; +import org.apache.kafka.connect.runtime.HerderProvider; import org.apache.kafka.connect.runtime.WorkerConfig; import org.apache.kafka.connect.runtime.health.ConnectClusterStateImpl; +import org.apache.kafka.connect.runtime.isolation.Plugins; import org.apache.kafka.connect.runtime.rest.errors.ConnectExceptionMapper; import org.apache.kafka.connect.runtime.rest.resources.ConnectorPluginsResource; import org.apache.kafka.connect.runtime.rest.resources.ConnectorsResource; @@ -50,6 +50,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.servlet.DispatcherType; +import javax.ws.rs.core.UriBuilder; import java.io.IOException; import java.net.URI; import java.util.ArrayList; @@ -60,9 +62,6 @@ import java.util.regex.Matcher; import java.util.regex.Pattern; -import javax.servlet.DispatcherType; -import javax.ws.rs.core.UriBuilder; - /** * Embedded server for the REST API that provides the control plane for Kafka Connect workers. */ @@ -161,20 +160,20 @@ public Connector createConnector(String listener) { return connector; } - public void start(Herder herder) { + public void start(HerderProvider herderProvider, Plugins plugins) { log.info("Starting REST server"); ResourceConfig resourceConfig = new ResourceConfig(); resourceConfig.register(new JacksonJsonProvider()); - resourceConfig.register(new RootResource(herder)); - resourceConfig.register(new ConnectorsResource(herder, config)); - resourceConfig.register(new ConnectorPluginsResource(herder)); + resourceConfig.register(new RootResource(herderProvider)); + resourceConfig.register(new ConnectorsResource(herderProvider, config)); + resourceConfig.register(new ConnectorPluginsResource(herderProvider)); resourceConfig.register(ConnectExceptionMapper.class); resourceConfig.property(ServerProperties.WADL_FEATURE_DISABLE, true); - registerRestExtensions(herder, resourceConfig); + registerRestExtensions(herderProvider, plugins, resourceConfig); ServletContainer servletContainer = new ServletContainer(resourceConfig); ServletHolder servletHolder = new ServletHolder(servletContainer); @@ -304,15 +303,15 @@ ServerConnector findConnector(String protocol) { return null; } - void registerRestExtensions(Herder herder, ResourceConfig resourceConfig) { - connectRestExtensions = herder.plugins().newPlugins( + void registerRestExtensions(HerderProvider provider, Plugins plugins, ResourceConfig resourceConfig) { + connectRestExtensions = plugins.newPlugins( config.getList(WorkerConfig.REST_EXTENSION_CLASSES_CONFIG), config, ConnectRestExtension.class); ConnectRestExtensionContext connectRestExtensionContext = new ConnectRestExtensionContextImpl( new ConnectRestConfigurable(resourceConfig), - new ConnectClusterStateImpl(herder) + new ConnectClusterStateImpl(provider) ); for (ConnectRestExtension connectRestExtension : connectRestExtensions) { connectRestExtension.register(connectRestExtensionContext); diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResource.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResource.java index 24eb93b8c0d5f..87f25b29cb51a 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResource.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResource.java @@ -18,7 +18,7 @@ import org.apache.kafka.connect.connector.Connector; import org.apache.kafka.connect.runtime.ConnectorConfig; -import org.apache.kafka.connect.runtime.Herder; +import org.apache.kafka.connect.runtime.HerderProvider; import org.apache.kafka.connect.runtime.isolation.PluginDesc; import org.apache.kafka.connect.runtime.rest.entities.ConfigInfos; import org.apache.kafka.connect.runtime.rest.entities.ConnectorPluginInfo; @@ -49,7 +49,7 @@ public class ConnectorPluginsResource { private static final String ALIAS_SUFFIX = "Connector"; - private final Herder herder; + private final HerderProvider herderProvider; private final List connectorPlugins; private static final List> CONNECTOR_EXCLUDES = Arrays.asList( @@ -58,8 +58,8 @@ public class ConnectorPluginsResource { SchemaSourceConnector.class ); - public ConnectorPluginsResource(Herder herder) { - this.herder = herder; + public ConnectorPluginsResource(HerderProvider herderProvider) { + this.herderProvider = herderProvider; this.connectorPlugins = new ArrayList<>(); } @@ -78,7 +78,7 @@ public ConfigInfos validateConfigs( ); } - return herder.validateConnectorConfig(connectorConfig); + return herderProvider.get().validateConnectorConfig(connectorConfig); } @GET @@ -90,7 +90,7 @@ public List listConnectorPlugins() { // TODO: improve once plugins are allowed to be added/removed during runtime. private synchronized List getConnectorPlugins() { if (connectorPlugins.isEmpty()) { - for (PluginDesc plugin : herder.plugins().connectors()) { + for (PluginDesc plugin : herderProvider.get().plugins().connectors()) { if (!CONNECTOR_EXCLUDES.contains(plugin.pluginClass())) { connectorPlugins.add(new ConnectorPluginInfo(plugin)); } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResource.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResource.java index e9661046d31e9..29a8c39028ef7 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResource.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResource.java @@ -19,6 +19,7 @@ import com.fasterxml.jackson.core.type.TypeReference; import org.apache.kafka.connect.runtime.ConnectorConfig; import org.apache.kafka.connect.runtime.Herder; +import org.apache.kafka.connect.runtime.HerderProvider; import org.apache.kafka.connect.runtime.WorkerConfig; import org.apache.kafka.connect.runtime.distributed.RebalanceNeededException; import org.apache.kafka.connect.runtime.distributed.RequestTargetException; @@ -67,21 +68,25 @@ public class ConnectorsResource { // but currently a worker simply leaving the group can take this long as well. private static final long REQUEST_TIMEOUT_MS = 90 * 1000; - private final Herder herder; + private final HerderProvider herderProvider; private final WorkerConfig config; @javax.ws.rs.core.Context private ServletContext context; - public ConnectorsResource(Herder herder, WorkerConfig config) { - this.herder = herder; + public ConnectorsResource(HerderProvider herder, WorkerConfig config) { + this.herderProvider = herder; this.config = config; } + private Herder herder() { + return herderProvider.get(); + } + @GET @Path("/") public Collection listConnectors(final @QueryParam("forward") Boolean forward) throws Throwable { FutureCallback> cb = new FutureCallback<>(); - herder.connectors(cb); + herder().connectors(cb); return completeOrForwardRequest(cb, "/connectors", "GET", null, new TypeReference>() { }, forward); } @@ -99,7 +104,7 @@ public Response createConnector(final @QueryParam("forward") Boolean forward, checkAndPutConnectorConfigName(name, configs); FutureCallback> cb = new FutureCallback<>(); - herder.putConnectorConfig(name, configs, false, cb); + herder().putConnectorConfig(name, configs, false, cb); Herder.Created info = completeOrForwardRequest(cb, "/connectors", "POST", createRequest, new TypeReference() { }, new CreatedConnectorInfoTranslator(), forward); @@ -112,7 +117,7 @@ public Response createConnector(final @QueryParam("forward") Boolean forward, public ConnectorInfo getConnector(final @PathParam("connector") String connector, final @QueryParam("forward") Boolean forward) throws Throwable { FutureCallback cb = new FutureCallback<>(); - herder.connectorInfo(connector, cb); + herder().connectorInfo(connector, cb); return completeOrForwardRequest(cb, "/connectors/" + connector, "GET", null, forward); } @@ -121,14 +126,14 @@ public ConnectorInfo getConnector(final @PathParam("connector") String connector public Map getConnectorConfig(final @PathParam("connector") String connector, final @QueryParam("forward") Boolean forward) throws Throwable { FutureCallback> cb = new FutureCallback<>(); - herder.connectorConfig(connector, cb); + herder().connectorConfig(connector, cb); return completeOrForwardRequest(cb, "/connectors/" + connector + "/config", "GET", null, forward); } @GET @Path("/{connector}/status") public ConnectorStateInfo getConnectorStatus(final @PathParam("connector") String connector) throws Throwable { - return herder.connectorStatus(connector); + return herder().connectorStatus(connector); } @PUT @@ -139,7 +144,7 @@ public Response putConnectorConfig(final @PathParam("connector") String connecto FutureCallback> cb = new FutureCallback<>(); checkAndPutConnectorConfigName(connector, connectorConfig); - herder.putConnectorConfig(connector, connectorConfig, true, cb); + herder().putConnectorConfig(connector, connectorConfig, true, cb); Herder.Created createdInfo = completeOrForwardRequest(cb, "/connectors/" + connector + "/config", "PUT", connectorConfig, new TypeReference() { }, new CreatedConnectorInfoTranslator(), forward); Response.ResponseBuilder response; @@ -157,21 +162,21 @@ public Response putConnectorConfig(final @PathParam("connector") String connecto public void restartConnector(final @PathParam("connector") String connector, final @QueryParam("forward") Boolean forward) throws Throwable { FutureCallback cb = new FutureCallback<>(); - herder.restartConnector(connector, cb); + herder().restartConnector(connector, cb); completeOrForwardRequest(cb, "/connectors/" + connector + "/restart", "POST", null, forward); } @PUT @Path("/{connector}/pause") public Response pauseConnector(@PathParam("connector") String connector) { - herder.pauseConnector(connector); + herder().pauseConnector(connector); return Response.accepted().build(); } @PUT @Path("/{connector}/resume") public Response resumeConnector(@PathParam("connector") String connector) { - herder.resumeConnector(connector); + herder().resumeConnector(connector); return Response.accepted().build(); } @@ -180,7 +185,7 @@ public Response resumeConnector(@PathParam("connector") String connector) { public List getTaskConfigs(final @PathParam("connector") String connector, final @QueryParam("forward") Boolean forward) throws Throwable { FutureCallback> cb = new FutureCallback<>(); - herder.taskConfigs(connector, cb); + herder().taskConfigs(connector, cb); return completeOrForwardRequest(cb, "/connectors/" + connector + "/tasks", "GET", null, new TypeReference>() { }, forward); } @@ -191,7 +196,7 @@ public void putTaskConfigs(final @PathParam("connector") String connector, final @QueryParam("forward") Boolean forward, final List> taskConfigs) throws Throwable { FutureCallback cb = new FutureCallback<>(); - herder.putTaskConfigs(connector, taskConfigs, cb); + herder().putTaskConfigs(connector, taskConfigs, cb); completeOrForwardRequest(cb, "/connectors/" + connector + "/tasks", "POST", taskConfigs, forward); } @@ -199,7 +204,7 @@ public void putTaskConfigs(final @PathParam("connector") String connector, @Path("/{connector}/tasks/{task}/status") public ConnectorStateInfo.TaskState getTaskStatus(final @PathParam("connector") String connector, final @PathParam("task") Integer task) throws Throwable { - return herder.taskStatus(new ConnectorTaskId(connector, task)); + return herder().taskStatus(new ConnectorTaskId(connector, task)); } @POST @@ -209,7 +214,7 @@ public void restartTask(final @PathParam("connector") String connector, final @QueryParam("forward") Boolean forward) throws Throwable { FutureCallback cb = new FutureCallback<>(); ConnectorTaskId taskId = new ConnectorTaskId(connector, task); - herder.restartTask(taskId, cb); + herder().restartTask(taskId, cb); completeOrForwardRequest(cb, "/connectors/" + connector + "/tasks/" + task + "/restart", "POST", null, forward); } @@ -218,7 +223,7 @@ public void restartTask(final @PathParam("connector") String connector, public void destroyConnector(final @PathParam("connector") String connector, final @QueryParam("forward") Boolean forward) throws Throwable { FutureCallback> cb = new FutureCallback<>(); - herder.deleteConnectorConfig(connector, cb); + herder().deleteConnectorConfig(connector, cb); completeOrForwardRequest(cb, "/connectors/" + connector, "DELETE", null, forward); } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/RootResource.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/RootResource.java index 9666bf15954f9..56516cd410924 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/RootResource.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/RootResource.java @@ -16,7 +16,7 @@ */ package org.apache.kafka.connect.runtime.rest.resources; -import org.apache.kafka.connect.runtime.Herder; +import org.apache.kafka.connect.runtime.HerderProvider; import org.apache.kafka.connect.runtime.rest.entities.ServerInfo; import javax.ws.rs.GET; @@ -28,15 +28,15 @@ @Produces(MediaType.APPLICATION_JSON) public class RootResource { - private final Herder herder; + private final HerderProvider herder; - public RootResource(Herder herder) { + public RootResource(HerderProvider herder) { this.herder = herder; } @GET @Path("/") public ServerInfo serverInfo() { - return new ServerInfo(herder.kafkaClusterId()); + return new ServerInfo(herder.get().kafkaClusterId()); } } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/RestServerTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/RestServerTest.java index 39e35cb01df76..0f55b66fb0a32 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/RestServerTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/RestServerTest.java @@ -19,6 +19,7 @@ import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.connect.rest.ConnectRestExtension; import org.apache.kafka.connect.runtime.Herder; +import org.apache.kafka.connect.runtime.HerderProvider; import org.apache.kafka.connect.runtime.WorkerConfig; import org.apache.kafka.connect.runtime.distributed.DistributedConfig; import org.apache.kafka.connect.runtime.isolation.Plugins; @@ -214,7 +215,7 @@ public void checkCORSRequest(String corsDomain, String origin, String expectedHe server = new RestServer(workerConfig); - server.start(herder); + server.start(new HerderProvider(herder), herder.plugins()); Response response = request("/connectors") .header("Referer", origin + "/page") diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResourceTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResourceTest.java index a3aee6a407d2e..684064d30df4d 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResourceTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResourceTest.java @@ -30,6 +30,7 @@ import org.apache.kafka.connect.runtime.AbstractHerder; import org.apache.kafka.connect.runtime.ConnectorConfig; import org.apache.kafka.connect.runtime.Herder; +import org.apache.kafka.connect.runtime.HerderProvider; import org.apache.kafka.connect.runtime.TestSinkConnector; import org.apache.kafka.connect.runtime.TestSourceConnector; import org.apache.kafka.connect.runtime.WorkerConfig; @@ -186,7 +187,7 @@ public void setUp() throws Exception { plugins = PowerMock.createMock(Plugins.class); herder = PowerMock.createMock(AbstractHerder.class); - connectorPluginsResource = new ConnectorPluginsResource(herder); + connectorPluginsResource = new ConnectorPluginsResource(new HerderProvider(herder)); } private void expectPlugins() { diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResourceTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResourceTest.java index f84cd258fd4ef..5a520744bcd26 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResourceTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResourceTest.java @@ -23,6 +23,7 @@ import org.apache.kafka.connect.errors.NotFoundException; import org.apache.kafka.connect.runtime.ConnectorConfig; import org.apache.kafka.connect.runtime.Herder; +import org.apache.kafka.connect.runtime.HerderProvider; import org.apache.kafka.connect.runtime.WorkerConfig; import org.apache.kafka.connect.runtime.distributed.NotAssignedException; import org.apache.kafka.connect.runtime.distributed.NotLeaderException; @@ -126,7 +127,7 @@ public class ConnectorsResourceTest { public void setUp() throws NoSuchMethodException { PowerMock.mockStatic(RestClient.class, RestClient.class.getMethod("httpRequest", String.class, String.class, Object.class, TypeReference.class, WorkerConfig.class)); - connectorsResource = new ConnectorsResource(herder, null); + connectorsResource = new ConnectorsResource(new HerderProvider(herder), null); } private static final Map getConnectorConfig(Map mapToClone) { diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/RootResourceTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/RootResourceTest.java index 4e928a370372f..be80e28f42da9 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/RootResourceTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/RootResourceTest.java @@ -19,6 +19,7 @@ import org.apache.kafka.clients.admin.MockAdminClient; import org.apache.kafka.common.utils.AppInfoParser; import org.apache.kafka.connect.runtime.Herder; +import org.apache.kafka.connect.runtime.HerderProvider; import org.apache.kafka.connect.runtime.rest.entities.ServerInfo; import org.easymock.EasyMock; import org.easymock.EasyMockRunner; @@ -39,7 +40,7 @@ public class RootResourceTest extends EasyMockSupport { @Before public void setUp() { - rootResource = new RootResource(herder); + rootResource = new RootResource(new HerderProvider(herder)); } @Test From 016da0a7e402857da19401bf78055a6c95193559 Mon Sep 17 00:00:00 2001 From: Arjun Satish Date: Mon, 19 Nov 2018 00:47:54 -0800 Subject: [PATCH 47/52] KAFKA-7503: Set REST_PORT_CONFIG to 0 for integration tests Signed-off-by: Arjun Satish --- .../connect/runtime/rest/RestServer.java | 2 +- .../util/clusters/EmbeddedConnectCluster.java | 33 +------------------ 2 files changed, 2 insertions(+), 33 deletions(-) diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServer.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServer.java index fe662b5a397e2..5cc31cd5cca3c 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServer.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServer.java @@ -265,7 +265,7 @@ else if (serverConnector != null && serverConnector.getHost() != null && serverC Integer advertisedPort = config.getInt(WorkerConfig.REST_ADVERTISED_PORT_CONFIG); if (advertisedPort != null) builder.port(advertisedPort); - else if (serverConnector != null) + else if (serverConnector != null && serverConnector.getPort() > 0) builder.port(serverConnector.getPort()); log.info("Advertised URI: {}", builder.build()); 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 19a8a167ddc27..fd8f45850e2d2 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 @@ -29,9 +29,7 @@ import java.io.InputStream; import java.io.OutputStreamWriter; import java.net.HttpURLConnection; -import java.net.ServerSocket; import java.net.URL; -import java.util.Arrays; import java.util.HashMap; import java.util.Map; import java.util.Properties; @@ -110,6 +108,7 @@ public void startConnect() { workerProps.put(BOOTSTRAP_SERVERS_CONFIG, kafka().bootstrapServers()); workerProps.put(REST_HOST_NAME_CONFIG, REST_HOST_NAME); + workerProps.put(REST_PORT_CONFIG, "0"); // use a random available port putIfAbsent(workerProps, GROUP_ID_CONFIG, "connect-integration-test-" + connectClusterName); putIfAbsent(workerProps, OFFSET_STORAGE_TOPIC_CONFIG, "connect-offset-topic-" + connectClusterName); @@ -121,13 +120,7 @@ public void startConnect() { putIfAbsent(workerProps, KEY_CONVERTER_CLASS_CONFIG, "org.apache.kafka.connect.storage.StringConverter"); putIfAbsent(workerProps, VALUE_CONVERTER_CLASS_CONFIG, "org.apache.kafka.connect.storage.StringConverter"); - int[] ports = new int[connectCluster.length]; - for (int i = 0; i < ports.length; i++) { - ports[i] = findFreePort(); - } - log.debug("Found available ports: {}", Arrays.toString(ports)); for (int i = 0; i < connectCluster.length; i++) { - workerProps.put(REST_PORT_CONFIG, String.valueOf(ports[i])); connectCluster[i] = new ConnectDistributed().startConnect(workerProps); } } @@ -242,30 +235,6 @@ public int executeDelete(String url) throws IOException { return httpCon.getResponseCode(); } - - private int findFreePort() { - ServerSocket socket = null; - try { - socket = new ServerSocket(0); - // make this port available to bind during timeout state after connection is released - socket.setReuseAddress(true); - int port = socket.getLocalPort(); - try { - socket.close(); - } catch (IOException ignored) { - } - return port; - } catch (Exception e) { - if (socket != null) { - try { - socket.close(); - } catch (IOException ignored) { - } - } - } - throw new IllegalStateException("Could not find a free TCP/IP port"); - } - public static class Builder { private String name = UUID.randomUUID().toString(); private Map workerProps = new HashMap<>(); From 6c6b5e46b44888a325817917cd6fffff51dbea43 Mon Sep 17 00:00:00 2001 From: Arjun Satish Date: Mon, 19 Nov 2018 12:31:19 -0800 Subject: [PATCH 48/52] MINOR: Suppress deprecation errors Signed-off-by: Arjun Satish --- .../kafka/connect/util/clusters/EmbeddedConnectCluster.java | 1 + 1 file changed, 1 insertion(+) 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 fd8f45850e2d2..4e820b6907221 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 @@ -103,6 +103,7 @@ public void stop() { } } + @SuppressWarnings("deprecation") public void startConnect() { log.info("Starting Connect cluster with {} workers. clusterName {}", connectCluster.length, connectClusterName); From cc4d11b3e685f4f580084e217097b800edb8f692 Mon Sep 17 00:00:00 2001 From: Arjun Satish Date: Thu, 22 Nov 2018 08:57:28 -1000 Subject: [PATCH 49/52] KAFKA-7503: Replace premature exception with a warning Signed-off-by: Arjun Satish --- .../apache/kafka/connect/integration/TaskHandle.java | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/TaskHandle.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/TaskHandle.java index 2a402ddebeac0..09c53a4c94b94 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/TaskHandle.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/TaskHandle.java @@ -70,10 +70,14 @@ public void expectedRecords(int expectedRecords) { * @param numPartitions number of partitions */ public void partitionsAssigned(int numPartitions) { - if (numPartitions > 1) { - throw new IllegalStateException("Expected only one partition. But, assigned " + numPartitions + "."); + if (numPartitions != 1) { + // this can happen if all tasks have not started, and more partitions are assigned + // to the first task. Subsequent rebalances should allocate the right number of + // partitions to each task. + log.warn("Expected only one partition. But, assigned " + numPartitions + "."); + } else { + expectedPartitionsLatch.countDown(); } - expectedPartitionsLatch.countDown(); } /** From c12fedd78e61a18ce71474214f3d40a293f556bd Mon Sep 17 00:00:00 2001 From: Arjun Satish Date: Tue, 18 Dec 2018 22:43:48 -0800 Subject: [PATCH 50/52] KAFKA-7503: Add waitUntil and address reviewer comments Signed-off-by: Arjun Satish --- .../kafka/connect/runtime/HerderProvider.java | 2 +- .../ConnectIntegrationTestUtils.java | 51 +++++++++++++++++++ .../connect/integration/ConnectorHandle.java | 13 ++++- .../ErrorHandlingIntegrationTest.java | 15 ++++-- .../ExampleConnectIntegrationTest.java | 14 +++-- .../kafka/connect/integration/TaskHandle.java | 29 +++-------- .../util/clusters/EmbeddedConnectCluster.java | 14 +++-- .../util/clusters/EmbeddedKafkaCluster.java | 13 ++++- 8 files changed, 111 insertions(+), 40 deletions(-) create mode 100644 connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTestUtils.java diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/HerderProvider.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/HerderProvider.java index 50de21c65aead..74fade421e017 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/HerderProvider.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/HerderProvider.java @@ -59,8 +59,8 @@ public Herder get() { * @param herder set a herder, and signal to all threads waiting on get(). */ public void setHerder(Herder herder) { - initialized.countDown(); this.herder = herder; + initialized.countDown(); } } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTestUtils.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTestUtils.java new file mode 100644 index 0000000000000..1111f11d7e6fb --- /dev/null +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTestUtils.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.connect.integration; + +import org.apache.kafka.common.errors.TimeoutException; +import org.apache.kafka.connect.errors.ConnectException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.function.Supplier; + +public class ConnectIntegrationTestUtils { + + private static final Logger log = LoggerFactory.getLogger(ConnectIntegrationTestUtils.class); + private static final long CONNECTOR_SETUP_DURATION_MS = 100; + + public static void waitUntil(Supplier condition, long waitForMillis, String errorMessage) { + long deadline = System.currentTimeMillis() + waitForMillis; + long delay = CONNECTOR_SETUP_DURATION_MS / 2; + while (!condition.get()) { + long current = System.currentTimeMillis(); + if (System.currentTimeMillis() > deadline) { + throw new TimeoutException(errorMessage); + } + try { + delay = 2 * delay; + if (current + delay > deadline) { + delay = deadline - current; + } + log.debug("Condition not met, sleeping for {} millis.", delay); + Thread.sleep(delay); + } catch (InterruptedException e) { + throw new ConnectException("Thread was interrupted", e); + } + } + } +} diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectorHandle.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectorHandle.java index d6b47b227b608..e59691b843d09 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectorHandle.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectorHandle.java @@ -20,6 +20,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.Collection; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CountDownLatch; @@ -53,6 +54,10 @@ public TaskHandle taskHandle(String taskId) { return taskHandles.computeIfAbsent(taskId, k -> new TaskHandle(this, taskId)); } + public Collection tasks() { + return taskHandles.values(); + } + /** * Delete the task handle for this task id. * @@ -89,7 +94,7 @@ public void record() { * @throws InterruptedException if another threads interrupts this one while waiting for records */ public void awaitRecords(int consumeMaxDurationMs) throws InterruptedException { - if (recordsRemainingLatch == null) { + if (recordsRemainingLatch == null || expectedRecords < 0) { throw new IllegalStateException("expectedRecords() was not set for this task?"); } if (!recordsRemainingLatch.await(consumeMaxDurationMs, TimeUnit.MILLISECONDS)) { @@ -102,4 +107,10 @@ public void awaitRecords(int consumeMaxDurationMs) throws InterruptedException { } } + @Override + public String toString() { + return "ConnectorHandle{" + + "connectorName='" + connectorName + '\'' + + '}'; + } } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ErrorHandlingIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ErrorHandlingIntegrationTest.java index 3b15e15a5c671..9fed9dcc73da6 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ErrorHandlingIntegrationTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ErrorHandlingIntegrationTest.java @@ -37,6 +37,7 @@ import java.util.HashMap; import java.util.Map; +import static org.apache.kafka.connect.integration.ConnectIntegrationTestUtils.waitUntil; import static org.apache.kafka.connect.runtime.ConnectorConfig.CONNECTOR_CLASS_CONFIG; import static org.apache.kafka.connect.runtime.ConnectorConfig.ERRORS_LOG_ENABLE_CONFIG; import static org.apache.kafka.connect.runtime.ConnectorConfig.ERRORS_LOG_INCLUDE_MESSAGES_CONFIG; @@ -72,6 +73,7 @@ public class ErrorHandlingIntegrationTest { private static final int NUM_RECORDS_PRODUCED = 20; private static final int EXPECTED_CORRECT_RECORDS = 19; private static final int EXPECTED_INCORRECT_RECORDS = 1; + private static final int CONNECTOR_SETUP_DURATION_MS = 5000; private static final int CONSUME_MAX_DURATION_MS = 5000; private EmbeddedConnectCluster connect; @@ -100,6 +102,7 @@ public void testSkipRetryAndDLQWithHeaders() throws Exception { // create test topic connect.kafka().createTopic("test-topic"); + // setup connector config Map props = new HashMap<>(); props.put(CONNECTOR_CLASS_CONFIG, "MonitorableSink"); props.put(TASKS_MAX_CONFIG, "1"); @@ -124,12 +127,15 @@ public void testSkipRetryAndDLQWithHeaders() throws Exception { // retry for up to one second props.put(ERRORS_RETRY_TIMEOUT_CONFIG, "1000"); + // set expected records to successfully reach the task connectorHandle.taskHandle(TASK_ID).expectedRecords(EXPECTED_CORRECT_RECORDS); - connect.configureConnector("error-conn", props); + connect.configureConnector(CONNECTOR_NAME, props); - // wait for partition assignment - connectorHandle.taskHandle(TASK_ID).awaitPartitionAssignment(CONSUME_MAX_DURATION_MS); + waitUntil(() -> connect.connectorStatus(CONNECTOR_NAME).tasks().size() == 1 + && connectorHandle.taskHandle(TASK_ID).partitionsAssigned() == 1, + CONNECTOR_SETUP_DURATION_MS, + "Timed out waiting for connector task to be assigned a partition."); // produce some strings into test topic for (int i = 0; i < NUM_RECORDS_PRODUCED; i++) { @@ -148,6 +154,7 @@ public void testSkipRetryAndDLQWithHeaders() throws Exception { i++; } + // wait for records to reach the task connectorHandle.taskHandle(TASK_ID).awaitRecords(CONSUME_MAX_DURATION_MS); // consume failed records from dead letter queue topic @@ -162,7 +169,7 @@ public void testSkipRetryAndDLQWithHeaders() throws Exception { assertValue("Error when value='value-7'", recs.headers(), ERROR_HEADER_EXCEPTION_MESSAGE); } - connect.deleteConnector("error-conn"); + connect.deleteConnector(CONNECTOR_NAME); } private void assertValue(String expected, Headers headers, String headerKey) { 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 3a529fa6fa76c..8bebfae89147d 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 @@ -29,6 +29,7 @@ import java.util.Map; import java.util.Properties; +import static org.apache.kafka.connect.integration.ConnectIntegrationTestUtils.waitUntil; 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; @@ -47,8 +48,9 @@ public class ExampleConnectIntegrationTest { private static final int NUM_RECORDS_PRODUCED = 2000; - private static final int NUM_TOPIC_PARTITIONS = 2; + private static final int NUM_TOPIC_PARTITIONS = 3; private static final int CONSUME_MAX_DURATION_MS = 5000; + private static final int CONNECTOR_SETUP_DURATION_MS = 15000; private static final String CONNECTOR_NAME = "simple-conn"; private EmbeddedConnectCluster connect; @@ -101,7 +103,7 @@ public void testProduceConsumeConnector() throws Exception { // setup up props for the sink connector Map props = new HashMap<>(); props.put(CONNECTOR_CLASS_CONFIG, "MonitorableSink"); - props.put(TASKS_MAX_CONFIG, "2"); + props.put(TASKS_MAX_CONFIG, "3"); props.put(TOPICS_CONFIG, "test-topic"); props.put(KEY_CONVERTER_CLASS_CONFIG, StringConverter.class.getName()); props.put(VALUE_CONVERTER_CLASS_CONFIG, StringConverter.class.getName()); @@ -112,6 +114,11 @@ public void testProduceConsumeConnector() throws Exception { // start a sink connector connect.configureConnector(CONNECTOR_NAME, props); + waitUntil(() -> connect.connectorStatus(CONNECTOR_NAME).tasks().size() == 3 + && connectorHandle.tasks().stream().allMatch(th -> th.partitionsAssigned() == 1), + CONNECTOR_SETUP_DURATION_MS, + "Timed out waiting for connector tasks to be assigned a partition each."); + // produce some messages into source topic partitions for (int i = 0; i < NUM_RECORDS_PRODUCED; i++) { connect.kafka().produce("test-topic", i % NUM_TOPIC_PARTITIONS, "key", "simple-message-value-" + i); @@ -124,9 +131,6 @@ public void testProduceConsumeConnector() throws Exception { // wait for the connector tasks to consume all records. connectorHandle.awaitRecords(CONSUME_MAX_DURATION_MS); - // at least one task should have initalized and executed - assertEquals("Incorrect task count in connector", 2, connect.getConnectorStatus(CONNECTOR_NAME).tasks().size()); - // delete connector connect.deleteConnector(CONNECTOR_NAME); } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/TaskHandle.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/TaskHandle.java index 09c53a4c94b94..de3d9240d1be7 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/TaskHandle.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/TaskHandle.java @@ -22,6 +22,7 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; /** * A handle to an executing task in a worker. Use this class to record progress, for example: number of records seen @@ -32,16 +33,16 @@ public class TaskHandle { private static final Logger log = LoggerFactory.getLogger(TaskHandle.class); private final String taskId; - private final CountDownLatch expectedPartitionsLatch; private final ConnectorHandle connectorHandle; + private final AtomicInteger partitionsAssigned = new AtomicInteger(0); private CountDownLatch recordsRemainingLatch; private int expectedRecords = -1; public TaskHandle(ConnectorHandle connectorHandle, String taskId) { + log.info("Created task {} for connector {}", taskId, connectorHandle); this.taskId = taskId; this.connectorHandle = connectorHandle; - this.expectedPartitionsLatch = new CountDownLatch(1); } /** @@ -70,30 +71,14 @@ public void expectedRecords(int expectedRecords) { * @param numPartitions number of partitions */ public void partitionsAssigned(int numPartitions) { - if (numPartitions != 1) { - // this can happen if all tasks have not started, and more partitions are assigned - // to the first task. Subsequent rebalances should allocate the right number of - // partitions to each task. - log.warn("Expected only one partition. But, assigned " + numPartitions + "."); - } else { - expectedPartitionsLatch.countDown(); - } + partitionsAssigned.set(numPartitions); } /** - * Wait for this task to be assigned partitions. - * - * @param consumeMaxDurationMs max duration to wait for partition assignment. - * @throws InterruptedException if another threads interrupts this one while waiting for partitions to be assigned + * @return the number of topic partitions assigned to this task. */ - public void awaitPartitionAssignment(int consumeMaxDurationMs) throws InterruptedException { - if (!expectedPartitionsLatch.await(consumeMaxDurationMs, TimeUnit.MILLISECONDS)) { - String msg = String.format("No partitions were assigned to task %s in %d millis.", - taskId, - consumeMaxDurationMs); - throw new DataException(msg); - } - log.debug("Task {} was assigned partitions", taskId); + public int partitionsAssigned() { + return partitionsAssigned.get(); } /** 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 4e820b6907221..9ba0e06bfd013 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 @@ -18,6 +18,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.kafka.connect.cli.ConnectDistributed; +import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.runtime.Connect; import org.apache.kafka.connect.runtime.rest.entities.ConnectorStateInfo; import org.apache.kafka.connect.runtime.rest.errors.ConnectRestException; @@ -65,10 +66,12 @@ public class EmbeddedConnectCluster { private final EmbeddedKafkaCluster kafkaCluster; private final Map workerProps; private final String connectClusterName; + private final int numBrokers; private EmbeddedConnectCluster(String name, Map workerProps, int numWorkers, int numBrokers, Properties brokerProps) { this.workerProps = workerProps; this.connectClusterName = name; + this.numBrokers = numBrokers; this.kafkaCluster = new EmbeddedKafkaCluster(numBrokers, brokerProps); this.connectCluster = new Connect[numWorkers]; } @@ -111,13 +114,14 @@ public void startConnect() { workerProps.put(REST_HOST_NAME_CONFIG, REST_HOST_NAME); workerProps.put(REST_PORT_CONFIG, "0"); // use a random available port + String internalTopicsReplFactor = String.valueOf(numBrokers); putIfAbsent(workerProps, GROUP_ID_CONFIG, "connect-integration-test-" + connectClusterName); putIfAbsent(workerProps, OFFSET_STORAGE_TOPIC_CONFIG, "connect-offset-topic-" + connectClusterName); - putIfAbsent(workerProps, OFFSET_STORAGE_REPLICATION_FACTOR_CONFIG, "1"); + putIfAbsent(workerProps, OFFSET_STORAGE_REPLICATION_FACTOR_CONFIG, internalTopicsReplFactor); putIfAbsent(workerProps, CONFIG_TOPIC_CONFIG, "connect-config-topic-" + connectClusterName); - putIfAbsent(workerProps, CONFIG_STORAGE_REPLICATION_FACTOR_CONFIG, "1"); + putIfAbsent(workerProps, CONFIG_STORAGE_REPLICATION_FACTOR_CONFIG, internalTopicsReplFactor); putIfAbsent(workerProps, STATUS_STORAGE_TOPIC_CONFIG, "connect-storage-topic-" + connectClusterName); - putIfAbsent(workerProps, STATUS_STORAGE_REPLICATION_FACTOR_CONFIG, "1"); + putIfAbsent(workerProps, STATUS_STORAGE_REPLICATION_FACTOR_CONFIG, internalTopicsReplFactor); putIfAbsent(workerProps, KEY_CONVERTER_CLASS_CONFIG, "org.apache.kafka.connect.storage.StringConverter"); putIfAbsent(workerProps, VALUE_CONVERTER_CLASS_CONFIG, "org.apache.kafka.connect.storage.StringConverter"); @@ -165,14 +169,14 @@ public void deleteConnector(String connName) throws IOException { } } - public ConnectorStateInfo getConnectorStatus(String connectorName) { + public ConnectorStateInfo connectorStatus(String connectorName) { ObjectMapper mapper = new ObjectMapper(); String url = endpointForResource(String.format("connectors/%s/status", connectorName)); try { return mapper.readerFor(ConnectorStateInfo.class).readValue(executeGet(url)); } catch (IOException e) { log.error("Could not read connector state", e); - return null; + throw new ConnectException("Could not read connector state", e); } } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java b/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java index 159ea5c649cb8..109ba14a0ddb2 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/util/clusters/EmbeddedKafkaCluster.java @@ -32,6 +32,7 @@ import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.config.SslConfigs; import org.apache.kafka.common.config.types.Password; @@ -56,6 +57,7 @@ import java.util.Properties; import java.util.UUID; import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; import static org.apache.kafka.clients.consumer.ConsumerConfig.AUTO_OFFSET_RESET_CONFIG; @@ -73,6 +75,8 @@ public class EmbeddedKafkaCluster extends ExternalResource { private static final Logger log = LoggerFactory.getLogger(EmbeddedKafkaCluster.class); + private static final long DEFAULT_PRODUCE_SEND_DURATION_MS = 500; + // Kafka Config private final KafkaServer[] brokers; private final Properties brokerConfig; @@ -107,7 +111,7 @@ private void start() throws IOException { putIfAbsent(brokerConfig, KafkaConfig$.MODULE$.HostNameProp(), "localhost"); putIfAbsent(brokerConfig, KafkaConfig$.MODULE$.DeleteTopicEnableProp(), true); putIfAbsent(brokerConfig, KafkaConfig$.MODULE$.GroupInitialRebalanceDelayMsProp(), 0); - putIfAbsent(brokerConfig, KafkaConfig$.MODULE$.OffsetsTopicReplicationFactorProp(), (short) 1); + putIfAbsent(brokerConfig, KafkaConfig$.MODULE$.OffsetsTopicReplicationFactorProp(), (short) brokers.length); putIfAbsent(brokerConfig, KafkaConfig$.MODULE$.AutoCreateTopicsEnableProp(), false); Object listenerConfig = brokerConfig.get(KafkaConfig$.MODULE$.InterBrokerListenerNameProp()); @@ -246,7 +250,12 @@ public void produce(String topic, String key, String value) { } public void produce(String topic, Integer partition, String key, String value) { - producer.send(new ProducerRecord<>(topic, partition, key == null ? null : key.getBytes(), value == null ? null : value.getBytes())); + ProducerRecord msg = new ProducerRecord<>(topic, partition, key == null ? null : key.getBytes(), value == null ? null : value.getBytes()); + try { + producer.send(msg).get(DEFAULT_PRODUCE_SEND_DURATION_MS, TimeUnit.MILLISECONDS); + } catch (Exception e) { + throw new KafkaException("Could not produce message to topic=" + topic, e); + } } public AdminClient createAdminClient() { From 8b24c6fbf2806b6e3c6d764f212e392c8e204fff Mon Sep 17 00:00:00 2001 From: Arjun Satish Date: Sat, 12 Jan 2019 13:35:49 -0800 Subject: [PATCH 51/52] KAFKA-7503: Use waitForCondition from TestUtils; throw error if Latch doesn't count down Signed-off-by: Arjun Satish --- .../kafka/connect/runtime/HerderProvider.java | 10 ++-- .../ConnectIntegrationTestUtils.java | 51 ------------------- .../ErrorHandlingIntegrationTest.java | 12 ++--- .../ExampleConnectIntegrationTest.java | 6 +-- 4 files changed, 15 insertions(+), 64 deletions(-) delete mode 100644 connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTestUtils.java diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/HerderProvider.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/HerderProvider.java index 74fade421e017..42c0925a704a2 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/HerderProvider.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/HerderProvider.java @@ -27,7 +27,7 @@ public class HerderProvider { private final CountDownLatch initialized = new CountDownLatch(1); - Herder herder = null; + volatile Herder herder = null; public HerderProvider() { } @@ -47,10 +47,12 @@ public HerderProvider(Herder herder) { */ public Herder get() { try { - // wait up to 15 seconds - initialized.await(15, TimeUnit.SECONDS); + // wait for herder to be initialized + if (!initialized.await(1, TimeUnit.MINUTES)) { + throw new ConnectException("Timed out waiting for herder to be initialized."); + } } catch (InterruptedException e) { - throw new ConnectException("Timed out waiting for herder to be initialized"); + throw new ConnectException("Interrupted while waiting for herder to be initialized.", e); } return herder; } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTestUtils.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTestUtils.java deleted file mode 100644 index 1111f11d7e6fb..0000000000000 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ConnectIntegrationTestUtils.java +++ /dev/null @@ -1,51 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.kafka.connect.integration; - -import org.apache.kafka.common.errors.TimeoutException; -import org.apache.kafka.connect.errors.ConnectException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.function.Supplier; - -public class ConnectIntegrationTestUtils { - - private static final Logger log = LoggerFactory.getLogger(ConnectIntegrationTestUtils.class); - private static final long CONNECTOR_SETUP_DURATION_MS = 100; - - public static void waitUntil(Supplier condition, long waitForMillis, String errorMessage) { - long deadline = System.currentTimeMillis() + waitForMillis; - long delay = CONNECTOR_SETUP_DURATION_MS / 2; - while (!condition.get()) { - long current = System.currentTimeMillis(); - if (System.currentTimeMillis() > deadline) { - throw new TimeoutException(errorMessage); - } - try { - delay = 2 * delay; - if (current + delay > deadline) { - delay = deadline - current; - } - log.debug("Condition not met, sleeping for {} millis.", delay); - Thread.sleep(delay); - } catch (InterruptedException e) { - throw new ConnectException("Thread was interrupted", e); - } - } - } -} diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ErrorHandlingIntegrationTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ErrorHandlingIntegrationTest.java index 9fed9dcc73da6..af3ab4421a363 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ErrorHandlingIntegrationTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/integration/ErrorHandlingIntegrationTest.java @@ -37,7 +37,6 @@ import java.util.HashMap; import java.util.Map; -import static org.apache.kafka.connect.integration.ConnectIntegrationTestUtils.waitUntil; import static org.apache.kafka.connect.runtime.ConnectorConfig.CONNECTOR_CLASS_CONFIG; import static org.apache.kafka.connect.runtime.ConnectorConfig.ERRORS_LOG_ENABLE_CONFIG; import static org.apache.kafka.connect.runtime.ConnectorConfig.ERRORS_LOG_INCLUDE_MESSAGES_CONFIG; @@ -54,6 +53,7 @@ import static org.apache.kafka.connect.runtime.errors.DeadLetterQueueReporter.ERROR_HEADER_EXCEPTION; import static org.apache.kafka.connect.runtime.errors.DeadLetterQueueReporter.ERROR_HEADER_EXCEPTION_MESSAGE; import static org.apache.kafka.connect.runtime.errors.DeadLetterQueueReporter.ERROR_HEADER_ORIG_TOPIC; +import static org.apache.kafka.test.TestUtils.waitForCondition; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -81,14 +81,14 @@ public class ErrorHandlingIntegrationTest { @Before public void setup() throws IOException { - // get connector handles before starting test. - connectorHandle = RuntimeHandles.get().connectorHandle(CONNECTOR_NAME); - // setup Connect cluster with defaults connect = new EmbeddedConnectCluster.Builder().build(); // start Connect cluster connect.start(); + + // get connector handles before starting test. + connectorHandle = RuntimeHandles.get().connectorHandle(CONNECTOR_NAME); } @After @@ -132,10 +132,10 @@ public void testSkipRetryAndDLQWithHeaders() throws Exception { connect.configureConnector(CONNECTOR_NAME, props); - waitUntil(() -> connect.connectorStatus(CONNECTOR_NAME).tasks().size() == 1 + waitForCondition(() -> connect.connectorStatus(CONNECTOR_NAME).tasks().size() == 1 && connectorHandle.taskHandle(TASK_ID).partitionsAssigned() == 1, CONNECTOR_SETUP_DURATION_MS, - "Timed out waiting for connector task to be assigned a partition."); + "Connector task was not assigned a partition."); // produce some strings into test topic for (int i = 0; i < NUM_RECORDS_PRODUCED; i++) { 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 8bebfae89147d..5d887cf4cbf89 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 @@ -29,13 +29,13 @@ import java.util.Map; import java.util.Properties; -import static org.apache.kafka.connect.integration.ConnectIntegrationTestUtils.waitUntil; import static org.apache.kafka.connect.runtime.ConnectorConfig.CONNECTOR_CLASS_CONFIG; import static org.apache.kafka.connect.runtime.ConnectorConfig.KEY_CONVERTER_CLASS_CONFIG; import static org.apache.kafka.connect.runtime.ConnectorConfig.TASKS_MAX_CONFIG; import static org.apache.kafka.connect.runtime.ConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG; import static org.apache.kafka.connect.runtime.SinkConnectorConfig.TOPICS_CONFIG; import static org.apache.kafka.connect.runtime.WorkerConfig.OFFSET_COMMIT_INTERVAL_MS_CONFIG; +import static org.apache.kafka.test.TestUtils.waitForCondition; import static org.junit.Assert.assertEquals; /** @@ -114,10 +114,10 @@ public void testProduceConsumeConnector() throws Exception { // start a sink connector connect.configureConnector(CONNECTOR_NAME, props); - waitUntil(() -> connect.connectorStatus(CONNECTOR_NAME).tasks().size() == 3 + waitForCondition(() -> connect.connectorStatus(CONNECTOR_NAME).tasks().size() == 3 && connectorHandle.tasks().stream().allMatch(th -> th.partitionsAssigned() == 1), CONNECTOR_SETUP_DURATION_MS, - "Timed out waiting for connector tasks to be assigned a partition each."); + "Connector tasks were not assigned a partition each."); // produce some messages into source topic partitions for (int i = 0; i < NUM_RECORDS_PRODUCED; i++) { From 027e68ee156e015bf3b38d14d529b5780009e221 Mon Sep 17 00:00:00 2001 From: Arjun Satish Date: Sat, 12 Jan 2019 13:55:08 -0800 Subject: [PATCH 52/52] KAFKA-7503: Fix call to RestServer#start in unit test Signed-off-by: Arjun Satish --- .../org/apache/kafka/connect/runtime/rest/RestServerTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/RestServerTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/RestServerTest.java index 0f55b66fb0a32..d5802cb5222d2 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/RestServerTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/RestServerTest.java @@ -178,7 +178,7 @@ public void testOptionsDoesNotIncludeWadlOutput() { PowerMock.replayAll(); server = new RestServer(workerConfig); - server.start(herder); + server.start(new HerderProvider(herder), herder.plugins()); Response response = request("/connectors") .accept(MediaType.WILDCARD)