mirror of https://github.com/apache/kafka.git
				
				
				
			MINOR: Small Connect integration test fixes (#8100)
Author: Konstantine Karantasis <konstantine@confluent.io> Reviewer: Randall Hauch <rhauch@gmail.com>
This commit is contained in:
		
							parent
							
								
									f51e1e6548
								
							
						
					
					
						commit
						97d2c726f1
					
				|  | @ -16,17 +16,13 @@ | ||||||
|  */ |  */ | ||||||
| package org.apache.kafka.connect.mirror; | package org.apache.kafka.connect.mirror; | ||||||
| 
 | 
 | ||||||
| import org.apache.kafka.connect.runtime.AbstractStatus; | import org.apache.kafka.clients.admin.Admin; | ||||||
| import org.apache.kafka.connect.runtime.rest.entities.ConnectorStateInfo; | import org.apache.kafka.clients.consumer.Consumer; | ||||||
|  | import org.apache.kafka.clients.consumer.OffsetAndMetadata; | ||||||
|  | import org.apache.kafka.common.TopicPartition; | ||||||
| import org.apache.kafka.connect.util.clusters.EmbeddedConnectCluster; | import org.apache.kafka.connect.util.clusters.EmbeddedConnectCluster; | ||||||
| import org.apache.kafka.connect.util.clusters.EmbeddedKafkaCluster; | import org.apache.kafka.connect.util.clusters.EmbeddedKafkaCluster; | ||||||
| import org.apache.kafka.test.IntegrationTest; | import org.apache.kafka.test.IntegrationTest; | ||||||
| import org.apache.kafka.test.TestUtils; |  | ||||||
| import org.apache.kafka.clients.consumer.Consumer; |  | ||||||
| import org.apache.kafka.clients.consumer.OffsetAndMetadata; |  | ||||||
| import org.apache.kafka.clients.admin.Admin; |  | ||||||
| import org.apache.kafka.common.TopicPartition; |  | ||||||
| 
 |  | ||||||
| import org.junit.After; | import org.junit.After; | ||||||
| import org.junit.Before; | import org.junit.Before; | ||||||
| import org.junit.Test; | import org.junit.Test; | ||||||
|  | @ -34,19 +30,18 @@ import org.junit.experimental.categories.Category; | ||||||
| import org.slf4j.Logger; | import org.slf4j.Logger; | ||||||
| import org.slf4j.LoggerFactory; | import org.slf4j.LoggerFactory; | ||||||
| 
 | 
 | ||||||
|  | import java.time.Duration; | ||||||
| import java.util.Arrays; | import java.util.Arrays; | ||||||
| import java.io.IOException; | import java.util.Collections; | ||||||
| import java.util.HashMap; | import java.util.HashMap; | ||||||
| import java.util.HashSet; | import java.util.HashSet; | ||||||
| import java.util.Map; | import java.util.Map; | ||||||
| import java.util.Set; |  | ||||||
| import java.util.Collections; |  | ||||||
| import java.util.Properties; | import java.util.Properties; | ||||||
| import java.time.Duration; | import java.util.Set; | ||||||
| 
 | 
 | ||||||
|  | import static org.apache.kafka.test.TestUtils.waitForCondition; | ||||||
| import static org.junit.Assert.assertEquals; | import static org.junit.Assert.assertEquals; | ||||||
| import static org.junit.Assert.assertTrue; | import static org.junit.Assert.assertTrue; | ||||||
| import static org.apache.kafka.test.TestUtils.waitForCondition; |  | ||||||
| 
 | 
 | ||||||
| /** | /** | ||||||
|  * Tests MM2 replication and failover/failback logic. |  * Tests MM2 replication and failover/failback logic. | ||||||
|  | @ -71,7 +66,7 @@ public class MirrorConnectorsIntegrationTest { | ||||||
|     private EmbeddedConnectCluster backup; |     private EmbeddedConnectCluster backup; | ||||||
| 
 | 
 | ||||||
|     @Before |     @Before | ||||||
|     public void setup() throws IOException, InterruptedException { |     public void setup() throws InterruptedException { | ||||||
|         Properties brokerProps = new Properties(); |         Properties brokerProps = new Properties(); | ||||||
|         brokerProps.put("auto.create.topics.enable", "false"); |         brokerProps.put("auto.create.topics.enable", "false"); | ||||||
| 
 | 
 | ||||||
|  | @ -116,7 +111,11 @@ public class MirrorConnectorsIntegrationTest { | ||||||
|                 .build(); |                 .build(); | ||||||
| 
 | 
 | ||||||
|         primary.start(); |         primary.start(); | ||||||
|  |         primary.assertions().assertAtLeastNumWorkersAreUp(3, | ||||||
|  |                 "Workers of primary-connect-cluster did not start in time."); | ||||||
|         backup.start(); |         backup.start(); | ||||||
|  |         primary.assertions().assertAtLeastNumWorkersAreUp(3, | ||||||
|  |                 "Workers of backup-connect-cluster did not start in time."); | ||||||
| 
 | 
 | ||||||
|         // create these topics before starting the connectors so we don't need to wait for discovery |         // create these topics before starting the connectors so we don't need to wait for discovery | ||||||
|         primary.kafka().createTopic("test-topic-1", NUM_PARTITIONS); |         primary.kafka().createTopic("test-topic-1", NUM_PARTITIONS); | ||||||
|  | @ -189,30 +188,13 @@ public class MirrorConnectorsIntegrationTest { | ||||||
|     private void waitUntilMirrorMakerIsRunning(EmbeddedConnectCluster connectCluster, |     private void waitUntilMirrorMakerIsRunning(EmbeddedConnectCluster connectCluster, | ||||||
|         Set<String> connNames) throws InterruptedException { |         Set<String> connNames) throws InterruptedException { | ||||||
|         for (String connector : connNames) { |         for (String connector : connNames) { | ||||||
|             TestUtils.waitForCondition(() -> areConnectorAndTasksRunning(connectCluster, |             connectCluster.assertions().assertConnectorAndAtLeastNumTasksAreRunning(connector, 1, | ||||||
|                 connector), "Timed out trying to verify connector " + |                     "Connector " + connector + " tasks did not start in time on cluster: " + connectCluster); | ||||||
|                 connector + " was up!"); |  | ||||||
|         } |  | ||||||
|     } |  | ||||||
| 
 |  | ||||||
|     private boolean areConnectorAndTasksRunning(EmbeddedConnectCluster connectCluster, |  | ||||||
|         String connectorName) { |  | ||||||
|         try { |  | ||||||
|             ConnectorStateInfo info = connectCluster.connectorStatus(connectorName); |  | ||||||
|             boolean result = info != null |  | ||||||
|                 && !info.tasks().isEmpty() |  | ||||||
|                 && info.connector().state().equals(AbstractStatus.State.RUNNING.toString()) |  | ||||||
|                 && info.tasks().stream().allMatch(s -> s.state().equals(AbstractStatus.State.RUNNING.toString())); |  | ||||||
|             log.debug("Found connector and tasks running: {}", result); |  | ||||||
|             return result; |  | ||||||
|         } catch (Exception e) { |  | ||||||
|             log.error("Could not check connector state info.", e); |  | ||||||
|             return false; |  | ||||||
|         } |         } | ||||||
|     } |     } | ||||||
| 
 | 
 | ||||||
|     @After |     @After | ||||||
|     public void close() throws IOException { |     public void close() { | ||||||
|         for (String x : primary.connectors()) { |         for (String x : primary.connectors()) { | ||||||
|             primary.deleteConnector(x); |             primary.deleteConnector(x); | ||||||
|         } |         } | ||||||
|  |  | ||||||
|  | @ -28,7 +28,6 @@ import org.junit.experimental.categories.Category; | ||||||
| import org.slf4j.Logger; | import org.slf4j.Logger; | ||||||
| import org.slf4j.LoggerFactory; | import org.slf4j.LoggerFactory; | ||||||
| 
 | 
 | ||||||
| import java.io.IOException; |  | ||||||
| import java.util.Collections; | import java.util.Collections; | ||||||
| import java.util.HashMap; | import java.util.HashMap; | ||||||
| import java.util.Map; | import java.util.Map; | ||||||
|  | @ -66,7 +65,7 @@ public class ConnectWorkerIntegrationTest { | ||||||
|     Properties brokerProps = new Properties(); |     Properties brokerProps = new Properties(); | ||||||
| 
 | 
 | ||||||
|     @Before |     @Before | ||||||
|     public void setup() throws IOException { |     public void setup() { | ||||||
|         // setup Connect worker properties |         // setup Connect worker properties | ||||||
|         workerProps.put(OFFSET_COMMIT_INTERVAL_MS_CONFIG, String.valueOf(OFFSET_COMMIT_INTERVAL_MS)); |         workerProps.put(OFFSET_COMMIT_INTERVAL_MS_CONFIG, String.valueOf(OFFSET_COMMIT_INTERVAL_MS)); | ||||||
|         workerProps.put(CONNECTOR_CLIENT_POLICY_CLASS_CONFIG, "All"); |         workerProps.put(CONNECTOR_CLIENT_POLICY_CLASS_CONFIG, "All"); | ||||||
|  |  | ||||||
|  | @ -29,7 +29,6 @@ import org.junit.After; | ||||||
| import org.junit.Test; | import org.junit.Test; | ||||||
| import org.junit.experimental.categories.Category; | import org.junit.experimental.categories.Category; | ||||||
| 
 | 
 | ||||||
| import java.io.IOException; |  | ||||||
| import java.util.HashMap; | import java.util.HashMap; | ||||||
| import java.util.Map; | import java.util.Map; | ||||||
| import java.util.Properties; | import java.util.Properties; | ||||||
|  | @ -50,7 +49,6 @@ public class ConnectorClientPolicyIntegrationTest { | ||||||
|     private static final int NUM_WORKERS = 1; |     private static final int NUM_WORKERS = 1; | ||||||
|     private static final String CONNECTOR_NAME = "simple-conn"; |     private static final String CONNECTOR_NAME = "simple-conn"; | ||||||
| 
 | 
 | ||||||
| 
 |  | ||||||
|     @After |     @After | ||||||
|     public void close() { |     public void close() { | ||||||
|     } |     } | ||||||
|  | @ -73,7 +71,7 @@ public class ConnectorClientPolicyIntegrationTest { | ||||||
|     @Test |     @Test | ||||||
|     public void testCreateWithAllowedOverridesForPrincipalPolicy() throws Exception { |     public void testCreateWithAllowedOverridesForPrincipalPolicy() throws Exception { | ||||||
|         Map<String, String> props = basicConnectorConfig(); |         Map<String, String> props = basicConnectorConfig(); | ||||||
|         props.put(ConnectorConfig.CONNECTOR_CLIENT_CONSUMER_OVERRIDES_PREFIX + CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, "PLAIN"); |         props.put(ConnectorConfig.CONNECTOR_CLIENT_CONSUMER_OVERRIDES_PREFIX + CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, "PLAINTEXT"); | ||||||
|         assertPassCreateConnector("Principal", props); |         assertPassCreateConnector("Principal", props); | ||||||
|     } |     } | ||||||
| 
 | 
 | ||||||
|  | @ -85,7 +83,7 @@ public class ConnectorClientPolicyIntegrationTest { | ||||||
|         assertPassCreateConnector("All", props); |         assertPassCreateConnector("All", props); | ||||||
|     } |     } | ||||||
| 
 | 
 | ||||||
|     private EmbeddedConnectCluster connectClusterWithPolicy(String policy) throws IOException { |     private EmbeddedConnectCluster connectClusterWithPolicy(String policy) throws InterruptedException { | ||||||
|         // setup Connect worker properties |         // setup Connect worker properties | ||||||
|         Map<String, String> workerProps = new HashMap<>(); |         Map<String, String> workerProps = new HashMap<>(); | ||||||
|         workerProps.put(OFFSET_COMMIT_INTERVAL_MS_CONFIG, String.valueOf(5_000)); |         workerProps.put(OFFSET_COMMIT_INTERVAL_MS_CONFIG, String.valueOf(5_000)); | ||||||
|  | @ -106,10 +104,13 @@ public class ConnectorClientPolicyIntegrationTest { | ||||||
| 
 | 
 | ||||||
|         // start the clusters |         // start the clusters | ||||||
|         connect.start(); |         connect.start(); | ||||||
|  |         connect.assertions().assertAtLeastNumWorkersAreUp(NUM_WORKERS, | ||||||
|  |                 "Initial group of workers did not start in time."); | ||||||
|  | 
 | ||||||
|         return connect; |         return connect; | ||||||
|     } |     } | ||||||
| 
 | 
 | ||||||
|     private void assertFailCreateConnector(String policy, Map<String, String> props) throws IOException { |     private void assertFailCreateConnector(String policy, Map<String, String> props) throws InterruptedException { | ||||||
|         EmbeddedConnectCluster connect = connectClusterWithPolicy(policy); |         EmbeddedConnectCluster connect = connectClusterWithPolicy(policy); | ||||||
|         try { |         try { | ||||||
|             connect.configureConnector(CONNECTOR_NAME, props); |             connect.configureConnector(CONNECTOR_NAME, props); | ||||||
|  | @ -121,10 +122,12 @@ public class ConnectorClientPolicyIntegrationTest { | ||||||
|         } |         } | ||||||
|     } |     } | ||||||
| 
 | 
 | ||||||
|     private void assertPassCreateConnector(String policy, Map<String, String> props) throws IOException { |     private void assertPassCreateConnector(String policy, Map<String, String> props) throws InterruptedException { | ||||||
|         EmbeddedConnectCluster connect = connectClusterWithPolicy(policy); |         EmbeddedConnectCluster connect = connectClusterWithPolicy(policy); | ||||||
|         try { |         try { | ||||||
|             connect.configureConnector(CONNECTOR_NAME, props); |             connect.configureConnector(CONNECTOR_NAME, props); | ||||||
|  |             connect.assertions().assertConnectorAndAtLeastNumTasksAreRunning(CONNECTOR_NAME, NUM_TASKS, | ||||||
|  |                     "Connector tasks did not start in time."); | ||||||
|         } catch (ConnectRestException e) { |         } catch (ConnectRestException e) { | ||||||
|             fail("Should be able to create connector"); |             fail("Should be able to create connector"); | ||||||
|         } finally { |         } finally { | ||||||
|  |  | ||||||
|  | @ -34,7 +34,6 @@ import org.junit.experimental.categories.Category; | ||||||
| import org.slf4j.Logger; | import org.slf4j.Logger; | ||||||
| import org.slf4j.LoggerFactory; | import org.slf4j.LoggerFactory; | ||||||
| 
 | 
 | ||||||
| import java.io.IOException; |  | ||||||
| import java.util.HashMap; | import java.util.HashMap; | ||||||
| import java.util.Map; | import java.util.Map; | ||||||
| import java.util.concurrent.TimeUnit; | import java.util.concurrent.TimeUnit; | ||||||
|  | @ -69,6 +68,7 @@ public class ErrorHandlingIntegrationTest { | ||||||
| 
 | 
 | ||||||
|     private static final Logger log = LoggerFactory.getLogger(ErrorHandlingIntegrationTest.class); |     private static final Logger log = LoggerFactory.getLogger(ErrorHandlingIntegrationTest.class); | ||||||
| 
 | 
 | ||||||
|  |     private static final int NUM_WORKERS = 1; | ||||||
|     private static final String DLQ_TOPIC = "my-connector-errors"; |     private static final String DLQ_TOPIC = "my-connector-errors"; | ||||||
|     private static final String CONNECTOR_NAME = "error-conn"; |     private static final String CONNECTOR_NAME = "error-conn"; | ||||||
|     private static final String TASK_ID = "error-conn-0"; |     private static final String TASK_ID = "error-conn-0"; | ||||||
|  | @ -83,12 +83,14 @@ public class ErrorHandlingIntegrationTest { | ||||||
|     private ConnectorHandle connectorHandle; |     private ConnectorHandle connectorHandle; | ||||||
| 
 | 
 | ||||||
|     @Before |     @Before | ||||||
|     public void setup() throws IOException { |     public void setup() throws InterruptedException { | ||||||
|         // setup Connect cluster with defaults |         // setup Connect cluster with defaults | ||||||
|         connect = new EmbeddedConnectCluster.Builder().build(); |         connect = new EmbeddedConnectCluster.Builder().build(); | ||||||
| 
 | 
 | ||||||
|         // start Connect cluster |         // start Connect cluster | ||||||
|         connect.start(); |         connect.start(); | ||||||
|  |         connect.assertions().assertAtLeastNumWorkersAreUp(NUM_WORKERS, | ||||||
|  |                 "Initial group of workers did not start in time."); | ||||||
| 
 | 
 | ||||||
|         // get connector handles before starting test. |         // get connector handles before starting test. | ||||||
|         connectorHandle = RuntimeHandles.get().connectorHandle(CONNECTOR_NAME); |         connectorHandle = RuntimeHandles.get().connectorHandle(CONNECTOR_NAME); | ||||||
|  | @ -134,6 +136,8 @@ public class ErrorHandlingIntegrationTest { | ||||||
|         connectorHandle.taskHandle(TASK_ID).expectedRecords(EXPECTED_CORRECT_RECORDS); |         connectorHandle.taskHandle(TASK_ID).expectedRecords(EXPECTED_CORRECT_RECORDS); | ||||||
| 
 | 
 | ||||||
|         connect.configureConnector(CONNECTOR_NAME, props); |         connect.configureConnector(CONNECTOR_NAME, props); | ||||||
|  |         connect.assertions().assertConnectorAndAtLeastNumTasksAreRunning(CONNECTOR_NAME, NUM_TASKS, | ||||||
|  |                 "Connector tasks did not start in time."); | ||||||
| 
 | 
 | ||||||
|         waitForCondition(this::checkForPartitionAssignment, |         waitForCondition(this::checkForPartitionAssignment, | ||||||
|                 CONNECTOR_SETUP_DURATION_MS, |                 CONNECTOR_SETUP_DURATION_MS, | ||||||
|  | @ -172,6 +176,9 @@ public class ErrorHandlingIntegrationTest { | ||||||
|         } |         } | ||||||
| 
 | 
 | ||||||
|         connect.deleteConnector(CONNECTOR_NAME); |         connect.deleteConnector(CONNECTOR_NAME); | ||||||
|  |         connect.assertions().assertConnectorAndTasksAreStopped(CONNECTOR_NAME, | ||||||
|  |                 "Connector tasks did not stop in time."); | ||||||
|  | 
 | ||||||
|     } |     } | ||||||
| 
 | 
 | ||||||
|     /** |     /** | ||||||
|  |  | ||||||
|  | @ -27,7 +27,6 @@ import org.junit.experimental.categories.Category; | ||||||
| import org.slf4j.Logger; | import org.slf4j.Logger; | ||||||
| import org.slf4j.LoggerFactory; | import org.slf4j.LoggerFactory; | ||||||
| 
 | 
 | ||||||
| import java.io.IOException; |  | ||||||
| import java.util.HashMap; | import java.util.HashMap; | ||||||
| import java.util.Map; | import java.util.Map; | ||||||
| import java.util.Properties; | import java.util.Properties; | ||||||
|  | @ -66,7 +65,7 @@ public class ExampleConnectIntegrationTest { | ||||||
|     private ConnectorHandle connectorHandle; |     private ConnectorHandle connectorHandle; | ||||||
| 
 | 
 | ||||||
|     @Before |     @Before | ||||||
|     public void setup() throws IOException { |     public void setup() { | ||||||
|         // setup Connect worker properties |         // setup Connect worker properties | ||||||
|         Map<String, String> exampleWorkerProps = new HashMap<>(); |         Map<String, String> exampleWorkerProps = new HashMap<>(); | ||||||
|         exampleWorkerProps.put(OFFSET_COMMIT_INTERVAL_MS_CONFIG, String.valueOf(5_000)); |         exampleWorkerProps.put(OFFSET_COMMIT_INTERVAL_MS_CONFIG, String.valueOf(5_000)); | ||||||
|  |  | ||||||
|  | @ -28,7 +28,6 @@ import org.junit.experimental.categories.Category; | ||||||
| import org.slf4j.Logger; | import org.slf4j.Logger; | ||||||
| import org.slf4j.LoggerFactory; | import org.slf4j.LoggerFactory; | ||||||
| 
 | 
 | ||||||
| import java.io.IOException; |  | ||||||
| import java.util.ArrayList; | import java.util.ArrayList; | ||||||
| import java.util.Collection; | import java.util.Collection; | ||||||
| import java.util.HashMap; | import java.util.HashMap; | ||||||
|  | @ -70,7 +69,7 @@ public class RebalanceSourceConnectorsIntegrationTest { | ||||||
|     private EmbeddedConnectCluster connect; |     private EmbeddedConnectCluster connect; | ||||||
| 
 | 
 | ||||||
|     @Before |     @Before | ||||||
|     public void setup() throws IOException { |     public void setup() { | ||||||
|         // setup Connect worker properties |         // setup Connect worker properties | ||||||
|         Map<String, String> workerProps = new HashMap<>(); |         Map<String, String> workerProps = new HashMap<>(); | ||||||
|         workerProps.put(CONNECT_PROTOCOL_CONFIG, COMPATIBLE.toString()); |         workerProps.put(CONNECT_PROTOCOL_CONFIG, COMPATIBLE.toString()); | ||||||
|  |  | ||||||
|  | @ -34,7 +34,6 @@ import org.junit.experimental.categories.Category; | ||||||
| import javax.ws.rs.GET; | import javax.ws.rs.GET; | ||||||
| import javax.ws.rs.Path; | import javax.ws.rs.Path; | ||||||
| import javax.ws.rs.core.Response; | import javax.ws.rs.core.Response; | ||||||
| import java.io.IOException; |  | ||||||
| import java.util.Collections; | import java.util.Collections; | ||||||
| import java.util.HashMap; | import java.util.HashMap; | ||||||
| import java.util.Map; | import java.util.Map; | ||||||
|  | @ -57,11 +56,12 @@ public class RestExtensionIntegrationTest { | ||||||
| 
 | 
 | ||||||
|     private static final long REST_EXTENSION_REGISTRATION_TIMEOUT_MS = TimeUnit.MINUTES.toMillis(1); |     private static final long REST_EXTENSION_REGISTRATION_TIMEOUT_MS = TimeUnit.MINUTES.toMillis(1); | ||||||
|     private static final long CONNECTOR_HEALTH_AND_CONFIG_TIMEOUT_MS = TimeUnit.MINUTES.toMillis(1); |     private static final long CONNECTOR_HEALTH_AND_CONFIG_TIMEOUT_MS = TimeUnit.MINUTES.toMillis(1); | ||||||
|  |     private static final int NUM_WORKERS = 1; | ||||||
| 
 | 
 | ||||||
|     private EmbeddedConnectCluster connect; |     private EmbeddedConnectCluster connect; | ||||||
| 
 | 
 | ||||||
|     @Test |     @Test | ||||||
|     public void testRestExtensionApi() throws IOException, InterruptedException { |     public void testRestExtensionApi() throws InterruptedException { | ||||||
|         // setup Connect worker properties |         // setup Connect worker properties | ||||||
|         Map<String, String> workerProps = new HashMap<>(); |         Map<String, String> workerProps = new HashMap<>(); | ||||||
|         workerProps.put(REST_EXTENSION_CLASSES_CONFIG, IntegrationTestRestExtension.class.getName()); |         workerProps.put(REST_EXTENSION_CLASSES_CONFIG, IntegrationTestRestExtension.class.getName()); | ||||||
|  | @ -69,7 +69,7 @@ public class RestExtensionIntegrationTest { | ||||||
|         // build a Connect cluster backed by Kafka and Zk |         // build a Connect cluster backed by Kafka and Zk | ||||||
|         connect = new EmbeddedConnectCluster.Builder() |         connect = new EmbeddedConnectCluster.Builder() | ||||||
|             .name("connect-cluster") |             .name("connect-cluster") | ||||||
|             .numWorkers(1) |             .numWorkers(NUM_WORKERS) | ||||||
|             .numBrokers(1) |             .numBrokers(1) | ||||||
|             .workerProps(workerProps) |             .workerProps(workerProps) | ||||||
|             .build(); |             .build(); | ||||||
|  | @ -77,6 +77,9 @@ public class RestExtensionIntegrationTest { | ||||||
|         // start the clusters |         // start the clusters | ||||||
|         connect.start(); |         connect.start(); | ||||||
| 
 | 
 | ||||||
|  |         connect.assertions().assertAtLeastNumWorkersAreUp(NUM_WORKERS, | ||||||
|  |                 "Initial group of workers did not start in time."); | ||||||
|  | 
 | ||||||
|         WorkerHandle worker = connect.workers().stream() |         WorkerHandle worker = connect.workers().stream() | ||||||
|             .findFirst() |             .findFirst() | ||||||
|             .orElseThrow(() -> new AssertionError("At least one worker handle should be available")); |             .orElseThrow(() -> new AssertionError("At least one worker handle should be available")); | ||||||
|  | @ -99,6 +102,8 @@ public class RestExtensionIntegrationTest { | ||||||
|             connectorHandle.taskHandle(connectorHandle.name() + "-0"); |             connectorHandle.taskHandle(connectorHandle.name() + "-0"); | ||||||
|             StartAndStopLatch connectorStartLatch = connectorHandle.expectedStarts(1); |             StartAndStopLatch connectorStartLatch = connectorHandle.expectedStarts(1); | ||||||
|             connect.configureConnector(connectorHandle.name(), connectorProps); |             connect.configureConnector(connectorHandle.name(), connectorProps); | ||||||
|  |             connect.assertions().assertConnectorAndAtLeastNumTasksAreRunning(connectorHandle.name(), 1, | ||||||
|  |                     "Connector tasks did not start in time."); | ||||||
|             connectorStartLatch.await(CONNECTOR_HEALTH_AND_CONFIG_TIMEOUT_MS, TimeUnit.MILLISECONDS); |             connectorStartLatch.await(CONNECTOR_HEALTH_AND_CONFIG_TIMEOUT_MS, TimeUnit.MILLISECONDS); | ||||||
| 
 | 
 | ||||||
|             String workerId = String.format("%s:%d", worker.url().getHost(), worker.url().getPort()); |             String workerId = String.format("%s:%d", worker.url().getHost(), worker.url().getPort()); | ||||||
|  |  | ||||||
|  | @ -28,7 +28,6 @@ import org.junit.experimental.categories.Category; | ||||||
| import org.slf4j.Logger; | import org.slf4j.Logger; | ||||||
| import org.slf4j.LoggerFactory; | import org.slf4j.LoggerFactory; | ||||||
| 
 | 
 | ||||||
| import java.io.IOException; |  | ||||||
| import java.util.HashMap; | import java.util.HashMap; | ||||||
| import java.util.Map; | import java.util.Map; | ||||||
| import java.util.concurrent.TimeUnit; | import java.util.concurrent.TimeUnit; | ||||||
|  | @ -61,7 +60,7 @@ public class SessionedProtocolIntegrationTest { | ||||||
|     private ConnectorHandle connectorHandle; |     private ConnectorHandle connectorHandle; | ||||||
| 
 | 
 | ||||||
|     @Before |     @Before | ||||||
|     public void setup() throws IOException { |     public void setup() { | ||||||
|         // setup Connect worker properties |         // setup Connect worker properties | ||||||
|         Map<String, String> workerProps = new HashMap<>(); |         Map<String, String> workerProps = new HashMap<>(); | ||||||
|         workerProps.put(CONNECT_PROTOCOL_CONFIG, ConnectProtocolCompatibility.SESSIONED.protocol()); |         workerProps.put(CONNECT_PROTOCOL_CONFIG, ConnectProtocolCompatibility.SESSIONED.protocol()); | ||||||
|  |  | ||||||
|  | @ -284,7 +284,8 @@ public class EmbeddedConnectCluster { | ||||||
|         if (response.getStatus() < Response.Status.BAD_REQUEST.getStatusCode()) { |         if (response.getStatus() < Response.Status.BAD_REQUEST.getStatusCode()) { | ||||||
|             return responseToString(response); |             return responseToString(response); | ||||||
|         } |         } | ||||||
|         throw new ConnectRestException(response.getStatus(), "Could not execute PUT request"); |         throw new ConnectRestException(response.getStatus(), | ||||||
|  |                 "Could not execute PUT request. Error response: " + responseToString(response)); | ||||||
|     } |     } | ||||||
| 
 | 
 | ||||||
|     /** |     /** | ||||||
|  | @ -298,7 +299,8 @@ public class EmbeddedConnectCluster { | ||||||
|         String url = endpointForResource(String.format("connectors/%s", connName)); |         String url = endpointForResource(String.format("connectors/%s", connName)); | ||||||
|         Response response = requestDelete(url); |         Response response = requestDelete(url); | ||||||
|         if (response.getStatus() >= Response.Status.BAD_REQUEST.getStatusCode()) { |         if (response.getStatus() >= Response.Status.BAD_REQUEST.getStatusCode()) { | ||||||
|             throw new ConnectRestException(response.getStatus(), "Could not execute DELETE request."); |             throw new ConnectRestException(response.getStatus(), | ||||||
|  |                     "Could not execute DELETE request. Error response: " + responseToString(response)); | ||||||
|         } |         } | ||||||
|     } |     } | ||||||
| 
 | 
 | ||||||
|  | @ -358,7 +360,7 @@ public class EmbeddedConnectCluster { | ||||||
|      * |      * | ||||||
|      * @param resource the resource under the worker's admin endpoint |      * @param resource the resource under the worker's admin endpoint | ||||||
|      * @return the admin endpoint URL |      * @return the admin endpoint URL | ||||||
|      * @throws ConnectRestException if no admin REST endpoint is available |      * @throws ConnectException if no admin REST endpoint is available | ||||||
|      */ |      */ | ||||||
|     public String adminEndpoint(String resource) { |     public String adminEndpoint(String resource) { | ||||||
|         String url = connectCluster.stream() |         String url = connectCluster.stream() | ||||||
|  | @ -375,7 +377,7 @@ public class EmbeddedConnectCluster { | ||||||
|      * |      * | ||||||
|      * @param resource the resource under the worker's admin endpoint |      * @param resource the resource under the worker's admin endpoint | ||||||
|      * @return the admin endpoint URL |      * @return the admin endpoint URL | ||||||
|      * @throws ConnectRestException if no REST endpoint is available |      * @throws ConnectException if no REST endpoint is available | ||||||
|      */ |      */ | ||||||
|     public String endpointForResource(String resource) { |     public String endpointForResource(String resource) { | ||||||
|         String url = connectCluster.stream() |         String url = connectCluster.stream() | ||||||
|  |  | ||||||
		Loading…
	
		Reference in New Issue