ning2008wisc commented on a change in pull request #9224:
URL: https://github.com/apache/kafka/pull/9224#discussion_r527396830



##########
File path: 
connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationBaseTest.java
##########
@@ -0,0 +1,407 @@
+/*
+ * 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.mirror.integration;
+
+import org.apache.kafka.clients.CommonClientConfigs;
+import org.apache.kafka.clients.admin.Admin;
+import org.apache.kafka.clients.admin.Config;
+import org.apache.kafka.clients.admin.ConfigEntry;
+import org.apache.kafka.clients.admin.DescribeConfigsResult;
+import org.apache.kafka.clients.consumer.Consumer;
+import org.apache.kafka.clients.consumer.ConsumerConfig;
+import org.apache.kafka.clients.consumer.ConsumerRecords;
+import org.apache.kafka.clients.consumer.KafkaConsumer;
+import org.apache.kafka.clients.consumer.OffsetAndMetadata;
+import org.apache.kafka.common.config.ConfigResource;
+import org.apache.kafka.common.config.SslConfigs;
+import org.apache.kafka.common.config.TopicConfig;
+import org.apache.kafka.common.config.types.Password;
+import org.apache.kafka.common.utils.Exit;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.connect.connector.Connector;
+import org.apache.kafka.connect.mirror.MirrorMakerConfig;
+import org.apache.kafka.connect.mirror.SourceAndTarget;
+import org.apache.kafka.connect.util.clusters.EmbeddedConnectCluster;
+import org.apache.kafka.connect.util.clusters.EmbeddedKafkaCluster;
+import static org.apache.kafka.connect.mirror.TestUtils.expectedRecords;
+
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Properties;
+import java.util.UUID;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+
+import static org.apache.kafka.test.TestUtils.waitForCondition;
+import org.apache.kafka.test.IntegrationTest;
+import kafka.server.KafkaConfig$;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import static org.junit.Assert.assertFalse;
+import org.junit.experimental.categories.Category;
+
+/**
+ * Common Test functions for MM2 integration tests
+ */
+@Category(IntegrationTest.class)
+public class MirrorConnectorsIntegrationBaseTest {
+    private static final Logger log = 
LoggerFactory.getLogger(MirrorConnectorsIntegrationBaseTest.class);
+    
+    protected static final int NUM_RECORDS_PER_PARTITION = 10;
+    public static final int NUM_PARTITIONS = 10;
+    protected static final int NUM_RECORDS_PRODUCED = NUM_PARTITIONS * 
NUM_RECORDS_PER_PARTITION;
+    protected static final int RECORD_TRANSFER_DURATION_MS = 30_000;
+    protected static final int CHECKPOINT_DURATION_MS = 20_000;
+    protected static final int RECORD_CONSUME_DURATION_MS = 20_000;
+    protected static final int OFFSET_SYNC_DURATION_MS = 30_000;
+    protected static final int NUM_WORKERS = 3;
+    protected static final int CONSUMER_POLL_TIMEOUT_MS = 500;
+    protected static final int BROKER_RESTART_TIMEOUT_MS = 10_000;
+    protected static final String PRIMARY_CLUSTER_ALIAS = "primary";
+    protected static final String BACKUP_CLUSTER_ALIAS = "backup";
+
+    protected Map<String, String> mm2Props;
+    protected MirrorMakerConfig mm2Config; 
+    protected EmbeddedConnectCluster primary;
+    protected EmbeddedConnectCluster backup;
+    
+    private final AtomicBoolean exited = new AtomicBoolean(false);
+    private Properties primaryBrokerProps = new Properties();
+    protected Properties backupBrokerProps = new Properties();
+    private Map<String, String> primaryWorkerProps = new HashMap<>();
+    private Map<String, String> backupWorkerProps = new HashMap<>();
+    private Properties sslProps = new Properties();
+    
+    private void loadSslPropsFromBrokerConfig() {       
+        sslProps.put(SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG, 
backupBrokerProps.get(SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG));
+        sslProps.put(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, ((Password) 
backupBrokerProps.get(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG)).value());
+        sslProps.put(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, "SSL");
+    }
+    
+    protected void setSslConfig() {
+        // set SSL config for kafka connect worker
+        
backupWorkerProps.putAll(sslProps.entrySet().stream().collect(Collectors.toMap(
+            e -> String.valueOf(e.getKey()), e ->  
String.valueOf(e.getValue()))));
+        mm2Props.putAll(sslProps.entrySet().stream().collect(Collectors.toMap(
+            e -> BACKUP_CLUSTER_ALIAS + "." + String.valueOf(e.getKey()), e -> 
 String.valueOf(e.getValue()))));
+        // set SSL config for producer used by source task in MM2
+        mm2Props.putAll(sslProps.entrySet().stream().collect(Collectors.toMap(
+            e -> BACKUP_CLUSTER_ALIAS + ".producer." + 
String.valueOf(e.getKey()), e ->  String.valueOf(e.getValue()))));
+    }
+    
+    protected void startClusters() throws InterruptedException {
+        primaryBrokerProps.put("auto.create.topics.enable", "false");
+        backupBrokerProps.put("auto.create.topics.enable", "false");
+        
+        mm2Props = basicMM2Config();
+        
+        // if backup kafka cluster contains ssl config, enable ssl of kafka 
connect and mm2
+        final Object listeners = 
backupBrokerProps.get(KafkaConfig$.MODULE$.ListenersProp());
+        if (listeners != null && listeners.toString().contains("SSL")) {
+            loadSslPropsFromBrokerConfig();
+            setSslConfig();
+        }
+
+        mm2Config = new MirrorMakerConfig(mm2Props); 
+        primaryWorkerProps = mm2Config.workerConfig(new 
SourceAndTarget(BACKUP_CLUSTER_ALIAS, PRIMARY_CLUSTER_ALIAS));
+        backupWorkerProps.putAll(mm2Config.workerConfig(new 
SourceAndTarget(PRIMARY_CLUSTER_ALIAS, BACKUP_CLUSTER_ALIAS)));
+        
+        primary = new EmbeddedConnectCluster.Builder()
+                .name(PRIMARY_CLUSTER_ALIAS + "-connect-cluster")
+                .numWorkers(3)
+                .numBrokers(1)
+                .brokerProps(primaryBrokerProps)
+                .workerProps(primaryWorkerProps)
+                .build();
+
+        backup = new EmbeddedConnectCluster.Builder()
+                .name(BACKUP_CLUSTER_ALIAS + "-connect-cluster")
+                .numWorkers(3)
+                .numBrokers(1)
+                .brokerProps(backupBrokerProps)
+                .workerProps(backupWorkerProps)
+                .build();
+        
+        primary.start();
+        primary.assertions().assertAtLeastNumWorkersAreUp(3,
+                "Workers of " + PRIMARY_CLUSTER_ALIAS + "-connect-cluster did 
not start in time.");
+        
+        backup.start();
+        backup.assertions().assertAtLeastNumWorkersAreUp(3,
+                "Workers of " + BACKUP_CLUSTER_ALIAS + "-connect-cluster did 
not start in time.");
+
+        createTopics();
+ 
+        dummyConsumption();
+        
+        log.info(PRIMARY_CLUSTER_ALIAS + " REST service: {}", 
primary.endpointForResource("connectors"));
+        log.info(BACKUP_CLUSTER_ALIAS + " REST service: {}", 
backup.endpointForResource("connectors"));
+        log.info(PRIMARY_CLUSTER_ALIAS + " brokers: {}", 
primary.kafka().bootstrapServers());
+        log.info(BACKUP_CLUSTER_ALIAS + " brokers: {}", 
backup.kafka().bootstrapServers());
+        
+        // now that the brokers are running, we can finish setting up the 
Connectors
+        mm2Props.put(PRIMARY_CLUSTER_ALIAS + ".bootstrap.servers", 
primary.kafka().bootstrapServers());
+        mm2Props.put(BACKUP_CLUSTER_ALIAS + ".bootstrap.servers", 
backup.kafka().bootstrapServers());
+        
+        Exit.setExitProcedure((status, errorCode) -> exited.set(true));
+    }
+    
+    public void shutdownClusters() {
+        for (String x : primary.connectors()) {
+            primary.deleteConnector(x);
+        }
+        for (String x : backup.connectors()) {
+            backup.deleteConnector(x);
+        }
+        deleteAllTopics(primary.kafka());
+        deleteAllTopics(backup.kafka());
+        primary.stop();
+        backup.stop();
+        try {
+            assertFalse(exited.get());
+        } finally {
+            Exit.resetExitProcedure();
+        }
+    }
+    /*
+     * launch the connectors on kafka connect cluster and check if they are 
running
+     */
+    protected static void waitUntilMirrorMakerIsRunning(EmbeddedConnectCluster 
connectCluster, 
+            List<Class<? extends Connector>> connectorClasses, 
MirrorMakerConfig mm2Config, 
+            String primary, String backup) throws InterruptedException {
+        for (int i = 0; i < connectorClasses.size(); i++) {
+            String connector = connectorClasses.get(i).getSimpleName();
+            connectCluster.configureConnector(connector, 
mm2Config.connectorBaseConfig(
+                new SourceAndTarget(primary, backup), 
connectorClasses.get(i)));
+        }
+        
+        // we wait for the connector and tasks to come up for each connector, 
so that when we do the
+        // actual testing, we are certain that the tasks are up and running; 
this will prevent
+        // flaky tests where the connector and tasks didn't start up in time 
for the tests to be run
+        List<String> connectorNames = connectorClasses.stream().map(x -> 
x.getSimpleName())
+                .collect(Collectors.toList());
+        for (String connector : connectorNames) {
+            
connectCluster.assertions().assertConnectorAndAtLeastNumTasksAreRunning(connector,
 1,
+                    "Connector " + connector + " tasks did not start in time 
on cluster: " + connectCluster);
+        }
+    }
+ 
+    /*
+     * delete all topics of the input kafka cluster
+     */
+    protected static void deleteAllTopics(EmbeddedKafkaCluster cluster) {
+        Admin client = cluster.createAdminClient();
+        try {
+            client.deleteTopics(client.listTopics().names().get());
+        } catch (Throwable e) {
+            // should not run into exception normally. In case of Exception, 
+            // simply fail the test and investigate
+        }
+    }
+    
+    /*
+     * retrieve the config value based on the input cluster, topic and config 
name
+     */
+    protected static String getTopicConfig(EmbeddedKafkaCluster cluster, 
String topic, String configName) {
+        Admin client = cluster.createAdminClient();
+        Collection<ConfigResource> cr =  Collections.singleton(
+                new ConfigResource(ConfigResource.Type.TOPIC, topic)); 
+        try {
+            DescribeConfigsResult configsResult = client.describeConfigs(cr);
+            Config allConfigs = (Config) 
configsResult.all().get().values().toArray()[0];
+            Iterator<ConfigEntry> configIterator = 
allConfigs.entries().iterator();
+            while (configIterator.hasNext()) {
+                ConfigEntry currentConfig = configIterator.next();     
+                if (currentConfig.name().equals(configName)) {
+                    return currentConfig.value();
+                }
+            }
+        } catch (Throwable e) {
+            // should not run into exception normally. In case of Exception, 

Review comment:
       catching Throwable has been here since the original author wrote MM2. It 
seems a tricky thing and I am happy to hear your suggestions on what to put in 
the "catch" block.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to