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



##########
File path: 
connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationTest.java
##########
@@ -0,0 +1,370 @@
+/*
+ * 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.consumer.Consumer;
+import org.apache.kafka.clients.consumer.ConsumerRecords;
+import org.apache.kafka.clients.consumer.OffsetAndMetadata;
+import org.apache.kafka.common.config.TopicConfig;
+import org.apache.kafka.common.errors.TimeoutException;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.connect.connector.Connector;
+import org.apache.kafka.connect.mirror.MirrorCheckpointConnector;
+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.utils.ThreadedConsumer;
+import org.apache.kafka.connect.mirror.utils.ThreadedProducer;
+
+import org.junit.Before;
+import org.junit.After;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertNotNull;
+
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Collections;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.kafka.test.TestUtils.waitForCondition;
+import org.apache.kafka.test.IntegrationTest;
+
+/**
+ * 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 class MirrorConnectorsIntegrationTest extends 
MirrorConnectorsIntegrationBaseTest {
+    
+    private static final List<Class<? extends Connector>> CONNECTOR_LIST = 
+            Arrays.asList(MirrorSourceConnector.class, 
MirrorCheckpointConnector.class, MirrorHeartbeatConnector.class);
+    
+    @Before
+    public void setup() throws InterruptedException {
+        startClusters();
+    }
+
+    @After
+    public void close() {
+        shutdownClusters();
+    }
+
+    @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();
+
+        waitForCondition(() -> {
+            try {
+                return primaryClient.remoteConsumerOffsets(consumerGroupName, 
BACKUP_CLUSTER_ALIAS,
+                    Duration.ofMillis(CHECKPOINT_DURATION_MS)).containsKey(new 
TopicPartition("backup.test-topic-1", 0));
+            } catch (Throwable e) {
+                return false;
+            }
+        }, CHECKPOINT_DURATION_MS, "Offsets not translated downstream to 
primary cluster.");
+
+        waitForCondition(() -> {
+            try {
+                return primaryClient.remoteConsumerOffsets(consumerGroupName, 
BACKUP_CLUSTER_ALIAS,
+                    Duration.ofMillis(CHECKPOINT_DURATION_MS)).containsKey(new 
TopicPartition("test-topic-1", 0));
+            } catch (Throwable e) {
+                return false;
+            }
+        }, CHECKPOINT_DURATION_MS, "Offsets not translated upstream to primary 
cluster.");
+
+        Map<TopicPartition, OffsetAndMetadata> primaryOffsets = 
primaryClient.remoteConsumerOffsets(consumerGroupName, BACKUP_CLUSTER_ALIAS,
+                Duration.ofMillis(CHECKPOINT_DURATION_MS));
+ 
+        // Failback consumer group to primary cluster
+        backupConsumer = 
primary.kafka().createConsumer(Collections.singletonMap("group.id", 
consumerGroupName));
+        backupConsumer.assign(primaryOffsets.keySet());
+        primaryOffsets.forEach(backupConsumer::seek);
+        backupConsumer.poll(Duration.ofMillis(CONSUMER_POLL_TIMEOUT_MS));
+        backupConsumer.commitAsync();
+        
+        assertTrue("Consumer failedback to zero upstream offset.", 
backupConsumer.position(new TopicPartition("test-topic-1", 0)) > 0);
+        assertTrue("Consumer failedback to zero downstream offset.", 
backupConsumer.position(new TopicPartition("backup.test-topic-1", 0)) > 0);
+        assertTrue("Consumer failedback beyond expected upstream offset.", 
backupConsumer.position(
+            new TopicPartition("test-topic-1", 0)) <= NUM_RECORDS_PRODUCED);
+        assertTrue("Consumer failedback beyond expected downstream offset.", 
backupConsumer.position(
+            new TopicPartition("backup.test-topic-1", 0)) <= 
NUM_RECORDS_PRODUCED);
+        
+        backupConsumer.close();
+      
+        // create more matching topics
+        primary.kafka().createTopic("test-topic-2", NUM_PARTITIONS);
+        backup.kafka().createTopic("test-topic-3", NUM_PARTITIONS);
+
+        // only produce messages to the first partition
+        produceMessages(primary, "test-topic-2", 1);
+        produceMessages(backup, "test-topic-3", 1);
+        
+        // expect total consumed messages equals to NUM_RECORDS_PER_PARTITION
+        assertEquals("Records were not produced to primary cluster.", 
NUM_RECORDS_PER_PARTITION,
+            primary.kafka().consume(NUM_RECORDS_PER_PARTITION, 
RECORD_TRANSFER_DURATION_MS, "test-topic-2").count());
+        assertEquals("Records were not produced to backup cluster.", 
NUM_RECORDS_PER_PARTITION,
+            backup.kafka().consume(NUM_RECORDS_PER_PARTITION, 
RECORD_TRANSFER_DURATION_MS, "test-topic-3").count());
+
+        assertEquals("New topic was not replicated to primary cluster.", 
NUM_RECORDS_PER_PARTITION,
+            primary.kafka().consume(NUM_RECORDS_PER_PARTITION, 2 * 
RECORD_TRANSFER_DURATION_MS, "backup.test-topic-3").count());
+        assertEquals("New topic was not replicated to backup cluster.", 
NUM_RECORDS_PER_PARTITION,
+            backup.kafka().consume(NUM_RECORDS_PER_PARTITION, 2 * 
RECORD_TRANSFER_DURATION_MS, "primary.test-topic-2").count());
+
+    }
+    
+    @Test
+    public void testReplicationWithEmptyPartition() throws Exception {
+        String consumerGroupName = 
"consumer-group-testReplicationWithEmptyPartition";
+        Map<String, Object> consumerProps  = 
Collections.singletonMap("group.id", consumerGroupName);
+
+        // create topic
+        String topic = "test-topic-with-empty-partition";
+        primary.kafka().createTopic(topic, NUM_PARTITIONS);
+
+        // produce to all test-topic-empty's partitions, except the last 
partition
+        produceMessages(primary, topic, NUM_PARTITIONS - 1);
+        
+        // consume before starting the connectors so we don't need to wait for 
discovery
+        int expectedRecords = NUM_RECORDS_PER_PARTITION * (NUM_PARTITIONS - 1);
+        try (Consumer<byte[], byte[]> primaryConsumer = 
primary.kafka().createConsumerAndSubscribeTo(consumerProps, topic)) {
+            waitForConsumingAllRecords(primaryConsumer, expectedRecords);
+        }
+        
+        // one way replication from primary to backup
+        mm2Props.put(BACKUP_CLUSTER_ALIAS + "->" + PRIMARY_CLUSTER_ALIAS + 
".enabled", "false");
+        mm2Config = new MirrorMakerConfig(mm2Props);
+        waitUntilMirrorMakerIsRunning(backup, CONNECTOR_LIST, mm2Config, 
PRIMARY_CLUSTER_ALIAS, BACKUP_CLUSTER_ALIAS);
+        
+        // sleep few seconds to have MM2 finish replication so that "end" 
consumer will consume some record
+        Thread.sleep(TimeUnit.SECONDS.toMillis(3));
+
+        // consume all records from backup cluster
+        try (Consumer<byte[], byte[]> backupConsumer = 
backup.kafka().createConsumerAndSubscribeTo(consumerProps, 
+                PRIMARY_CLUSTER_ALIAS + "." + topic)) {
+            waitForConsumingAllRecords(backupConsumer, expectedRecords);
+        }
+        
+        Admin backupClient = backup.kafka().createAdminClient();
+        // retrieve the consumer group offset from backup cluster
+        Map<TopicPartition, OffsetAndMetadata> remoteOffsets =
+                
backupClient.listConsumerGroupOffsets(consumerGroupName).partitionsToOffsetAndMetadata().get();
+        // pinpoint the offset of the last partition which does not receive 
records 
+        OffsetAndMetadata offset = remoteOffsets.get(new 
TopicPartition(PRIMARY_CLUSTER_ALIAS + "." + topic, NUM_PARTITIONS - 1));
+        // offset of the last partition should exist, but its value should be 0
+        assertNotNull("Offset of last partition was not replicated", offset);
+        assertEquals("Offset of last partition is not zero", 0, 
offset.offset());
+    }
+    
+    @Test
+    public void testOneWayReplicationWithAutoOffsetSync() throws 
InterruptedException {
+        produceMessages(primary, "test-topic-1");
+        String consumerGroupName = 
"consumer-group-testOneWayReplicationWithAutoOffsetSync";
+        Map<String, Object> consumerProps  = new HashMap<String, Object>() {{
+                put("group.id", consumerGroupName);
+                put("auto.offset.reset", "earliest");
+            }};
+        // create consumers before starting the connectors so we don't need to 
wait for discovery
+        try (Consumer<byte[], byte[]> primaryConsumer = 
primary.kafka().createConsumerAndSubscribeTo(consumerProps, 
+                "test-topic-1")) {
+            // we need to wait for consuming all the records for MM2 
replicating the expected offsets
+            waitForConsumingAllRecords(primaryConsumer, NUM_RECORDS_PRODUCED);
+        }
+
+        // enable automated consumer group offset sync
+        mm2Props.put("sync.group.offsets.enabled", "true");
+        mm2Props.put("sync.group.offsets.interval.seconds", "1");
+        // one way replication from primary to backup
+        mm2Props.put(BACKUP_CLUSTER_ALIAS + "->" + PRIMARY_CLUSTER_ALIAS + 
".enabled", "false");
+
+        mm2Config = new MirrorMakerConfig(mm2Props);
+
+        waitUntilMirrorMakerIsRunning(backup, CONNECTOR_LIST, mm2Config, 
PRIMARY_CLUSTER_ALIAS, BACKUP_CLUSTER_ALIAS);
+
+        // create a consumer at backup cluster with same consumer group Id to 
consume 1 topic
+        Consumer<byte[], byte[]> backupConsumer = 
backup.kafka().createConsumerAndSubscribeTo(
+            consumerProps, "primary.test-topic-1");
+
+        waitForConsumerGroupOffsetSync(backup, backupConsumer, 
Collections.singletonList("primary.test-topic-1"), 
+            consumerGroupName, NUM_RECORDS_PRODUCED);
+
+        ConsumerRecords<byte[], byte[]> records = 
backupConsumer.poll(Duration.ofMillis(CONSUMER_POLL_TIMEOUT_MS));
+
+        // the size of consumer record should be zero, because the offsets of 
the same consumer group
+        // have been automatically synchronized from primary to backup by the 
background job, so no
+        // more records to consume from the replicated topic by the same 
consumer group at backup cluster
+        assertEquals("consumer record size is not zero", 0, records.count());
+
+        // now create a new topic in primary cluster
+        primary.kafka().createTopic("test-topic-2", NUM_PARTITIONS);
+        backup.kafka().createTopic("primary.test-topic-2", 1);
+        // produce some records to the new topic in primary cluster
+        produceMessages(primary, "test-topic-2");
+
+        // create a consumer at primary cluster to consume the new topic
+        try (Consumer<byte[], byte[]> consumer1 = 
primary.kafka().createConsumerAndSubscribeTo(Collections.singletonMap(
+                "group.id", "consumer-group-1"), "test-topic-2")) {
+            // we need to wait for consuming all the records for MM2 
replicating the expected offsets
+            waitForConsumingAllRecords(consumer1, NUM_RECORDS_PRODUCED);
+        }
+
+        // create a consumer at backup cluster with same consumer group Id to 
consume old and new topic
+        backupConsumer = 
backup.kafka().createConsumerAndSubscribeTo(Collections.singletonMap(
+            "group.id", consumerGroupName), "primary.test-topic-1", 
"primary.test-topic-2");
+
+        waitForConsumerGroupOffsetSync(backup, backupConsumer, 
Arrays.asList("primary.test-topic-1", "primary.test-topic-2"), 
+            consumerGroupName, NUM_RECORDS_PRODUCED);
+
+        records = 
backupConsumer.poll(Duration.ofMillis(CONSUMER_POLL_TIMEOUT_MS));
+        // similar reasoning as above, no more records to consume by the same 
consumer group at backup cluster
+        assertEquals("consumer record size is not zero", 0, records.count());
+        backupConsumer.close();
+    }
+    
+    /*
+     * This test is to validate MirrorSourceConnector follows "at most once" 
delivery guarantee
+     * under broker restart / failure
+     */
+    @Test
+    public void testWithBrokerRestart() throws InterruptedException {

Review comment:
       > Why do we need background clients instead of producing upfront and 
consuming the data mirrorred at the end of the test?
   
   I think I answered the above question as the comments  in the code. Please 
see below,
   
   > // the purpose of background producer and consumer is to better test the 
failure case
   > // to avoid serialization (produce -> broker restart -> consumer) by 
decoupling the
   > // producer, embedded kafka and consumer. Since the consumer offsets are 
stored in
   > // kafka topic at backup cluster and offset commit is periodic, when kafka 
broker (at backup)
   > // restarts, duplicate records will be consumed, meaning "at least once" 
delivery guarantee
   
   I believe this test has unique value and is more close to the realistic 
scenario, since the producer -> kafka -> consumer are always running on 
different machines, rather than one process or thread.
   
   Due to the complexity limitation, I am happy to remove this test for now




----------------------------------------------------------------
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