ning2008wisc commented on a change in pull request #9224: URL: https://github.com/apache/kafka/pull/9224#discussion_r535856935
########## File path: connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationBaseTest.java ########## @@ -0,0 +1,617 @@ +/* + * 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.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.ConsumerRecords; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.common.config.ConfigResource; +import org.apache.kafka.common.config.TopicConfig; +import org.apache.kafka.common.config.SslConfigs; +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.MirrorClient; +import org.apache.kafka.connect.mirror.MirrorHeartbeatConnector; +import org.apache.kafka.connect.mirror.MirrorMakerConfig; +import org.apache.kafka.connect.mirror.MirrorSourceConnector; +import org.apache.kafka.connect.mirror.SourceAndTarget; +import org.apache.kafka.connect.mirror.MirrorCheckpointConnector; +import org.apache.kafka.connect.util.clusters.EmbeddedConnectCluster; +import org.apache.kafka.connect.util.clusters.EmbeddedKafkaCluster; +import org.apache.kafka.test.IntegrationTest; +import static org.apache.kafka.test.TestUtils.waitForCondition; + +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.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.assertNotNull; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import static org.apache.kafka.connect.mirror.TestUtils.generateRecords; + +/** + * Tests MM2 replication and failover/failback logic. + * + * MM2 is configured with active/active replication between two Kafka clusters. Tests validate that + * records sent to either cluster arrive at the other cluster. Then, a consumer group is migrated from + * one cluster to the other and back. Tests validate that consumer offsets are translated and replicated + * between clusters during this failover and failback. + */ +@Category(IntegrationTest.class) +public abstract class MirrorConnectorsIntegrationBaseTest { + private static final Logger log = LoggerFactory.getLogger(MirrorConnectorsIntegrationBaseTest.class); + + private static final int NUM_RECORDS_PER_PARTITION = 10; + private static final int NUM_PARTITIONS = 10; + private static final int NUM_RECORDS_PRODUCED = NUM_PARTITIONS * NUM_RECORDS_PER_PARTITION; + private static final int RECORD_TRANSFER_DURATION_MS = 30_000; + private static final int CHECKPOINT_DURATION_MS = 20_000; + private static final int RECORD_CONSUME_DURATION_MS = 20_000; + private static final int OFFSET_SYNC_DURATION_MS = 30_000; + private static final int NUM_WORKERS = 3; + private static final int CONSUMER_POLL_TIMEOUT_MS = 500; + private static final int BROKER_RESTART_TIMEOUT_MS = 10_000; + private static final long DEFAULT_PRODUCE_SEND_DURATION_MS = TimeUnit.SECONDS.toMillis(120); + private static final String PRIMARY_CLUSTER_ALIAS = "primary"; + private static final String BACKUP_CLUSTER_ALIAS = "backup"; + private static final List<Class<? extends Connector>> CONNECTOR_LIST = + Arrays.asList(MirrorSourceConnector.class, MirrorCheckpointConnector.class, MirrorHeartbeatConnector.class); + + private Map<String, String> mm2Props; + private MirrorMakerConfig mm2Config; + private EmbeddedConnectCluster primary; + private EmbeddedConnectCluster backup; + + private final AtomicBoolean exited = new AtomicBoolean(false); + protected Properties primaryBrokerProps = new Properties(); + protected Properties backupBrokerProps = new Properties(); + private Map<String, String> primaryWorkerProps = new HashMap<>(); + private Map<String, String> backupWorkerProps = new HashMap<>(); + abstract Map<String, Object> getSslConfig(); + + protected void startClusters() throws InterruptedException { + primaryBrokerProps.put("auto.create.topics.enable", "false"); + backupBrokerProps.put("auto.create.topics.enable", "false"); + + mm2Props = basicMM2Config(); + + final Map<String, Object> sslConfig = getSslConfig(); + if (sslConfig != null) { + Properties sslProps = new Properties(); + sslProps.put(SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG, sslConfig.get(SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG)); + sslProps.put(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, ((Password) sslConfig.get(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG)).value()); + sslProps.put(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, "SSL"); + + // 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())))); + } + + 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(); + + warmUpConsumer(); + + 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(); + } + } + + @Test + public void testReplication() throws InterruptedException { + produceMessages(primary, "test-topic-1"); + produceMessages(backup, "test-topic-1"); + String consumerGroupName = "consumer-group-testReplication"; + Map<String, Object> consumerProps = new HashMap<String, Object>() {{ + put("group.id", consumerGroupName); + put("auto.offset.reset", "latest"); + }}; + // create consumers before starting the connectors so we don't need to wait for discovery + Consumer<byte[], byte[]> primaryConsumer = primary.kafka().createConsumerAndSubscribeTo(consumerProps, "test-topic-1"); + waitForConsumingAllRecords(primaryConsumer, 0); + + Consumer<byte[], byte[]> backupConsumer = backup.kafka().createConsumerAndSubscribeTo(consumerProps, "test-topic-1"); + waitForConsumingAllRecords(backupConsumer, 0); + + mm2Config = new MirrorMakerConfig(mm2Props); + + waitUntilMirrorMakerIsRunning(backup, CONNECTOR_LIST, mm2Config, PRIMARY_CLUSTER_ALIAS, BACKUP_CLUSTER_ALIAS); + waitUntilMirrorMakerIsRunning(primary, CONNECTOR_LIST, mm2Config, BACKUP_CLUSTER_ALIAS, PRIMARY_CLUSTER_ALIAS); + + MirrorClient primaryClient = new MirrorClient(mm2Config.clientConfig(PRIMARY_CLUSTER_ALIAS)); + MirrorClient backupClient = new MirrorClient(mm2Config.clientConfig(BACKUP_CLUSTER_ALIAS)); + + assertEquals("topic config was not synced", TopicConfig.CLEANUP_POLICY_COMPACT, + getTopicConfig(backup.kafka(), "primary.test-topic-1", TopicConfig.CLEANUP_POLICY_CONFIG)); + + assertEquals("Records were not produced to primary cluster.", NUM_RECORDS_PRODUCED, + primary.kafka().consume(NUM_RECORDS_PRODUCED, RECORD_TRANSFER_DURATION_MS, "test-topic-1").count()); + assertEquals("Records were not replicated to backup cluster.", NUM_RECORDS_PRODUCED, + backup.kafka().consume(NUM_RECORDS_PRODUCED, RECORD_TRANSFER_DURATION_MS, "primary.test-topic-1").count()); + assertEquals("Records were not produced to backup cluster.", NUM_RECORDS_PRODUCED, + backup.kafka().consume(NUM_RECORDS_PRODUCED, RECORD_TRANSFER_DURATION_MS, "test-topic-1").count()); + assertEquals("Records were not replicated to primary cluster.", NUM_RECORDS_PRODUCED, + primary.kafka().consume(NUM_RECORDS_PRODUCED, RECORD_TRANSFER_DURATION_MS, "backup.test-topic-1").count()); + + assertEquals("Primary cluster doesn't have all records from both clusters.", NUM_RECORDS_PRODUCED * 2, + primary.kafka().consume(NUM_RECORDS_PRODUCED * 2, RECORD_TRANSFER_DURATION_MS, "backup.test-topic-1", "test-topic-1").count()); + assertEquals("Backup cluster doesn't have all records from both clusters.", NUM_RECORDS_PRODUCED * 2, + backup.kafka().consume(NUM_RECORDS_PRODUCED * 2, RECORD_TRANSFER_DURATION_MS, "primary.test-topic-1", "test-topic-1").count()); + + assertTrue("Heartbeats were not emitted to primary cluster.", primary.kafka().consume(1, + RECORD_TRANSFER_DURATION_MS, "heartbeats").count() > 0); + assertTrue("Heartbeats were not emitted to backup cluster.", backup.kafka().consume(1, + RECORD_TRANSFER_DURATION_MS, "heartbeats").count() > 0); + assertTrue("Heartbeats were not replicated downstream to backup cluster.", backup.kafka().consume(1, + RECORD_TRANSFER_DURATION_MS, "primary.heartbeats").count() > 0); + assertTrue("Heartbeats were not replicated downstream to primary cluster.", primary.kafka().consume(1, + RECORD_TRANSFER_DURATION_MS, "backup.heartbeats").count() > 0); + + assertTrue("Did not find upstream primary cluster.", backupClient.upstreamClusters().contains(PRIMARY_CLUSTER_ALIAS)); + assertEquals("Did not calculate replication hops correctly.", 1, backupClient.replicationHops(PRIMARY_CLUSTER_ALIAS)); + assertTrue("Did not find upstream backup cluster.", primaryClient.upstreamClusters().contains(BACKUP_CLUSTER_ALIAS)); + assertEquals("Did not calculate replication hops correctly.", 1, primaryClient.replicationHops(BACKUP_CLUSTER_ALIAS)); + assertTrue("Checkpoints were not emitted downstream to backup cluster.", backup.kafka().consume(1, + CHECKPOINT_DURATION_MS, "primary.checkpoints.internal").count() > 0); + + Map<TopicPartition, OffsetAndMetadata> backupOffsets = backupClient.remoteConsumerOffsets(consumerGroupName, PRIMARY_CLUSTER_ALIAS, + Duration.ofMillis(CHECKPOINT_DURATION_MS)); + + assertTrue("Offsets not translated downstream to backup cluster. Found: " + backupOffsets, backupOffsets.containsKey( + new TopicPartition("primary.test-topic-1", 0))); + + // Failover consumer group to backup cluster. + primaryConsumer = backup.kafka().createConsumer(Collections.singletonMap("group.id", consumerGroupName)); + primaryConsumer.assign(backupOffsets.keySet()); + backupOffsets.forEach(primaryConsumer::seek); + primaryConsumer.poll(Duration.ofMillis(CONSUMER_POLL_TIMEOUT_MS)); + primaryConsumer.commitAsync(); + + assertTrue("Consumer failedover to zero offset.", primaryConsumer.position(new TopicPartition("primary.test-topic-1", 0)) > 0); + assertTrue("Consumer failedover beyond expected offset.", primaryConsumer.position( + new TopicPartition("primary.test-topic-1", 0)) <= NUM_RECORDS_PRODUCED); + assertTrue("Checkpoints were not emitted upstream to primary cluster.", primary.kafka().consume(1, + CHECKPOINT_DURATION_MS, "backup.checkpoints.internal").count() > 0); + + primaryConsumer.close(); Review comment: yeah - I think we can do "try with resource block", updated ---------------------------------------------------------------- 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]
