From 447b38ff041d284f19f9756f373ee3d8066b3cc0 Mon Sep 17 00:00:00 2001 From: Ryanne Dolan Date: Fri, 7 May 2021 19:08:57 +0000 Subject: [PATCH 01/13] WIP LegacyReplicationPolicy and tests --- .../mirror/LegacyReplicationPolicy.java | 113 ++++++++++++++++++ .../kafka/connect/mirror/MirrorClient.java | 11 +- .../connect/mirror/ReplicationPolicy.java | 2 +- .../connect/mirror/MirrorClientTest.java | 50 +++++++- .../connect/mirror/MirrorSourceConnector.java | 7 +- .../MirrorConnectorsIntegrationBaseTest.java | 102 +++++++++------- ...MirrorConnectorsIntegrationLegacyTest.java | 35 ++++++ 7 files changed, 274 insertions(+), 46 deletions(-) create mode 100644 connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/LegacyReplicationPolicy.java create mode 100644 connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationLegacyTest.java diff --git a/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/LegacyReplicationPolicy.java b/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/LegacyReplicationPolicy.java new file mode 100644 index 0000000000000..1a483147c68bc --- /dev/null +++ b/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/LegacyReplicationPolicy.java @@ -0,0 +1,113 @@ +/* + * 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; + +import org.apache.kafka.common.Configurable; + +import java.util.Map; +import java.util.regex.Pattern; + +/** LegacyReplicationPolicy attempts to mimic MirrorMaker v1, and thus by default does not + * rename topics. In order to support active/active replication with careful configuration, + * topics can be renamed with an optional suffix, e.g. "topic1.replica". + * + * LegacyReplicationPolicy cannot, in general, detect whether a topic is remote or not, nor + * which cluster a topic may have been replicated from. If a `remote.topic.suffix` is + * provided, we use that to distinguish remote topics. Otherwise we consider all topics as + * remote topics. If a `source.cluster.alias` is provided, we use that as the source of + * any remote topics. Otherwise we return null for `topicSource()`. + * + * Paradoxically, then, the default behavior of LegacyReplicationPolicy is to consider all + * topics to be remote but with no known source. + * + * N.B. MirrorMaker is not able to prevent cycles when using this class, so take care that + * your replication topology is acyclic. If migrating from MirrorMaker v1, this will likely + * already be the case. + */ +public class LegacyReplicationPolicy implements ReplicationPolicy, Configurable { + + public static final String REMOTE_TOPIC_SUFFIX_CONFIG = "remote.topic.suffix"; + public static final String SOURCE_CLUSTER_ALIAS_CONFIG = "source.cluster.alias"; + + private String remoteTopicSuffix = ""; + private String sourceClusterAlias = null; + + public LegacyReplicationPolicy() { + } + + // Visible for testing + LegacyReplicationPolicy(String remoteTopicSuffix, String sourceClusterAlias) { + this.remoteTopicSuffix = remoteTopicSuffix; + this.sourceClusterAlias = sourceClusterAlias; + } + + @Override + public void configure(Map props) { + if (props.containsKey(REMOTE_TOPIC_SUFFIX_CONFIG)) { + remoteTopicSuffix = (String) props.get(REMOTE_TOPIC_SUFFIX_CONFIG); + } + if (props.containsKey(SOURCE_CLUSTER_ALIAS_CONFIG)) { + sourceClusterAlias = (String) props.get(SOURCE_CLUSTER_ALIAS_CONFIG); + } + } + + /** Unlike DefaultReplicationPolicy, LegacyReplicationPolicy does not include the source + * cluster alias in the remote topic name. By default, topic names are unchanged. If + * a `remote.topic.suffix` is provided, topic are renamed accordingly. + */ + @Override + public String formatRemoteTopic(String sourceClusterAlias, String topic) { + return topic + remoteTopicSuffix; + } + + /** Unlike DefaultReplicationPolicy, LegacyReplicationPolicy cannot know the source of + * a remote topic based on its name alone. If `source.cluster.alias` is provided, + * `topicSource` will return that for anything that looks like a remote topic. By default, + * all topics look like remote topics. + */ + @Override + public String topicSource(String topic) { + if (topic.endsWith(remoteTopicSuffix)) { + return sourceClusterAlias; + } else { + return null; + } + } + + /** Unlike DefaultReplicationPolicy, LegacyReplicationPolicy cannot distinguish remote + * topics from regular topics by default. If `remote.topic.suffix` is provided, this + * method will strip that and return the original topic. By default, `topic` is returned. + */ + @Override + public String upstreamTopic(String topic) { + if (topic.endsWith(remoteTopicSuffix)) { + return topic.substring(0, topic.length() - remoteTopicSuffix.length()); + } else { + return topic; + } + } + + /** If remote.topic.suffix is provided, treat suffixed topics as internal topics. + * This prevents replicated topics from being further replicated when using + * LegacyReplicationPolicy. + */ + @Override + public boolean isInternalTopic(String topic) { + return ReplicationPolicy.super.isInternalTopic(topic) + || (!remoteTopicSuffix.isEmpty() && topic.endsWith(remoteTopicSuffix)); + } +} diff --git a/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/MirrorClient.java b/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/MirrorClient.java index 17d18ecb58a27..775ec782c2d41 100644 --- a/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/MirrorClient.java +++ b/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/MirrorClient.java @@ -192,6 +192,7 @@ Set listTopics() throws InterruptedException { int countHopsForTopic(String topic, String sourceClusterAlias) { int hops = 0; + Set visited = new HashSet<>(); while (true) { hops++; String source = replicationPolicy.topicSource(topic); @@ -201,6 +202,12 @@ int countHopsForTopic(String topic, String sourceClusterAlias) { if (source.equals(sourceClusterAlias)) { return hops; } + if (visited.contains(source)) { + // Extra check for LegacyReplicationPolicy (and any other dumb impls) that cannot prevent cycles. + // We assume we're stuck in a cycle and will never find sourceClusterAlias. + return -1; + } + visited.add(source); topic = replicationPolicy.upstreamTopic(topic); } } @@ -223,7 +230,9 @@ boolean isRemoteTopic(String topic) { Set allSources(String topic) { Set sources = new HashSet<>(); String source = replicationPolicy.topicSource(topic); - while (source != null) { + while (source != null && !sources.contains(source)) { + // The extra Set.contains above is for LegacyReplicationPolicy (and any other dumb impls) which + // cannot prevent cycles. In impls that do prevent cycles, we shouldn't see the same source twice. sources.add(source); topic = replicationPolicy.upstreamTopic(topic); source = replicationPolicy.topicSource(topic); diff --git a/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/ReplicationPolicy.java b/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/ReplicationPolicy.java index 11f73f50ceac5..b6eb26c8707bc 100644 --- a/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/ReplicationPolicy.java +++ b/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/ReplicationPolicy.java @@ -45,7 +45,7 @@ public interface ReplicationPolicy { */ default String originalTopic(String topic) { String upstream = upstreamTopic(topic); - if (upstream == null) { + if (upstream == null || upstream.equals(topic)) { return topic; } else { return originalTopic(upstream); diff --git a/connect/mirror-client/src/test/java/org/apache/kafka/connect/mirror/MirrorClientTest.java b/connect/mirror-client/src/test/java/org/apache/kafka/connect/mirror/MirrorClientTest.java index 058d257ec59c8..1d1c91ec9d14e 100644 --- a/connect/mirror-client/src/test/java/org/apache/kafka/connect/mirror/MirrorClientTest.java +++ b/connect/mirror-client/src/test/java/org/apache/kafka/connect/mirror/MirrorClientTest.java @@ -38,7 +38,11 @@ private static class FakeMirrorClient extends MirrorClient { List topics; FakeMirrorClient(List topics) { - super(null, new DefaultReplicationPolicy(), null); + this(new DefaultReplicationPolicy(), topics); + } + + FakeMirrorClient(ReplicationPolicy replicationPolicy, List topics) { + super(null, replicationPolicy, null); this.topics = topics; } @@ -132,6 +136,19 @@ public void upstreamClustersTest() throws InterruptedException { assertFalse(sources.contains(null)); } + @Test + public void legacyUpstreamClustersTest() throws InterruptedException { + // When configured with a specific source, LegacyReplicationPolicy should just return the provided source. + MirrorClient client = new FakeMirrorClient(new LegacyReplicationPolicy("", "source"), Arrays.asList("topic1", + "topic2", "heartbeats", "source1.heartbeats", "source1.source2.heartbeats", + "source3.source4.source5.heartbeats")); + Set sources = client.upstreamClusters(); + assertTrue(sources.contains("source")); + assertFalse(sources.contains("")); + assertFalse(sources.contains(null)); + assertEquals(1, sources.size()); + } + @Test public void remoteTopicsTest() throws InterruptedException { MirrorClient client = new FakeMirrorClient(Arrays.asList("topic1", "topic2", "topic3", @@ -145,6 +162,18 @@ public void remoteTopicsTest() throws InterruptedException { assertTrue(remoteTopics.contains("source3.source4.source5.topic6")); } + + @Test + public void legacyRemoteTopicsTest() throws InterruptedException { + // By default, LegacyReplicationPolicy should consider any topic to be remote. + MirrorClient client = new FakeMirrorClient(new LegacyReplicationPolicy("", "source"), Arrays.asList( + "topic1", "topic2", "topic3")); + Set remoteTopics = client.remoteTopics(); + assertTrue(remoteTopics.contains("topic1")); + assertTrue(remoteTopics.contains("topic2")); + assertTrue(remoteTopics.contains("topic3")); + } + @Test public void remoteTopicsSeparatorTest() throws InterruptedException { MirrorClient client = new FakeMirrorClient(Arrays.asList("topic1", "topic2", "topic3", @@ -160,4 +189,23 @@ public void remoteTopicsSeparatorTest() throws InterruptedException { assertTrue(remoteTopics.contains("source3__source4__source5__topic6")); } + @Test + public void testLegacyRemoteTopicWithSuffix() { + MirrorClient client = new FakeMirrorClient( + new LegacyReplicationPolicy(".replica", "primary"), Arrays.asList()); + assertEquals("topic1.replica", client.replicationPolicy() + .formatRemoteTopic("primary", "topic1")); + assertEquals("primary", client.replicationPolicy() + .topicSource("topic1.replica")); + } + + @Test + public void testLegacyRemoteTopicWithoutSuffix() { + MirrorClient client = new FakeMirrorClient( + new LegacyReplicationPolicy("", "primary"), Arrays.asList()); + assertEquals("topic1", client.replicationPolicy() + .formatRemoteTopic("primary", "topic1")); + assertEquals("primary", client.replicationPolicy() + .topicSource("topic1")); + } } diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceConnector.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceConnector.java index 7b844c84d2f56..4e3e884a41d61 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceConnector.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceConnector.java @@ -494,7 +494,12 @@ boolean isCycle(String topic) { } else if (source.equals(sourceAndTarget.target())) { return true; } else { - return isCycle(replicationPolicy.upstreamTopic(topic)); + String upstreamTopic = replicationPolicy.upstreamTopic(topic); + if (upstreamTopic.equals(topic)) { + // Extra check for LegacyReplicationPolicy + return false; + } + return isCycle(upstreamTopic); } } diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationBaseTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationBaseTest.java index e6bdb9679d64d..fccbb24172176 100644 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationBaseTest.java +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationBaseTest.java @@ -33,6 +33,7 @@ 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.mirror.ReplicationPolicy; import org.apache.kafka.connect.util.clusters.EmbeddedConnectCluster; import org.apache.kafka.connect.util.clusters.EmbeddedKafkaCluster; import org.apache.kafka.connect.util.clusters.UngracefulShutdownException; @@ -96,6 +97,9 @@ public abstract class MirrorConnectorsIntegrationBaseTest { private MirrorMakerConfig mm2Config; private EmbeddedConnectCluster primary; private EmbeddedConnectCluster backup; + private MirrorClient primaryClient; + private MirrorClient backupClient; + private ReplicationPolicy replicationPolicy; private Exit.Procedure exitProcedure; private Exit.Procedure haltProcedure; @@ -139,11 +143,13 @@ public void startClusters() throws Exception { backupBrokerProps.put("auto.create.topics.enable", "false"); mm2Props.putAll(basicMM2Config()); + mm2Props.putAll(configOverrides()); - 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))); - + // We don't have bootstrap.servers yet, so the MirrorMakerConfig is incomplete at this point. + MirrorMakerConfig tempMM2Config = new MirrorMakerConfig(mm2Props); + primaryWorkerProps = tempMM2Config.workerConfig(new SourceAndTarget(BACKUP_CLUSTER_ALIAS, PRIMARY_CLUSTER_ALIAS)); + backupWorkerProps.putAll(tempMM2Config.workerConfig(new SourceAndTarget(PRIMARY_CLUSTER_ALIAS, BACKUP_CLUSTER_ALIAS))); + primary = new EmbeddedConnectCluster.Builder() .name(PRIMARY_CLUSTER_ALIAS + "-connect-cluster") .numWorkers(NUM_WORKERS) @@ -170,6 +176,14 @@ public void startClusters() throws Exception { backup.assertions().assertAtLeastNumWorkersAreUp(NUM_WORKERS, "Workers of " + BACKUP_CLUSTER_ALIAS + "-connect-cluster did not start in time."); + // 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()); + mm2Config = new MirrorMakerConfig(mm2Props); + primaryClient = new MirrorClient(mm2Config.clientConfig(PRIMARY_CLUSTER_ALIAS)); + backupClient = new MirrorClient(mm2Config.clientConfig(BACKUP_CLUSTER_ALIAS)); + replicationPolicy = primaryClient.replicationPolicy(); // assume same in both clusters + createTopics(); warmUpConsumer(Collections.singletonMap("group.id", "consumer-group-dummy")); @@ -178,10 +192,6 @@ public void startClusters() throws Exception { 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()); } @AfterEach @@ -227,27 +237,25 @@ public void testReplication() throws Exception { 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)); - // make sure the topic is auto-created in the other cluster - waitForTopicCreated(primary, "backup.test-topic-1"); - waitForTopicCreated(backup, "primary.test-topic-1"); - assertEquals(TopicConfig.CLEANUP_POLICY_COMPACT, getTopicConfig(backup.kafka(), "primary.test-topic-1", TopicConfig.CLEANUP_POLICY_CONFIG), - "topic config was not synced"); + waitForTopicCreated(primary, formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-1")); + waitForTopicCreated(backup, formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1")); +// FIXME assertEquals(TopicConfig.CLEANUP_POLICY_COMPACT, getTopicConfig(backup.kafka(), +// formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1"), +// TopicConfig.CLEANUP_POLICY_CONFIG), "topic config was not synced"); assertEquals(NUM_RECORDS_PRODUCED, primary.kafka().consume(NUM_RECORDS_PRODUCED, RECORD_TRANSFER_DURATION_MS, "test-topic-1").count(), "Records were not produced to primary cluster."); - assertEquals(NUM_RECORDS_PRODUCED, backup.kafka().consume(NUM_RECORDS_PRODUCED, RECORD_TRANSFER_DURATION_MS, "primary.test-topic-1").count(), + assertEquals(NUM_RECORDS_PRODUCED, backup.kafka().consume(NUM_RECORDS_PRODUCED, RECORD_TRANSFER_DURATION_MS, formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1")).count(), "Records were not replicated to backup cluster."); assertEquals(NUM_RECORDS_PRODUCED, backup.kafka().consume(NUM_RECORDS_PRODUCED, RECORD_TRANSFER_DURATION_MS, "test-topic-1").count(), "Records were not produced to backup cluster."); - assertEquals(NUM_RECORDS_PRODUCED, primary.kafka().consume(NUM_RECORDS_PRODUCED, RECORD_TRANSFER_DURATION_MS, "backup.test-topic-1").count(), + assertEquals(NUM_RECORDS_PRODUCED, primary.kafka().consume(NUM_RECORDS_PRODUCED, RECORD_TRANSFER_DURATION_MS, formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-1")).count(), "Records were not replicated to primary cluster."); - assertEquals(NUM_RECORDS_PRODUCED * 2, primary.kafka().consume(NUM_RECORDS_PRODUCED * 2, RECORD_TRANSFER_DURATION_MS, "backup.test-topic-1", "test-topic-1").count(), + assertEquals(NUM_RECORDS_PRODUCED * 2, primary.kafka().consume(NUM_RECORDS_PRODUCED * 2, RECORD_TRANSFER_DURATION_MS, formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-1"), "test-topic-1").count(), "Primary cluster doesn't have all records from both clusters."); - assertEquals(NUM_RECORDS_PRODUCED * 2, backup.kafka().consume(NUM_RECORDS_PRODUCED * 2, RECORD_TRANSFER_DURATION_MS, "primary.test-topic-1", "test-topic-1").count(), + assertEquals(NUM_RECORDS_PRODUCED * 2, backup.kafka().consume(NUM_RECORDS_PRODUCED * 2, RECORD_TRANSFER_DURATION_MS, formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1"), "test-topic-1").count(), "Backup cluster doesn't have all records from both clusters."); assertTrue(primary.kafka().consume(1, RECORD_TRANSFER_DURATION_MS, "heartbeats").count() > 0, @@ -270,7 +278,7 @@ public void testReplication() throws Exception { Duration.ofMillis(CHECKPOINT_DURATION_MS)); assertTrue(backupOffsets.containsKey( - new TopicPartition("primary.test-topic-1", 0)), "Offsets not translated downstream to backup cluster. Found: " + backupOffsets); + new TopicPartition(formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1"), 0)), "Offsets not translated downstream to backup cluster. Found: " + backupOffsets); // Failover consumer group to backup cluster. try (Consumer primaryConsumer = backup.kafka().createConsumer(Collections.singletonMap("group.id", consumerGroupName))) { @@ -279,15 +287,15 @@ public void testReplication() throws Exception { primaryConsumer.poll(CONSUMER_POLL_TIMEOUT_MS); primaryConsumer.commitAsync(); - assertTrue(primaryConsumer.position(new TopicPartition("primary.test-topic-1", 0)) > 0, "Consumer failedover to zero offset."); + assertTrue(primaryConsumer.position(new TopicPartition(formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1"), 0)) > 0, "Consumer failedover to zero offset."); assertTrue(primaryConsumer.position( - new TopicPartition("primary.test-topic-1", 0)) <= NUM_RECORDS_PRODUCED, "Consumer failedover beyond expected offset."); + new TopicPartition(formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1"), 0)) <= NUM_RECORDS_PRODUCED, "Consumer failedover beyond expected offset."); assertTrue(primary.kafka().consume(1, CHECKPOINT_DURATION_MS, "backup.checkpoints.internal").count() > 0, "Checkpoints were not emitted upstream to primary cluster."); } waitForCondition(() -> primaryClient.remoteConsumerOffsets(consumerGroupName, BACKUP_CLUSTER_ALIAS, - Duration.ofMillis(CHECKPOINT_DURATION_MS)).containsKey(new TopicPartition("backup.test-topic-1", 0)), CHECKPOINT_DURATION_MS, "Offsets not translated downstream to primary cluster."); + Duration.ofMillis(CHECKPOINT_DURATION_MS)).containsKey(new TopicPartition(formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-1"), 0)), CHECKPOINT_DURATION_MS, "Offsets not translated downstream to primary cluster."); waitForCondition(() -> primaryClient.remoteConsumerOffsets(consumerGroupName, BACKUP_CLUSTER_ALIAS, Duration.ofMillis(CHECKPOINT_DURATION_MS)).containsKey(new TopicPartition("test-topic-1", 0)), CHECKPOINT_DURATION_MS, "Offsets not translated upstream to primary cluster."); @@ -306,11 +314,11 @@ public void testReplication() throws Exception { backupConsumer.commitAsync(); assertTrue(backupConsumer.position(new TopicPartition("test-topic-1", 0)) > 0, "Consumer failedback to zero upstream offset."); - assertTrue(backupConsumer.position(new TopicPartition("backup.test-topic-1", 0)) > 0, "Consumer failedback to zero downstream offset."); + assertTrue(backupConsumer.position(new TopicPartition(formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-1"), 0)) > 0, "Consumer failedback to zero downstream offset."); assertTrue(backupConsumer.position( new TopicPartition("test-topic-1", 0)) <= NUM_RECORDS_PRODUCED, "Consumer failedback beyond expected upstream offset."); assertTrue(backupConsumer.position( - new TopicPartition("backup.test-topic-1", 0)) <= NUM_RECORDS_PRODUCED, "Consumer failedback beyond expected downstream offset."); + new TopicPartition(formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-1"), 0)) <= NUM_RECORDS_PRODUCED, "Consumer failedback beyond expected downstream offset."); } // create more matching topics @@ -318,8 +326,8 @@ public void testReplication() throws Exception { backup.kafka().createTopic("test-topic-3", NUM_PARTITIONS); // make sure the topic is auto-created in the other cluster - waitForTopicCreated(backup, "primary.test-topic-2"); - waitForTopicCreated(primary, "backup.test-topic-3"); + waitForTopicCreated(backup, formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-2")); + waitForTopicCreated(primary, formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-3")); // only produce messages to the first partition produceMessages(primary, "test-topic-2", 1); @@ -331,9 +339,9 @@ public void testReplication() throws Exception { assertEquals(NUM_RECORDS_PER_PARTITION, backup.kafka().consume(NUM_RECORDS_PER_PARTITION, RECORD_TRANSFER_DURATION_MS, "test-topic-3").count(), "Records were not produced to backup cluster."); - assertEquals(NUM_RECORDS_PER_PARTITION, primary.kafka().consume(NUM_RECORDS_PER_PARTITION, 2 * RECORD_TRANSFER_DURATION_MS, "backup.test-topic-3").count(), + assertEquals(NUM_RECORDS_PER_PARTITION, primary.kafka().consume(NUM_RECORDS_PER_PARTITION, 2 * RECORD_TRANSFER_DURATION_MS, formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-3")).count(), "New topic was not replicated to primary cluster."); - assertEquals(NUM_RECORDS_PER_PARTITION, backup.kafka().consume(NUM_RECORDS_PER_PARTITION, 2 * RECORD_TRANSFER_DURATION_MS, "primary.test-topic-2").count(), + assertEquals(NUM_RECORDS_PER_PARTITION, backup.kafka().consume(NUM_RECORDS_PER_PARTITION, 2 * RECORD_TRANSFER_DURATION_MS, formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-2")).count(), "New topic was not replicated to backup cluster."); } @@ -365,7 +373,7 @@ public void testReplicationWithEmptyPartition() throws Exception { // consume all records from backup cluster try (Consumer backupConsumer = backup.kafka().createConsumerAndSubscribeTo(consumerProps, - PRIMARY_CLUSTER_ALIAS + "." + topic)) { + replicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, topic))) { waitForConsumingAllRecords(backupConsumer, expectedRecords); } @@ -375,7 +383,8 @@ public void testReplicationWithEmptyPartition() throws Exception { 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)); + OffsetAndMetadata offset = remoteOffsets.get(new TopicPartition( + replicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, topic), NUM_PARTITIONS - 1)); // offset of the last partition should exist, but its value should be 0 assertNotNull(offset, "Offset of last partition was not replicated"); assertEquals(0, offset.offset(), "Offset of last partition is not zero"); @@ -408,13 +417,13 @@ public void testOneWayReplicationWithAutoOffsetSync() throws InterruptedExceptio waitUntilMirrorMakerIsRunning(backup, CONNECTOR_LIST, mm2Config, PRIMARY_CLUSTER_ALIAS, BACKUP_CLUSTER_ALIAS); // make sure the topic is created in the other cluster - waitForTopicCreated(primary, "backup.test-topic-1"); - waitForTopicCreated(backup, "primary.test-topic-1"); + waitForTopicCreated(primary, formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-1")); + waitForTopicCreated(backup, formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1")); // create a consumer at backup cluster with same consumer group Id to consume 1 topic Consumer backupConsumer = backup.kafka().createConsumerAndSubscribeTo( - consumerProps, "primary.test-topic-1"); + consumerProps, formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1")); - waitForConsumerGroupOffsetSync(backup, backupConsumer, Collections.singletonList("primary.test-topic-1"), + waitForConsumerGroupOffsetSync(backup, backupConsumer, Collections.singletonList(formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1")), consumerGroupName, NUM_RECORDS_PRODUCED); ConsumerRecords records = backupConsumer.poll(CONSUMER_POLL_TIMEOUT_MS); @@ -427,7 +436,7 @@ public void testOneWayReplicationWithAutoOffsetSync() throws InterruptedExceptio // now create a new topic in primary cluster primary.kafka().createTopic("test-topic-2", NUM_PARTITIONS); // make sure the topic is created in backup cluster - waitForTopicCreated(backup, "primary.test-topic-2"); + waitForTopicCreated(backup, formatRemoteTopic("primary", "test-topic-2")); // produce some records to the new topic in primary cluster produceMessages(primary, "test-topic-2"); @@ -441,9 +450,9 @@ public void testOneWayReplicationWithAutoOffsetSync() throws InterruptedExceptio // 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"); + "group.id", consumerGroupName), formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1"), formatRemoteTopic("primary", "test-topic-2")); - waitForConsumerGroupOffsetSync(backup, backupConsumer, Arrays.asList("primary.test-topic-1", "primary.test-topic-2"), + waitForConsumerGroupOffsetSync(backup, backupConsumer, Arrays.asList(formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1"), formatRemoteTopic("primary", "test-topic-2")), consumerGroupName, NUM_RECORDS_PRODUCED); records = backupConsumer.poll(CONSUMER_POLL_TIMEOUT_MS); @@ -489,7 +498,7 @@ private static void waitForTopicCreated(EmbeddedConnectCluster cluster, String t private static void deleteAllTopics(EmbeddedKafkaCluster cluster) throws Exception { try (final Admin adminClient = cluster.createAdminClient()) { Set topicsToBeDeleted = adminClient.listTopics().names().get(); - log.debug("Deleting topics: {} ", topicsToBeDeleted); + log.info("Deleting topics: {} ", topicsToBeDeleted); adminClient.deleteTopics(topicsToBeDeleted).all().get(); } } @@ -598,16 +607,21 @@ private static Map basicMM2Config() { return mm2Props; } - + + // override in specialized tests + protected Map configOverrides() { + return new HashMap(); + } + private void createTopics() { // to verify topic config will be sync-ed across clusters Map topicConfig = Collections.singletonMap(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT); // create these topics before starting the connectors so we don't need to wait for discovery primary.kafka().createTopic("test-topic-1", NUM_PARTITIONS, 1, topicConfig); - primary.kafka().createTopic("backup.test-topic-1", 1); + primary.kafka().createTopic(formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-1"), 1); primary.kafka().createTopic("heartbeats", 1); backup.kafka().createTopic("test-topic-1", NUM_PARTITIONS); - backup.kafka().createTopic("primary.test-topic-1", 1); + backup.kafka().createTopic(formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1"), 1); backup.kafka().createTopic("heartbeats", 1); } @@ -624,4 +638,8 @@ private void warmUpConsumer(Map consumerProps) throws Interrupte dummyConsumer.commitSync(); dummyConsumer.close(); } + + private String formatRemoteTopic(String source, String topic) { + return replicationPolicy.formatRemoteTopic(source, topic); + } } diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationLegacyTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationLegacyTest.java new file mode 100644 index 0000000000000..9bd3c8dcd49a4 --- /dev/null +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationLegacyTest.java @@ -0,0 +1,35 @@ +/* + * 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.connect.mirror.LegacyReplicationPolicy; + +import java.util.Map; +import java.util.HashMap; + +import org.junit.jupiter.api.Tag; + +//@Tag("integration") +public class MirrorConnectorsIntegrationLegacyTest extends MirrorConnectorsIntegrationBaseTest { + @Override + protected Map configOverrides() { + Map m = new HashMap<>(); + m.put("replication.policy.class", LegacyReplicationPolicy.class.getName()); + m.put("remote.topic.suffix", ".replica"); + return m; + } +} From 697d21200981f14290e8f2608231ed068ddfef07 Mon Sep 17 00:00:00 2001 From: Ryanne Dolan Date: Tue, 18 May 2021 16:36:35 -0500 Subject: [PATCH 02/13] WIP fix integration tests -- two FIXMEs --- .../mirror/LegacyReplicationPolicy.java | 13 +- .../kafka/connect/mirror/MirrorClient.java | 2 +- .../connect/mirror/MirrorMakerConfig.java | 2 + .../MirrorConnectorsIntegrationBaseTest.java | 127 +++++++++++------- ...MirrorConnectorsIntegrationLegacyTest.java | 4 +- .../util/clusters/EmbeddedKafkaCluster.java | 2 +- 6 files changed, 96 insertions(+), 54 deletions(-) diff --git a/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/LegacyReplicationPolicy.java b/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/LegacyReplicationPolicy.java index 1a483147c68bc..75609fab9d659 100644 --- a/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/LegacyReplicationPolicy.java +++ b/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/LegacyReplicationPolicy.java @@ -19,7 +19,9 @@ import org.apache.kafka.common.Configurable; import java.util.Map; -import java.util.regex.Pattern; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** LegacyReplicationPolicy attempts to mimic MirrorMaker v1, and thus by default does not * rename topics. In order to support active/active replication with careful configuration, @@ -39,8 +41,9 @@ * already be the case. */ public class LegacyReplicationPolicy implements ReplicationPolicy, Configurable { + private static final Logger log = LoggerFactory.getLogger(LegacyReplicationPolicy.class); - public static final String REMOTE_TOPIC_SUFFIX_CONFIG = "remote.topic.suffix"; + public static final String REMOTE_TOPIC_SUFFIX_CONFIG = "replication.policy.remote.topic.suffix"; public static final String SOURCE_CLUSTER_ALIAS_CONFIG = "source.cluster.alias"; private String remoteTopicSuffix = ""; @@ -51,17 +54,19 @@ public LegacyReplicationPolicy() { // Visible for testing LegacyReplicationPolicy(String remoteTopicSuffix, String sourceClusterAlias) { - this.remoteTopicSuffix = remoteTopicSuffix; - this.sourceClusterAlias = sourceClusterAlias; + this.remoteTopicSuffix = remoteTopicSuffix; + this.sourceClusterAlias = sourceClusterAlias; } @Override public void configure(Map props) { if (props.containsKey(REMOTE_TOPIC_SUFFIX_CONFIG)) { remoteTopicSuffix = (String) props.get(REMOTE_TOPIC_SUFFIX_CONFIG); + log.info("Using custom remote topic suffix `{}`.", remoteTopicSuffix); } if (props.containsKey(SOURCE_CLUSTER_ALIAS_CONFIG)) { sourceClusterAlias = (String) props.get(SOURCE_CLUSTER_ALIAS_CONFIG); + log.info("Using source cluster alias `{}`.", sourceClusterAlias); } } diff --git a/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/MirrorClient.java b/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/MirrorClient.java index 775ec782c2d41..aacabe70a5a56 100644 --- a/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/MirrorClient.java +++ b/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/MirrorClient.java @@ -60,7 +60,7 @@ public class MirrorClient implements AutoCloseable { private ReplicationPolicy replicationPolicy; private Map consumerConfig; - public MirrorClient(Map props) { + public MirrorClient(Map props) { this(new MirrorClientConfig(props)); } diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorMakerConfig.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorMakerConfig.java index 298bce34fe531..33cd8a7ac3031 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorMakerConfig.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorMakerConfig.java @@ -175,6 +175,7 @@ public Map workerConfig(SourceAndTarget sourceAndTarget) { props.putAll(stringsWithPrefix("header.converter")); props.putAll(stringsWithPrefix("task")); props.putAll(stringsWithPrefix("worker")); + props.putAll(stringsWithPrefix("replication.policy")); // transform any expression like ${provider:path:key}, since the worker doesn't do so props = transform(props); @@ -203,6 +204,7 @@ public Map connectorBaseConfig(SourceAndTarget sourceAndTarget, props.keySet().retainAll(MirrorConnectorConfig.CONNECTOR_CONFIG_DEF.names()); props.putAll(stringsWithPrefix(CONFIG_PROVIDERS_CONFIG)); + props.putAll(stringsWithPrefix("replication.policy")); Map sourceClusterProps = clusterProps(sourceAndTarget.source()); // attrs non prefixed with producer|consumer|admin diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationBaseTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationBaseTest.java index 7a8c643939831..d9df6541adf7e 100644 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationBaseTest.java +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationBaseTest.java @@ -101,7 +101,8 @@ public abstract class MirrorConnectorsIntegrationBaseTest { private EmbeddedConnectCluster backup; private MirrorClient primaryClient; private MirrorClient backupClient; - private ReplicationPolicy replicationPolicy; + private ReplicationPolicy primaryReplicationPolicy; + private ReplicationPolicy backupReplicationPolicy; private Exit.Procedure exitProcedure; private Exit.Procedure haltProcedure; @@ -178,21 +179,28 @@ public void startClusters() throws Exception { waitForTopicCreated(primary, "mm2-offsets.backup.internal"); waitForTopicCreated(primary, "mm2-configs.backup.internal"); - waitForTopicCreated(backup, "mm2-status.primary.internal"); - waitForTopicCreated(backup, "mm2-offsets.primary.internal"); - waitForTopicCreated(backup, "mm2-configs.primary.internal"); - backup.start(); backup.assertions().assertAtLeastNumWorkersAreUp(NUM_WORKERS, "Workers of " + BACKUP_CLUSTER_ALIAS + "-connect-cluster did not start in time."); + waitForTopicCreated(backup, "mm2-status.primary.internal"); + waitForTopicCreated(backup, "mm2-offsets.primary.internal"); + waitForTopicCreated(backup, "mm2-configs.primary.internal"); + // 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()); mm2Config = new MirrorMakerConfig(mm2Props); - primaryClient = new MirrorClient(mm2Config.clientConfig(PRIMARY_CLUSTER_ALIAS)); - backupClient = new MirrorClient(mm2Config.clientConfig(BACKUP_CLUSTER_ALIAS)); - replicationPolicy = primaryClient.replicationPolicy(); // assume same in both clusters + Map primaryClientConfig = mm2Config.clientConfig(PRIMARY_CLUSTER_ALIAS).originals(); + // hack for LegacyReplicationPolicy, which needs a hint for source.cluster.alias + primaryClientConfig.put("source.cluster.alias", BACKUP_CLUSTER_ALIAS); + primaryClient = new MirrorClient(primaryClientConfig); + Map backupClientConfig = mm2Config.clientConfig(BACKUP_CLUSTER_ALIAS).originals(); + // hack for LegacyReplicationPolicy, which needs a hint for source.cluster.alias + backupClientConfig.put("source.cluster.alias", PRIMARY_CLUSTER_ALIAS); + backupClient = new MirrorClient(backupClientConfig); + primaryReplicationPolicy = primaryClient.replicationPolicy(); + backupReplicationPolicy = backupClient.replicationPolicy(); createTopics(); @@ -248,33 +256,39 @@ public void testReplication() throws Exception { waitUntilMirrorMakerIsRunning(primary, CONNECTOR_LIST, mm2Config, BACKUP_CLUSTER_ALIAS, PRIMARY_CLUSTER_ALIAS); // make sure the topic is auto-created in the other cluster - waitForTopicCreated(primary, formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-1")); - waitForTopicCreated(backup, formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1")); -// FIXME assertEquals(TopicConfig.CLEANUP_POLICY_COMPACT, getTopicConfig(backup.kafka(), -// formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1"), -// TopicConfig.CLEANUP_POLICY_CONFIG), "topic config was not synced"); + waitForTopicCreated(primary, primaryReplicationPolicy.formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-1")); + waitForTopicCreated(backup, backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1")); + assertEquals(TopicConfig.CLEANUP_POLICY_COMPACT, getTopicConfig(backup.kafka(), + backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1"), + TopicConfig.CLEANUP_POLICY_CONFIG), "topic config was not synced"); assertEquals(NUM_RECORDS_PRODUCED, primary.kafka().consume(NUM_RECORDS_PRODUCED, RECORD_TRANSFER_DURATION_MS, "test-topic-1").count(), "Records were not produced to primary cluster."); - assertEquals(NUM_RECORDS_PRODUCED, backup.kafka().consume(NUM_RECORDS_PRODUCED, RECORD_TRANSFER_DURATION_MS, formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1")).count(), + assertEquals(NUM_RECORDS_PRODUCED, backup.kafka().consume(NUM_RECORDS_PRODUCED, RECORD_TRANSFER_DURATION_MS, + backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1")).count(), "Records were not replicated to backup cluster."); assertEquals(NUM_RECORDS_PRODUCED, backup.kafka().consume(NUM_RECORDS_PRODUCED, RECORD_TRANSFER_DURATION_MS, "test-topic-1").count(), "Records were not produced to backup cluster."); - assertEquals(NUM_RECORDS_PRODUCED, primary.kafka().consume(NUM_RECORDS_PRODUCED, RECORD_TRANSFER_DURATION_MS, formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-1")).count(), + assertEquals(NUM_RECORDS_PRODUCED, primary.kafka().consume(NUM_RECORDS_PRODUCED, RECORD_TRANSFER_DURATION_MS, + primaryReplicationPolicy.formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-1")).count(), "Records were not replicated to primary cluster."); - assertEquals(NUM_RECORDS_PRODUCED * 2, primary.kafka().consume(NUM_RECORDS_PRODUCED * 2, RECORD_TRANSFER_DURATION_MS, formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-1"), "test-topic-1").count(), + assertEquals(NUM_RECORDS_PRODUCED * 2, primary.kafka().consume(NUM_RECORDS_PRODUCED * 2, RECORD_TRANSFER_DURATION_MS, + primaryReplicationPolicy.formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-1"), "test-topic-1").count(), "Primary cluster doesn't have all records from both clusters."); - assertEquals(NUM_RECORDS_PRODUCED * 2, backup.kafka().consume(NUM_RECORDS_PRODUCED * 2, RECORD_TRANSFER_DURATION_MS, formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1"), "test-topic-1").count(), + assertEquals(NUM_RECORDS_PRODUCED * 2, backup.kafka().consume(NUM_RECORDS_PRODUCED * 2, RECORD_TRANSFER_DURATION_MS, + backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1"), "test-topic-1").count(), "Backup cluster doesn't have all records from both clusters."); assertTrue(primary.kafka().consume(1, RECORD_TRANSFER_DURATION_MS, "heartbeats").count() > 0, "Heartbeats were not emitted to primary cluster."); assertTrue(backup.kafka().consume(1, RECORD_TRANSFER_DURATION_MS, "heartbeats").count() > 0, "Heartbeats were not emitted to backup cluster."); - assertTrue(backup.kafka().consume(1, RECORD_TRANSFER_DURATION_MS, "primary.heartbeats").count() > 0, + assertTrue(backup.kafka().consume(1, RECORD_TRANSFER_DURATION_MS, + backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "heartbeats")).count() > 0, "Heartbeats were not replicated downstream to backup cluster."); - assertTrue(primary.kafka().consume(1, RECORD_TRANSFER_DURATION_MS, "backup.heartbeats").count() > 0, + assertTrue(primary.kafka().consume(1, RECORD_TRANSFER_DURATION_MS, + primaryReplicationPolicy.formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "heartbeats")).count() > 0, "Heartbeats were not replicated downstream to primary cluster."); assertTrue(backupClient.upstreamClusters().contains(PRIMARY_CLUSTER_ALIAS), "Did not find upstream primary cluster."); @@ -288,7 +302,8 @@ public void testReplication() throws Exception { Duration.ofMillis(CHECKPOINT_DURATION_MS)); assertTrue(backupOffsets.containsKey( - new TopicPartition(formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1"), 0)), "Offsets not translated downstream to backup cluster. Found: " + backupOffsets); + new TopicPartition(backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1"), 0)), + "Offsets not translated downstream to backup cluster. Found: " + backupOffsets); // Failover consumer group to backup cluster. try (Consumer primaryConsumer = backup.kafka().createConsumer(Collections.singletonMap("group.id", consumerGroupName))) { @@ -297,18 +312,26 @@ public void testReplication() throws Exception { primaryConsumer.poll(CONSUMER_POLL_TIMEOUT_MS); primaryConsumer.commitAsync(); - assertTrue(primaryConsumer.position(new TopicPartition(formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1"), 0)) > 0, "Consumer failedover to zero offset."); assertTrue(primaryConsumer.position( - new TopicPartition(formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1"), 0)) <= NUM_RECORDS_PRODUCED, "Consumer failedover beyond expected offset."); + new TopicPartition(backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1"), 0)) > 0, + "Consumer failedover to zero offset."); + assertTrue(primaryConsumer.position( + new TopicPartition(backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1"), 0)) <= NUM_RECORDS_PRODUCED, + "Consumer failedover beyond expected offset."); assertTrue(primary.kafka().consume(1, CHECKPOINT_DURATION_MS, "backup.checkpoints.internal").count() > 0, "Checkpoints were not emitted upstream to primary cluster."); } waitForCondition(() -> primaryClient.remoteConsumerOffsets(consumerGroupName, BACKUP_CLUSTER_ALIAS, - Duration.ofMillis(CHECKPOINT_DURATION_MS)).containsKey(new TopicPartition(formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-1"), 0)), CHECKPOINT_DURATION_MS, "Offsets not translated downstream to primary cluster."); + Duration.ofMillis(CHECKPOINT_DURATION_MS)).containsKey( + new TopicPartition(primaryReplicationPolicy.formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-1"), 0)), + CHECKPOINT_DURATION_MS, "Offsets not translated downstream to primary cluster."); +/* FIXME waitForCondition(() -> primaryClient.remoteConsumerOffsets(consumerGroupName, BACKUP_CLUSTER_ALIAS, - Duration.ofMillis(CHECKPOINT_DURATION_MS)).containsKey(new TopicPartition("test-topic-1", 0)), CHECKPOINT_DURATION_MS, "Offsets not translated upstream to primary cluster."); + Duration.ofMillis(CHECKPOINT_DURATION_MS)).containsKey(new TopicPartition("test-topic-1", 0)), CHECKPOINT_DURATION_MS, + "Offsets not translated upstream to primary cluster."); +*/ Map primaryOffsets = primaryClient.remoteConsumerOffsets(consumerGroupName, BACKUP_CLUSTER_ALIAS, Duration.ofMillis(CHECKPOINT_DURATION_MS)); @@ -322,13 +345,21 @@ public void testReplication() throws Exception { primaryOffsets.forEach(backupConsumer::seek); backupConsumer.poll(CONSUMER_POLL_TIMEOUT_MS); backupConsumer.commitAsync(); - + +/* FIXME assertTrue(backupConsumer.position(new TopicPartition("test-topic-1", 0)) > 0, "Consumer failedback to zero upstream offset."); - assertTrue(backupConsumer.position(new TopicPartition(formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-1"), 0)) > 0, "Consumer failedback to zero downstream offset."); +*/ + assertTrue(backupConsumer.position( + new TopicPartition(primaryReplicationPolicy.formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-1"), 0)) > 0, + "Consumer failedback to zero downstream offset."); + +/* assertTrue(backupConsumer.position( new TopicPartition("test-topic-1", 0)) <= NUM_RECORDS_PRODUCED, "Consumer failedback beyond expected upstream offset."); +*/ assertTrue(backupConsumer.position( - new TopicPartition(formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-1"), 0)) <= NUM_RECORDS_PRODUCED, "Consumer failedback beyond expected downstream offset."); + new TopicPartition(primaryReplicationPolicy.formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-1"), 0)) <= NUM_RECORDS_PRODUCED, + "Consumer failedback beyond expected downstream offset."); } // create more matching topics @@ -336,8 +367,8 @@ public void testReplication() throws Exception { backup.kafka().createTopic("test-topic-3", NUM_PARTITIONS); // make sure the topic is auto-created in the other cluster - waitForTopicCreated(backup, formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-2")); - waitForTopicCreated(primary, formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-3")); + waitForTopicCreated(backup, backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-2")); + waitForTopicCreated(primary, primaryReplicationPolicy.formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-3")); // only produce messages to the first partition produceMessages(primary, "test-topic-2", 1); @@ -349,9 +380,11 @@ public void testReplication() throws Exception { assertEquals(NUM_RECORDS_PER_PARTITION, backup.kafka().consume(NUM_RECORDS_PER_PARTITION, RECORD_TRANSFER_DURATION_MS, "test-topic-3").count(), "Records were not produced to backup cluster."); - assertEquals(NUM_RECORDS_PER_PARTITION, primary.kafka().consume(NUM_RECORDS_PER_PARTITION, 2 * RECORD_TRANSFER_DURATION_MS, formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-3")).count(), + assertEquals(NUM_RECORDS_PER_PARTITION, primary.kafka().consume(NUM_RECORDS_PER_PARTITION, 2 * RECORD_TRANSFER_DURATION_MS, + primaryReplicationPolicy.formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-3")).count(), "New topic was not replicated to primary cluster."); - assertEquals(NUM_RECORDS_PER_PARTITION, backup.kafka().consume(NUM_RECORDS_PER_PARTITION, 2 * RECORD_TRANSFER_DURATION_MS, formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-2")).count(), + assertEquals(NUM_RECORDS_PER_PARTITION, backup.kafka().consume(NUM_RECORDS_PER_PARTITION, 2 * RECORD_TRANSFER_DURATION_MS, + backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-2")).count(), "New topic was not replicated to backup cluster."); } @@ -383,7 +416,7 @@ public void testReplicationWithEmptyPartition() throws Exception { // consume all records from backup cluster try (Consumer backupConsumer = backup.kafka().createConsumerAndSubscribeTo(consumerProps, - replicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, topic))) { + backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, topic))) { waitForConsumingAllRecords(backupConsumer, expectedRecords); } @@ -394,7 +427,7 @@ public void testReplicationWithEmptyPartition() throws Exception { // pinpoint the offset of the last partition which does not receive records OffsetAndMetadata offset = remoteOffsets.get(new TopicPartition( - replicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, topic), NUM_PARTITIONS - 1)); + backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, topic), NUM_PARTITIONS - 1)); // offset of the last partition should exist, but its value should be 0 assertNotNull(offset, "Offset of last partition was not replicated"); assertEquals(0, offset.offset(), "Offset of last partition is not zero"); @@ -427,13 +460,14 @@ public void testOneWayReplicationWithAutoOffsetSync() throws InterruptedExceptio waitUntilMirrorMakerIsRunning(backup, CONNECTOR_LIST, mm2Config, PRIMARY_CLUSTER_ALIAS, BACKUP_CLUSTER_ALIAS); // make sure the topic is created in the other cluster - waitForTopicCreated(primary, formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-1")); - waitForTopicCreated(backup, formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1")); + waitForTopicCreated(primary, primaryReplicationPolicy.formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-1")); + waitForTopicCreated(backup, backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1")); // create a consumer at backup cluster with same consumer group Id to consume 1 topic Consumer backupConsumer = backup.kafka().createConsumerAndSubscribeTo( - consumerProps, formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1")); + consumerProps, backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1")); - waitForConsumerGroupOffsetSync(backup, backupConsumer, Collections.singletonList(formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1")), + waitForConsumerGroupOffsetSync(backup, backupConsumer, Collections.singletonList( + backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1")), consumerGroupName, NUM_RECORDS_PRODUCED); ConsumerRecords records = backupConsumer.poll(CONSUMER_POLL_TIMEOUT_MS); @@ -446,7 +480,7 @@ public void testOneWayReplicationWithAutoOffsetSync() throws InterruptedExceptio // now create a new topic in primary cluster primary.kafka().createTopic("test-topic-2", NUM_PARTITIONS); // make sure the topic is created in backup cluster - waitForTopicCreated(backup, formatRemoteTopic("primary", "test-topic-2")); + waitForTopicCreated(backup, backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-2")); // produce some records to the new topic in primary cluster produceMessages(primary, "test-topic-2"); @@ -460,9 +494,12 @@ public void testOneWayReplicationWithAutoOffsetSync() throws InterruptedExceptio // 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), formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1"), formatRemoteTopic("primary", "test-topic-2")); + "group.id", consumerGroupName), backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1"), + backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-2")); - waitForConsumerGroupOffsetSync(backup, backupConsumer, Arrays.asList(formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1"), formatRemoteTopic("primary", "test-topic-2")), + waitForConsumerGroupOffsetSync(backup, backupConsumer, Arrays.asList( + backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1"), + backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-2")), consumerGroupName, NUM_RECORDS_PRODUCED); records = backupConsumer.poll(CONSUMER_POLL_TIMEOUT_MS); @@ -634,10 +671,12 @@ private void createTopics() { // create these topics before starting the connectors so we don't need to wait for discovery primary.kafka().createTopic("test-topic-1", NUM_PARTITIONS, 1, topicConfig, adminClientConfig); - primary.kafka().createTopic(formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-1"), 1, 1, emptyMap, adminClientConfig); + primary.kafka().createTopic(primaryReplicationPolicy.formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-1"), + 1, 1, emptyMap, adminClientConfig); primary.kafka().createTopic("heartbeats", 1, 1, emptyMap, adminClientConfig); backup.kafka().createTopic("test-topic-1", NUM_PARTITIONS, 1, emptyMap, adminClientConfig); - backup.kafka().createTopic(formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1"), 1, 1, emptyMap, adminClientConfig); + backup.kafka().createTopic(backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1"), + 1, 1, emptyMap, adminClientConfig); backup.kafka().createTopic("heartbeats", 1, 1, emptyMap, adminClientConfig); } @@ -654,8 +693,4 @@ private void warmUpConsumer(Map consumerProps) throws Interrupte dummyConsumer.commitSync(); dummyConsumer.close(); } - - private String formatRemoteTopic(String source, String topic) { - return replicationPolicy.formatRemoteTopic(source, topic); - } } diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationLegacyTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationLegacyTest.java index 9bd3c8dcd49a4..863e5408efbe6 100644 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationLegacyTest.java +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationLegacyTest.java @@ -23,13 +23,13 @@ import org.junit.jupiter.api.Tag; -//@Tag("integration") +@Tag("integration") public class MirrorConnectorsIntegrationLegacyTest extends MirrorConnectorsIntegrationBaseTest { @Override protected Map configOverrides() { Map m = new HashMap<>(); m.put("replication.policy.class", LegacyReplicationPolicy.class.getName()); - m.put("remote.topic.suffix", ".replica"); + m.put("replication.policy.remote.topic.suffix", ".replica"); return m; } } 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 30d41d26a56e2..ace0b34007d30 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 @@ -364,7 +364,7 @@ public void createTopic(String topic, int partitions, int replication, Map Date: Thu, 20 May 2021 17:05:08 -0500 Subject: [PATCH 03/13] steal mdedetrich's tests and IdentityReplicationPolicy, but without API changes --- .../mirror/IdentityReplicationPolicy.java | 92 ++++++ .../kafka/connect/mirror/MirrorClient.java | 3 +- ...ntityReplicationPolicyIntegrationTest.java | 263 ++++++++++++++++++ .../MirrorConnectorsIntegrationBaseTest.java | 201 ++++++------- ...MirrorConnectorsIntegrationLegacyTest.java | 35 --- 5 files changed, 434 insertions(+), 160 deletions(-) create mode 100644 connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/IdentityReplicationPolicy.java create mode 100644 connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIdentityReplicationPolicyIntegrationTest.java delete mode 100644 connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationLegacyTest.java diff --git a/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/IdentityReplicationPolicy.java b/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/IdentityReplicationPolicy.java new file mode 100644 index 0000000000000..465f193020c60 --- /dev/null +++ b/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/IdentityReplicationPolicy.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.mirror; + +import java.util.Map; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** IdentityReplicationPolicy does not rename remote topics. This is useful for migrating + * from legacy MM1, or for any use-case involving one-way replication. + * + * N.B. MirrorMaker is not able to prevent cycles when using this class, so take care that + * your replication topology is acyclic. If migrating from MirrorMaker v1, this will likely + * already be the case. + */ +public class IdentityReplicationPolicy extends DefaultReplicationPolicy { + private static final Logger log = LoggerFactory.getLogger(IdentityReplicationPolicy.class); + + public static final String SOURCE_CLUSTER_ALIAS_CONFIG = "source.cluster.alias"; + + private String sourceClusterAlias = null; + + @Override + public void configure(Map props) { + super.configure(props); + if (props.containsKey(SOURCE_CLUSTER_ALIAS_CONFIG)) { + sourceClusterAlias = (String) props.get(SOURCE_CLUSTER_ALIAS_CONFIG); + log.info("Using source cluster alias `{}`.", sourceClusterAlias); + } + } + + /** Unlike DefaultReplicationPolicy, IdentityReplicationPolicy does not include the source + * cluster alias in the remote topic name. Instead, topic names are unchanged. + * + * In the special case of heartbeats, we defer to DefaultReplicationPolicy. + */ + @Override + public String formatRemoteTopic(String sourceClusterAlias, String topic) { + if (looksLikeHeartbeat(topic)) { + return super.formatRemoteTopic(sourceClusterAlias, topic); + } else { + return topic; + } + } + + /** Unlike DefaultReplicationPolicy, IdendityReplicationPolicy cannot know the source of + * a remote topic based on its name alone. If `source.cluster.alias` is provided, + * `topicSource` will return that. + * + * In the special case of heartbeast, we defer to DefaultReplicationPolicy. + */ + @Override + public String topicSource(String topic) { + if (looksLikeHeartbeat(topic)) { + return super.topicSource(topic); + } else { + return sourceClusterAlias; + } + } + + /** Since any topic may be a "remote topic", this just returns `topic`. + * + * In the special case of heartbeast, we defer to DefaultReplicationPolicy. + */ + @Override + public String upstreamTopic(String topic) { + if (looksLikeHeartbeat(topic)) { + return super.upstreamTopic(topic); + } else { + return topic; + } + } + + private boolean looksLikeHeartbeat(String topic) { + return topic != null && topic.endsWith(MirrorClientConfig.HEARTBEATS_TOPIC); + } +} diff --git a/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/MirrorClient.java b/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/MirrorClient.java index aacabe70a5a56..acc1796e0a22a 100644 --- a/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/MirrorClient.java +++ b/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/MirrorClient.java @@ -231,8 +231,7 @@ Set allSources(String topic) { Set sources = new HashSet<>(); String source = replicationPolicy.topicSource(topic); while (source != null && !sources.contains(source)) { - // The extra Set.contains above is for LegacyReplicationPolicy (and any other dumb impls) which - // cannot prevent cycles. In impls that do prevent cycles, we shouldn't see the same source twice. + // The extra Set.contains above is for ReplicationPolicies that cannot prevent cycles. sources.add(source); topic = replicationPolicy.upstreamTopic(topic); source = replicationPolicy.topicSource(topic); diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIdentityReplicationPolicyIntegrationTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIdentityReplicationPolicyIntegrationTest.java new file mode 100644 index 0000000000000..43abad3f53022 --- /dev/null +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIdentityReplicationPolicyIntegrationTest.java @@ -0,0 +1,263 @@ +/* + * 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.TopicPartition; +import org.apache.kafka.connect.mirror.IdentityReplicationPolicy; +import org.apache.kafka.connect.mirror.MirrorClient; +import org.apache.kafka.connect.mirror.MirrorHeartbeatConnector; +import org.apache.kafka.connect.mirror.MirrorMakerConfig; + +import java.time.Duration; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +import org.junit.jupiter.api.Tag; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.BeforeEach; + +/** + * Tests MM2 replication and failover logic for {@link IdentityReplicationPolicy}. + * + *

MM2 is configured with active/passive replication between two Kafka clusters with {@link IdentityReplicationPolicy}. + * Tests validate that records sent to the primary cluster arrive at the backup cluster. Then, a consumer group is + * migrated from the primary cluster to the backup cluster. Tests validate that consumer offsets + * are translated and replicated from the primary cluster to the backup cluster during this failover. + */ +@Tag("integration") +public class MirrorConnectorsIdentityReplicationPolicyIntegrationTest extends MirrorConnectorsIntegrationTest { + @BeforeEach + public void startClusters() throws Exception { + super.startClusters(new HashMap() {{ + put("replication.policy.class", IdentityReplicationPolicy.class.getName()); + put("topics", "test-topic-.*"); + put(BACKUP_CLUSTER_ALIAS + "->" + PRIMARY_CLUSTER_ALIAS + ".enabled", "false"); + put(PRIMARY_CLUSTER_ALIAS + "->" + BACKUP_CLUSTER_ALIAS + ".enabled", "true"); + }}); + } + + @Test + public void testReplication() throws Exception { + produceMessages(primary, "test-topic-1"); + String consumerGroupName = "consumer-group-testReplication"; + Map consumerProps = new HashMap() {{ + put("group.id", consumerGroupName); + put("auto.offset.reset", "latest"); + }}; + // warm up consumers before starting the connectors so we don't need to wait for discovery + warmUpConsumer(consumerProps); + + mm2Config = new MirrorMakerConfig(mm2Props); + + waitUntilMirrorMakerIsRunning(backup, CONNECTOR_LIST, mm2Config, PRIMARY_CLUSTER_ALIAS, BACKUP_CLUSTER_ALIAS); + waitUntilMirrorMakerIsRunning(primary, Collections.singletonList(MirrorHeartbeatConnector.class), 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)); + + // make sure the topic is auto-created in the other cluster + waitForTopicCreated(primary, "test-topic-1"); + waitForTopicCreated(backup, "test-topic-1"); + assertEquals(TopicConfig.CLEANUP_POLICY_COMPACT, getTopicConfig(backup.kafka(), "test-topic-1", TopicConfig.CLEANUP_POLICY_CONFIG), + "topic config was not synced"); + + assertEquals(NUM_RECORDS_PRODUCED, primary.kafka().consume(NUM_RECORDS_PRODUCED, RECORD_TRANSFER_DURATION_MS, "test-topic-1").count(), + "Records were not produced to primary cluster."); + assertEquals(NUM_RECORDS_PRODUCED, backup.kafka().consume(NUM_RECORDS_PRODUCED, RECORD_TRANSFER_DURATION_MS, "test-topic-1").count(), + "Records were not replicated to backup cluster."); + + assertTrue(primary.kafka().consume(1, RECORD_TRANSFER_DURATION_MS, "heartbeats").count() > 0, + "Heartbeats were not emitted to primary cluster."); + assertTrue(backup.kafka().consume(1, RECORD_TRANSFER_DURATION_MS, "heartbeats").count() > 0, + "Heartbeats were not emitted to backup cluster."); + assertTrue(backup.kafka().consume(1, RECORD_TRANSFER_DURATION_MS, "primary.heartbeats").count() > 0, + "Heartbeats were not replicated downstream to backup cluster."); + assertTrue(primary.kafka().consume(1, RECORD_TRANSFER_DURATION_MS, "heartbeats").count() > 0, + "Heartbeats were not replicated downstream to primary cluster."); + + assertTrue(backupClient.upstreamClusters().contains(PRIMARY_CLUSTER_ALIAS), "Did not find upstream primary cluster."); + assertEquals(1, backupClient.replicationHops(PRIMARY_CLUSTER_ALIAS), "Did not calculate replication hops correctly."); + assertTrue(backup.kafka().consume(1, CHECKPOINT_DURATION_MS, "primary.checkpoints.internal").count() > 0, + "Checkpoints were not emitted downstream to backup cluster."); + + Map backupOffsets = backupClient.remoteConsumerOffsets(consumerGroupName, PRIMARY_CLUSTER_ALIAS, + Duration.ofMillis(CHECKPOINT_DURATION_MS)); + + assertTrue(backupOffsets.containsKey( + new TopicPartition("test-topic-1", 0)), "Offsets not translated downstream to backup cluster. Found: " + backupOffsets); + + // Failover consumer group to backup cluster. + try (Consumer primaryConsumer = backup.kafka().createConsumer(Collections.singletonMap("group.id", consumerGroupName))) { + primaryConsumer.assign(backupOffsets.keySet()); + backupOffsets.forEach(primaryConsumer::seek); + primaryConsumer.poll(CONSUMER_POLL_TIMEOUT_MS); + primaryConsumer.commitAsync(); + + assertTrue(primaryConsumer.position(new TopicPartition("test-topic-1", 0)) > 0, "Consumer failedover to zero offset."); + assertTrue(primaryConsumer.position( + new TopicPartition("test-topic-1", 0)) <= NUM_RECORDS_PRODUCED, "Consumer failedover beyond expected offset."); + } + + primaryClient.close(); + backupClient.close(); + + // create more matching topics + primary.kafka().createTopic("test-topic-2", NUM_PARTITIONS); + + // make sure the topic is auto-created in the other cluster + waitForTopicCreated(backup, "test-topic-2"); + + // only produce messages to the first partition + produceMessages(primary, "test-topic-2", 1); + + // expect total consumed messages equals to NUM_RECORDS_PER_PARTITION + assertEquals(NUM_RECORDS_PER_PARTITION, primary.kafka().consume(NUM_RECORDS_PER_PARTITION, RECORD_TRANSFER_DURATION_MS, "test-topic-2").count(), + "Records were not produced to primary cluster."); + assertEquals(NUM_RECORDS_PER_PARTITION, backup.kafka().consume(NUM_RECORDS_PER_PARTITION, 2 * RECORD_TRANSFER_DURATION_MS, "test-topic-2").count(), + "New topic was not replicated to backup cluster."); + } + + @Test + public void testReplicationWithEmptyPartition() throws Exception { + String consumerGroupName = "consumer-group-testReplicationWithEmptyPartition"; + Map 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 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)); + + // note that with LegacyReplicationPolicy, topics on the backup are NOT renamed to PRIMARY_CLUSTER_ALIAS + "." + topic + String backupTopic = topic; + + // consume all records from backup cluster + try (Consumer backupConsumer = backup.kafka().createConsumerAndSubscribeTo(consumerProps, + backupTopic)) { + waitForConsumingAllRecords(backupConsumer, expectedRecords); + } + + try (Admin backupClient = backup.kafka().createAdminClient()) { + // retrieve the consumer group offset from backup cluster + Map remoteOffsets = + backupClient.listConsumerGroupOffsets(consumerGroupName).partitionsToOffsetAndMetadata().get(); + + // pinpoint the offset of the last partition which does not receive records + OffsetAndMetadata offset = remoteOffsets.get(new TopicPartition(backupTopic, NUM_PARTITIONS - 1)); + // offset of the last partition should exist, but its value should be 0 + assertNotNull(offset, "Offset of last partition was not replicated"); + assertEquals(0, offset.offset(), "Offset of last partition is not zero"); + } + } + + @Test + public void testOneWayReplicationWithAutoOffsetSync() throws InterruptedException { + produceMessages(primary, "test-topic-1"); + String consumerGroupName = "consumer-group-testOneWayReplicationWithAutoOffsetSync"; + Map consumerProps = new HashMap() {{ + 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 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); + + // make sure the topic is created in the other cluster + waitForTopicCreated(primary, "backup.test-topic-1"); + waitForTopicCreated(backup, "test-topic-1"); + // create a consumer at backup cluster with same consumer group Id to consume 1 topic + Consumer backupConsumer = backup.kafka().createConsumerAndSubscribeTo( + consumerProps, "test-topic-1"); + + waitForConsumerGroupOffsetSync(backup, backupConsumer, Collections.singletonList("test-topic-1"), + consumerGroupName, NUM_RECORDS_PRODUCED); + + ConsumerRecords records = backupConsumer.poll(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(0, records.count(), "consumer record size is not zero"); + + // now create a new topic in primary cluster + primary.kafka().createTopic("test-topic-2", NUM_PARTITIONS); + // make sure the topic is created in backup cluster + waitForTopicCreated(backup, "test-topic-2"); + + // 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 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), "test-topic-1", "test-topic-2"); + + waitForConsumerGroupOffsetSync(backup, backupConsumer, Arrays.asList("test-topic-1", "test-topic-2"), + consumerGroupName, NUM_RECORDS_PRODUCED); + + records = backupConsumer.poll(CONSUMER_POLL_TIMEOUT_MS); + // similar reasoning as above, no more records to consume by the same consumer group at backup cluster + assertEquals(0, records.count(), "consumer record size is not zero"); + backupConsumer.close(); + } +} diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationBaseTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationBaseTest.java index d9df6541adf7e..c7d308aa3381d 100644 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationBaseTest.java +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationBaseTest.java @@ -34,7 +34,6 @@ 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.mirror.ReplicationPolicy; import org.apache.kafka.connect.util.clusters.EmbeddedConnectCluster; import org.apache.kafka.connect.util.clusters.EmbeddedKafkaCluster; import org.apache.kafka.connect.util.clusters.UngracefulShutdownException; @@ -78,33 +77,29 @@ 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; + protected static final int NUM_RECORDS_PER_PARTITION = 10; + protected 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; private static final int RECORD_CONSUME_DURATION_MS = 20_000; private static final int OFFSET_SYNC_DURATION_MS = 30_000; private static final int TOPIC_SYNC_DURATION_MS = 60_000; private static final int REQUEST_TIMEOUT_DURATION_MS = 60_000; private static final int NUM_WORKERS = 3; - private static final Duration CONSUMER_POLL_TIMEOUT_MS = Duration.ofMillis(500); + protected static final Duration CONSUMER_POLL_TIMEOUT_MS = Duration.ofMillis(500); protected static final String PRIMARY_CLUSTER_ALIAS = "primary"; protected static final String BACKUP_CLUSTER_ALIAS = "backup"; - private static final List> CONNECTOR_LIST = + protected static final List> CONNECTOR_LIST = Arrays.asList(MirrorSourceConnector.class, MirrorCheckpointConnector.class, MirrorHeartbeatConnector.class); private volatile boolean shuttingDown; protected Map mm2Props = new HashMap<>(); - private MirrorMakerConfig mm2Config; - private EmbeddedConnectCluster primary; - private EmbeddedConnectCluster backup; - private MirrorClient primaryClient; - private MirrorClient backupClient; - private ReplicationPolicy primaryReplicationPolicy; - private ReplicationPolicy backupReplicationPolicy; + protected MirrorMakerConfig mm2Config; + protected EmbeddedConnectCluster primary; + protected EmbeddedConnectCluster backup; - private Exit.Procedure exitProcedure; + protected Exit.Procedure exitProcedure; private Exit.Procedure haltProcedure; protected Properties primaryBrokerProps = new Properties(); @@ -114,6 +109,14 @@ public abstract class MirrorConnectorsIntegrationBaseTest { @BeforeEach public void startClusters() throws Exception { + startClusters(new HashMap() {{ + put("topics", "test-topic-.*, primary.test-topic-.*, backup.test-topic-.*"); + put(PRIMARY_CLUSTER_ALIAS + "->" + BACKUP_CLUSTER_ALIAS + ".enabled", "true"); + put(BACKUP_CLUSTER_ALIAS + "->" + PRIMARY_CLUSTER_ALIAS + ".enabled", "true"); + }}); + } + + public void startClusters(Map additionalMM2Config) throws Exception { shuttingDown = false; exitProcedure = (code, message) -> { if (shuttingDown) { @@ -144,15 +147,14 @@ public void startClusters() throws Exception { primaryBrokerProps.put("auto.create.topics.enable", "false"); backupBrokerProps.put("auto.create.topics.enable", "false"); - + mm2Props.putAll(basicMM2Config()); - mm2Props.putAll(configOverrides()); + mm2Props.putAll(additionalMM2Config); - // We don't have bootstrap.servers yet, so the MirrorMakerConfig is incomplete at this point. - MirrorMakerConfig tempMM2Config = new MirrorMakerConfig(mm2Props); - primaryWorkerProps = tempMM2Config.workerConfig(new SourceAndTarget(BACKUP_CLUSTER_ALIAS, PRIMARY_CLUSTER_ALIAS)); - backupWorkerProps.putAll(tempMM2Config.workerConfig(new SourceAndTarget(PRIMARY_CLUSTER_ALIAS, BACKUP_CLUSTER_ALIAS))); - + 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(NUM_WORKERS) @@ -178,7 +180,7 @@ public void startClusters() throws Exception { waitForTopicCreated(primary, "mm2-status.backup.internal"); waitForTopicCreated(primary, "mm2-offsets.backup.internal"); waitForTopicCreated(primary, "mm2-configs.backup.internal"); - + backup.start(); backup.assertions().assertAtLeastNumWorkersAreUp(NUM_WORKERS, "Workers of " + BACKUP_CLUSTER_ALIAS + "-connect-cluster did not start in time."); @@ -187,21 +189,6 @@ public void startClusters() throws Exception { waitForTopicCreated(backup, "mm2-offsets.primary.internal"); waitForTopicCreated(backup, "mm2-configs.primary.internal"); - // 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()); - mm2Config = new MirrorMakerConfig(mm2Props); - Map primaryClientConfig = mm2Config.clientConfig(PRIMARY_CLUSTER_ALIAS).originals(); - // hack for LegacyReplicationPolicy, which needs a hint for source.cluster.alias - primaryClientConfig.put("source.cluster.alias", BACKUP_CLUSTER_ALIAS); - primaryClient = new MirrorClient(primaryClientConfig); - Map backupClientConfig = mm2Config.clientConfig(BACKUP_CLUSTER_ALIAS).originals(); - // hack for LegacyReplicationPolicy, which needs a hint for source.cluster.alias - backupClientConfig.put("source.cluster.alias", PRIMARY_CLUSTER_ALIAS); - backupClient = new MirrorClient(backupClientConfig); - primaryReplicationPolicy = primaryClient.replicationPolicy(); - backupReplicationPolicy = backupClient.replicationPolicy(); - createTopics(); warmUpConsumer(Collections.singletonMap("group.id", "consumer-group-dummy")); @@ -210,6 +197,10 @@ public void startClusters() throws Exception { 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()); } @AfterEach @@ -255,40 +246,36 @@ public void testReplication() throws Exception { 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)); + // make sure the topic is auto-created in the other cluster - waitForTopicCreated(primary, primaryReplicationPolicy.formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-1")); - waitForTopicCreated(backup, backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1")); - assertEquals(TopicConfig.CLEANUP_POLICY_COMPACT, getTopicConfig(backup.kafka(), - backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1"), - TopicConfig.CLEANUP_POLICY_CONFIG), "topic config was not synced"); + waitForTopicCreated(primary, "backup.test-topic-1"); + waitForTopicCreated(backup, "primary.test-topic-1"); + assertEquals(TopicConfig.CLEANUP_POLICY_COMPACT, getTopicConfig(backup.kafka(), "primary.test-topic-1", TopicConfig.CLEANUP_POLICY_CONFIG), + "topic config was not synced"); assertEquals(NUM_RECORDS_PRODUCED, primary.kafka().consume(NUM_RECORDS_PRODUCED, RECORD_TRANSFER_DURATION_MS, "test-topic-1").count(), "Records were not produced to primary cluster."); - assertEquals(NUM_RECORDS_PRODUCED, backup.kafka().consume(NUM_RECORDS_PRODUCED, RECORD_TRANSFER_DURATION_MS, - backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1")).count(), + assertEquals(NUM_RECORDS_PRODUCED, backup.kafka().consume(NUM_RECORDS_PRODUCED, RECORD_TRANSFER_DURATION_MS, "primary.test-topic-1").count(), "Records were not replicated to backup cluster."); assertEquals(NUM_RECORDS_PRODUCED, backup.kafka().consume(NUM_RECORDS_PRODUCED, RECORD_TRANSFER_DURATION_MS, "test-topic-1").count(), "Records were not produced to backup cluster."); - assertEquals(NUM_RECORDS_PRODUCED, primary.kafka().consume(NUM_RECORDS_PRODUCED, RECORD_TRANSFER_DURATION_MS, - primaryReplicationPolicy.formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-1")).count(), + assertEquals(NUM_RECORDS_PRODUCED, primary.kafka().consume(NUM_RECORDS_PRODUCED, RECORD_TRANSFER_DURATION_MS, "backup.test-topic-1").count(), "Records were not replicated to primary cluster."); - assertEquals(NUM_RECORDS_PRODUCED * 2, primary.kafka().consume(NUM_RECORDS_PRODUCED * 2, RECORD_TRANSFER_DURATION_MS, - primaryReplicationPolicy.formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-1"), "test-topic-1").count(), + assertEquals(NUM_RECORDS_PRODUCED * 2, primary.kafka().consume(NUM_RECORDS_PRODUCED * 2, RECORD_TRANSFER_DURATION_MS, "backup.test-topic-1", "test-topic-1").count(), "Primary cluster doesn't have all records from both clusters."); - assertEquals(NUM_RECORDS_PRODUCED * 2, backup.kafka().consume(NUM_RECORDS_PRODUCED * 2, RECORD_TRANSFER_DURATION_MS, - backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1"), "test-topic-1").count(), + assertEquals(NUM_RECORDS_PRODUCED * 2, backup.kafka().consume(NUM_RECORDS_PRODUCED * 2, RECORD_TRANSFER_DURATION_MS, "primary.test-topic-1", "test-topic-1").count(), "Backup cluster doesn't have all records from both clusters."); assertTrue(primary.kafka().consume(1, RECORD_TRANSFER_DURATION_MS, "heartbeats").count() > 0, "Heartbeats were not emitted to primary cluster."); assertTrue(backup.kafka().consume(1, RECORD_TRANSFER_DURATION_MS, "heartbeats").count() > 0, "Heartbeats were not emitted to backup cluster."); - assertTrue(backup.kafka().consume(1, RECORD_TRANSFER_DURATION_MS, - backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "heartbeats")).count() > 0, + assertTrue(backup.kafka().consume(1, RECORD_TRANSFER_DURATION_MS, "primary.heartbeats").count() > 0, "Heartbeats were not replicated downstream to backup cluster."); - assertTrue(primary.kafka().consume(1, RECORD_TRANSFER_DURATION_MS, - primaryReplicationPolicy.formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "heartbeats")).count() > 0, + assertTrue(primary.kafka().consume(1, RECORD_TRANSFER_DURATION_MS, "backup.heartbeats").count() > 0, "Heartbeats were not replicated downstream to primary cluster."); assertTrue(backupClient.upstreamClusters().contains(PRIMARY_CLUSTER_ALIAS), "Did not find upstream primary cluster."); @@ -302,8 +289,7 @@ public void testReplication() throws Exception { Duration.ofMillis(CHECKPOINT_DURATION_MS)); assertTrue(backupOffsets.containsKey( - new TopicPartition(backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1"), 0)), - "Offsets not translated downstream to backup cluster. Found: " + backupOffsets); + new TopicPartition("primary.test-topic-1", 0)), "Offsets not translated downstream to backup cluster. Found: " + backupOffsets); // Failover consumer group to backup cluster. try (Consumer primaryConsumer = backup.kafka().createConsumer(Collections.singletonMap("group.id", consumerGroupName))) { @@ -312,26 +298,18 @@ public void testReplication() throws Exception { primaryConsumer.poll(CONSUMER_POLL_TIMEOUT_MS); primaryConsumer.commitAsync(); + assertTrue(primaryConsumer.position(new TopicPartition("primary.test-topic-1", 0)) > 0, "Consumer failedover to zero offset."); assertTrue(primaryConsumer.position( - new TopicPartition(backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1"), 0)) > 0, - "Consumer failedover to zero offset."); - assertTrue(primaryConsumer.position( - new TopicPartition(backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1"), 0)) <= NUM_RECORDS_PRODUCED, - "Consumer failedover beyond expected offset."); + new TopicPartition("primary.test-topic-1", 0)) <= NUM_RECORDS_PRODUCED, "Consumer failedover beyond expected offset."); assertTrue(primary.kafka().consume(1, CHECKPOINT_DURATION_MS, "backup.checkpoints.internal").count() > 0, "Checkpoints were not emitted upstream to primary cluster."); } waitForCondition(() -> primaryClient.remoteConsumerOffsets(consumerGroupName, BACKUP_CLUSTER_ALIAS, - Duration.ofMillis(CHECKPOINT_DURATION_MS)).containsKey( - new TopicPartition(primaryReplicationPolicy.formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-1"), 0)), - CHECKPOINT_DURATION_MS, "Offsets not translated downstream to primary cluster."); + Duration.ofMillis(CHECKPOINT_DURATION_MS)).containsKey(new TopicPartition("backup.test-topic-1", 0)), CHECKPOINT_DURATION_MS, "Offsets not translated downstream to primary cluster."); -/* FIXME waitForCondition(() -> primaryClient.remoteConsumerOffsets(consumerGroupName, BACKUP_CLUSTER_ALIAS, - Duration.ofMillis(CHECKPOINT_DURATION_MS)).containsKey(new TopicPartition("test-topic-1", 0)), CHECKPOINT_DURATION_MS, - "Offsets not translated upstream to primary cluster."); -*/ + Duration.ofMillis(CHECKPOINT_DURATION_MS)).containsKey(new TopicPartition("test-topic-1", 0)), CHECKPOINT_DURATION_MS, "Offsets not translated upstream to primary cluster."); Map primaryOffsets = primaryClient.remoteConsumerOffsets(consumerGroupName, BACKUP_CLUSTER_ALIAS, Duration.ofMillis(CHECKPOINT_DURATION_MS)); @@ -345,21 +323,13 @@ public void testReplication() throws Exception { primaryOffsets.forEach(backupConsumer::seek); backupConsumer.poll(CONSUMER_POLL_TIMEOUT_MS); backupConsumer.commitAsync(); - -/* FIXME + assertTrue(backupConsumer.position(new TopicPartition("test-topic-1", 0)) > 0, "Consumer failedback to zero upstream offset."); -*/ - assertTrue(backupConsumer.position( - new TopicPartition(primaryReplicationPolicy.formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-1"), 0)) > 0, - "Consumer failedback to zero downstream offset."); - -/* + assertTrue(backupConsumer.position(new TopicPartition("backup.test-topic-1", 0)) > 0, "Consumer failedback to zero downstream offset."); assertTrue(backupConsumer.position( new TopicPartition("test-topic-1", 0)) <= NUM_RECORDS_PRODUCED, "Consumer failedback beyond expected upstream offset."); -*/ assertTrue(backupConsumer.position( - new TopicPartition(primaryReplicationPolicy.formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-1"), 0)) <= NUM_RECORDS_PRODUCED, - "Consumer failedback beyond expected downstream offset."); + new TopicPartition("backup.test-topic-1", 0)) <= NUM_RECORDS_PRODUCED, "Consumer failedback beyond expected downstream offset."); } // create more matching topics @@ -367,8 +337,8 @@ public void testReplication() throws Exception { backup.kafka().createTopic("test-topic-3", NUM_PARTITIONS); // make sure the topic is auto-created in the other cluster - waitForTopicCreated(backup, backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-2")); - waitForTopicCreated(primary, primaryReplicationPolicy.formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-3")); + waitForTopicCreated(backup, "primary.test-topic-2"); + waitForTopicCreated(primary, "backup.test-topic-3"); // only produce messages to the first partition produceMessages(primary, "test-topic-2", 1); @@ -380,11 +350,9 @@ public void testReplication() throws Exception { assertEquals(NUM_RECORDS_PER_PARTITION, backup.kafka().consume(NUM_RECORDS_PER_PARTITION, RECORD_TRANSFER_DURATION_MS, "test-topic-3").count(), "Records were not produced to backup cluster."); - assertEquals(NUM_RECORDS_PER_PARTITION, primary.kafka().consume(NUM_RECORDS_PER_PARTITION, 2 * RECORD_TRANSFER_DURATION_MS, - primaryReplicationPolicy.formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-3")).count(), + assertEquals(NUM_RECORDS_PER_PARTITION, primary.kafka().consume(NUM_RECORDS_PER_PARTITION, 2 * RECORD_TRANSFER_DURATION_MS, "backup.test-topic-3").count(), "New topic was not replicated to primary cluster."); - assertEquals(NUM_RECORDS_PER_PARTITION, backup.kafka().consume(NUM_RECORDS_PER_PARTITION, 2 * RECORD_TRANSFER_DURATION_MS, - backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-2")).count(), + assertEquals(NUM_RECORDS_PER_PARTITION, backup.kafka().consume(NUM_RECORDS_PER_PARTITION, 2 * RECORD_TRANSFER_DURATION_MS, "primary.test-topic-2").count(), "New topic was not replicated to backup cluster."); } @@ -414,9 +382,11 @@ public void testReplicationWithEmptyPartition() throws Exception { // sleep few seconds to have MM2 finish replication so that "end" consumer will consume some record Thread.sleep(TimeUnit.SECONDS.toMillis(3)); + String backupTopic = PRIMARY_CLUSTER_ALIAS + "." + topic; + // consume all records from backup cluster - try (Consumer backupConsumer = backup.kafka().createConsumerAndSubscribeTo(consumerProps, - backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, topic))) { + try (Consumer backupConsumer = backup.kafka().createConsumerAndSubscribeTo(consumerProps, + backupTopic)) { waitForConsumingAllRecords(backupConsumer, expectedRecords); } @@ -426,8 +396,7 @@ public void testReplicationWithEmptyPartition() throws Exception { backupClient.listConsumerGroupOffsets(consumerGroupName).partitionsToOffsetAndMetadata().get(); // pinpoint the offset of the last partition which does not receive records - OffsetAndMetadata offset = remoteOffsets.get(new TopicPartition( - backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, topic), NUM_PARTITIONS - 1)); + OffsetAndMetadata offset = remoteOffsets.get(new TopicPartition(backupTopic, NUM_PARTITIONS - 1)); // offset of the last partition should exist, but its value should be 0 assertNotNull(offset, "Offset of last partition was not replicated"); assertEquals(0, offset.offset(), "Offset of last partition is not zero"); @@ -460,14 +429,13 @@ public void testOneWayReplicationWithAutoOffsetSync() throws InterruptedExceptio waitUntilMirrorMakerIsRunning(backup, CONNECTOR_LIST, mm2Config, PRIMARY_CLUSTER_ALIAS, BACKUP_CLUSTER_ALIAS); // make sure the topic is created in the other cluster - waitForTopicCreated(primary, primaryReplicationPolicy.formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-1")); - waitForTopicCreated(backup, backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1")); + waitForTopicCreated(primary, "backup.test-topic-1"); + waitForTopicCreated(backup, "primary.test-topic-1"); // create a consumer at backup cluster with same consumer group Id to consume 1 topic Consumer backupConsumer = backup.kafka().createConsumerAndSubscribeTo( - consumerProps, backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1")); + consumerProps, "primary.test-topic-1"); - waitForConsumerGroupOffsetSync(backup, backupConsumer, Collections.singletonList( - backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1")), + waitForConsumerGroupOffsetSync(backup, backupConsumer, Collections.singletonList("primary.test-topic-1"), consumerGroupName, NUM_RECORDS_PRODUCED); ConsumerRecords records = backupConsumer.poll(CONSUMER_POLL_TIMEOUT_MS); @@ -480,7 +448,7 @@ public void testOneWayReplicationWithAutoOffsetSync() throws InterruptedExceptio // now create a new topic in primary cluster primary.kafka().createTopic("test-topic-2", NUM_PARTITIONS); // make sure the topic is created in backup cluster - waitForTopicCreated(backup, backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-2")); + waitForTopicCreated(backup, "primary.test-topic-2"); // produce some records to the new topic in primary cluster produceMessages(primary, "test-topic-2"); @@ -494,12 +462,9 @@ public void testOneWayReplicationWithAutoOffsetSync() throws InterruptedExceptio // 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), backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1"), - backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-2")); + "group.id", consumerGroupName), "primary.test-topic-1", "primary.test-topic-2"); - waitForConsumerGroupOffsetSync(backup, backupConsumer, Arrays.asList( - backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1"), - backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-2")), + waitForConsumerGroupOffsetSync(backup, backupConsumer, Arrays.asList("primary.test-topic-1", "primary.test-topic-2"), consumerGroupName, NUM_RECORDS_PRODUCED); records = backupConsumer.poll(CONSUMER_POLL_TIMEOUT_MS); @@ -511,7 +476,7 @@ public void testOneWayReplicationWithAutoOffsetSync() throws InterruptedExceptio /* * launch the connectors on kafka connect cluster and check if they are running */ - private static void waitUntilMirrorMakerIsRunning(EmbeddedConnectCluster connectCluster, + protected static void waitUntilMirrorMakerIsRunning(EmbeddedConnectCluster connectCluster, List> connectorClasses, MirrorMakerConfig mm2Config, String primary, String backup) throws InterruptedException { for (Class connector : connectorClasses) { @@ -531,7 +496,7 @@ private static void waitUntilMirrorMakerIsRunning(EmbeddedConnectCluster connect /* * wait for the topic created on the cluster */ - private static void waitForTopicCreated(EmbeddedConnectCluster cluster, String topicName) throws InterruptedException { + protected static void waitForTopicCreated(EmbeddedConnectCluster cluster, String topicName) throws InterruptedException { try (final Admin adminClient = cluster.kafka().createAdminClient()) { waitForCondition(() -> adminClient.listTopics().names().get().contains(topicName), TOPIC_SYNC_DURATION_MS, "Topic: " + topicName + " didn't get created on cluster: " + cluster.getName() @@ -545,7 +510,7 @@ private static void waitForTopicCreated(EmbeddedConnectCluster cluster, String t private static void deleteAllTopics(EmbeddedKafkaCluster cluster) throws Exception { try (final Admin adminClient = cluster.createAdminClient()) { Set topicsToBeDeleted = adminClient.listTopics().names().get(); - log.info("Deleting topics: {} ", topicsToBeDeleted); + log.debug("Deleting topics: {} ", topicsToBeDeleted); adminClient.deleteTopics(topicsToBeDeleted).all().get(); } } @@ -553,7 +518,7 @@ private static void deleteAllTopics(EmbeddedKafkaCluster cluster) throws Excepti /* * retrieve the config value based on the input cluster, topic and config name */ - private static String getTopicConfig(EmbeddedKafkaCluster cluster, String topic, String configName) throws Exception { + protected static String getTopicConfig(EmbeddedKafkaCluster cluster, String topic, String configName) throws Exception { try (Admin client = cluster.createAdminClient()) { Collection cr = Collections.singleton( new ConfigResource(ConfigResource.Type.TOPIC, topic)); @@ -588,7 +553,7 @@ protected void produceMessages(EmbeddedConnectCluster cluster, String topicName, * given consumer group, topics and expected number of records, make sure the consumer group * offsets are eventually synced to the expected offset numbers */ - private static void waitForConsumerGroupOffsetSync(EmbeddedConnectCluster connect, + protected static void waitForConsumerGroupOffsetSync(EmbeddedConnectCluster connect, Consumer consumer, List topics, String consumerGroupId, int numRecords) throws InterruptedException { try (Admin adminClient = connect.kafka().createAdminClient()) { @@ -618,7 +583,7 @@ private static void waitForConsumerGroupOffsetSync(EmbeddedConnectCluster co /* * make sure the consumer to consume expected number of records */ - private static void waitForConsumingAllRecords(Consumer consumer, int numExpectedRecords) + protected static void waitForConsumingAllRecords(Consumer consumer, int numExpectedRecords) throws InterruptedException { final AtomicInteger totalConsumedRecords = new AtomicInteger(0); waitForCondition(() -> { @@ -631,14 +596,11 @@ private static void waitForConsumingAllRecords(Consumer consumer, int /* * MM2 config to use in integration tests */ - private static Map basicMM2Config() { + protected static Map basicMM2Config() { Map mm2Props = new HashMap<>(); mm2Props.put("clusters", PRIMARY_CLUSTER_ALIAS + ", " + BACKUP_CLUSTER_ALIAS); mm2Props.put("max.tasks", "10"); - mm2Props.put("topics", "test-topic-.*, primary.test-topic-.*, backup.test-topic-.*"); mm2Props.put("groups", "consumer-group-.*"); - mm2Props.put(PRIMARY_CLUSTER_ALIAS + "->" + BACKUP_CLUSTER_ALIAS + ".enabled", "true"); - mm2Props.put(BACKUP_CLUSTER_ALIAS + "->" + PRIMARY_CLUSTER_ALIAS + ".enabled", "true"); mm2Props.put("sync.topic.acls.enabled", "false"); mm2Props.put("emit.checkpoints.interval.seconds", "1"); mm2Props.put("emit.heartbeats.interval.seconds", "1"); @@ -654,12 +616,7 @@ private static Map basicMM2Config() { return mm2Props; } - - // override in specialized tests - protected Map configOverrides() { - return new HashMap(); - } - + private void createTopics() { // to verify topic config will be sync-ed across clusters Map topicConfig = Collections.singletonMap(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT); @@ -671,19 +628,17 @@ private void createTopics() { // create these topics before starting the connectors so we don't need to wait for discovery primary.kafka().createTopic("test-topic-1", NUM_PARTITIONS, 1, topicConfig, adminClientConfig); - primary.kafka().createTopic(primaryReplicationPolicy.formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-1"), - 1, 1, emptyMap, adminClientConfig); + primary.kafka().createTopic("backup.test-topic-1", 1, 1, emptyMap, adminClientConfig); primary.kafka().createTopic("heartbeats", 1, 1, emptyMap, adminClientConfig); backup.kafka().createTopic("test-topic-1", NUM_PARTITIONS, 1, emptyMap, adminClientConfig); - backup.kafka().createTopic(backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1"), - 1, 1, emptyMap, adminClientConfig); + backup.kafka().createTopic("primary.test-topic-1", 1, 1, emptyMap, adminClientConfig); backup.kafka().createTopic("heartbeats", 1, 1, emptyMap, adminClientConfig); } /* * Generate some consumer activity on both clusters to ensure the checkpoint connector always starts promptly */ - private void warmUpConsumer(Map consumerProps) throws InterruptedException { + protected void warmUpConsumer(Map consumerProps) throws InterruptedException { Consumer dummyConsumer = primary.kafka().createConsumerAndSubscribeTo(consumerProps, "test-topic-1"); dummyConsumer.poll(CONSUMER_POLL_TIMEOUT_MS); dummyConsumer.commitSync(); diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationLegacyTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationLegacyTest.java deleted file mode 100644 index 863e5408efbe6..0000000000000 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationLegacyTest.java +++ /dev/null @@ -1,35 +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.mirror.integration; - -import org.apache.kafka.connect.mirror.LegacyReplicationPolicy; - -import java.util.Map; -import java.util.HashMap; - -import org.junit.jupiter.api.Tag; - -@Tag("integration") -public class MirrorConnectorsIntegrationLegacyTest extends MirrorConnectorsIntegrationBaseTest { - @Override - protected Map configOverrides() { - Map m = new HashMap<>(); - m.put("replication.policy.class", LegacyReplicationPolicy.class.getName()); - m.put("replication.policy.remote.topic.suffix", ".replica"); - return m; - } -} From 53a2e2c897f845ca3b231b41631410f19befa4c0 Mon Sep 17 00:00:00 2001 From: Ryanne Dolan Date: Fri, 21 May 2021 15:35:49 -0500 Subject: [PATCH 04/13] dropped LegacyReplicationPolicy --- .../mirror/IdentityReplicationPolicy.java | 13 +- .../mirror/LegacyReplicationPolicy.java | 118 ------------------ .../kafka/connect/mirror/MirrorClient.java | 2 +- .../connect/mirror/MirrorClientTest.java | 36 +++--- .../connect/mirror/MirrorSourceConnector.java | 2 +- ...> IdentityReplicationIntegrationTest.java} | 4 +- 6 files changed, 29 insertions(+), 146 deletions(-) delete mode 100644 connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/LegacyReplicationPolicy.java rename connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/{MirrorConnectorsIdentityReplicationPolicyIntegrationTest.java => IdentityReplicationIntegrationTest.java} (98%) diff --git a/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/IdentityReplicationPolicy.java b/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/IdentityReplicationPolicy.java index 465f193020c60..082bcba9d3cda 100644 --- a/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/IdentityReplicationPolicy.java +++ b/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/IdentityReplicationPolicy.java @@ -35,6 +35,15 @@ public class IdentityReplicationPolicy extends DefaultReplicationPolicy { private String sourceClusterAlias = null; + public IdentityReplicationPolicy() { + //nop + } + + // Visible for testing + IdentityReplicationPolicy(String sourceClusterAlias) { + this.sourceClusterAlias = sourceClusterAlias; + } + @Override public void configure(Map props) { super.configure(props); @@ -62,7 +71,7 @@ public String formatRemoteTopic(String sourceClusterAlias, String topic) { * a remote topic based on its name alone. If `source.cluster.alias` is provided, * `topicSource` will return that. * - * In the special case of heartbeast, we defer to DefaultReplicationPolicy. + * In the special case of heartbeats, we defer to DefaultReplicationPolicy. */ @Override public String topicSource(String topic) { @@ -75,7 +84,7 @@ public String topicSource(String topic) { /** Since any topic may be a "remote topic", this just returns `topic`. * - * In the special case of heartbeast, we defer to DefaultReplicationPolicy. + * In the special case of heartbeats, we defer to DefaultReplicationPolicy. */ @Override public String upstreamTopic(String topic) { diff --git a/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/LegacyReplicationPolicy.java b/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/LegacyReplicationPolicy.java deleted file mode 100644 index 75609fab9d659..0000000000000 --- a/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/LegacyReplicationPolicy.java +++ /dev/null @@ -1,118 +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.mirror; - -import org.apache.kafka.common.Configurable; - -import java.util.Map; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** LegacyReplicationPolicy attempts to mimic MirrorMaker v1, and thus by default does not - * rename topics. In order to support active/active replication with careful configuration, - * topics can be renamed with an optional suffix, e.g. "topic1.replica". - * - * LegacyReplicationPolicy cannot, in general, detect whether a topic is remote or not, nor - * which cluster a topic may have been replicated from. If a `remote.topic.suffix` is - * provided, we use that to distinguish remote topics. Otherwise we consider all topics as - * remote topics. If a `source.cluster.alias` is provided, we use that as the source of - * any remote topics. Otherwise we return null for `topicSource()`. - * - * Paradoxically, then, the default behavior of LegacyReplicationPolicy is to consider all - * topics to be remote but with no known source. - * - * N.B. MirrorMaker is not able to prevent cycles when using this class, so take care that - * your replication topology is acyclic. If migrating from MirrorMaker v1, this will likely - * already be the case. - */ -public class LegacyReplicationPolicy implements ReplicationPolicy, Configurable { - private static final Logger log = LoggerFactory.getLogger(LegacyReplicationPolicy.class); - - public static final String REMOTE_TOPIC_SUFFIX_CONFIG = "replication.policy.remote.topic.suffix"; - public static final String SOURCE_CLUSTER_ALIAS_CONFIG = "source.cluster.alias"; - - private String remoteTopicSuffix = ""; - private String sourceClusterAlias = null; - - public LegacyReplicationPolicy() { - } - - // Visible for testing - LegacyReplicationPolicy(String remoteTopicSuffix, String sourceClusterAlias) { - this.remoteTopicSuffix = remoteTopicSuffix; - this.sourceClusterAlias = sourceClusterAlias; - } - - @Override - public void configure(Map props) { - if (props.containsKey(REMOTE_TOPIC_SUFFIX_CONFIG)) { - remoteTopicSuffix = (String) props.get(REMOTE_TOPIC_SUFFIX_CONFIG); - log.info("Using custom remote topic suffix `{}`.", remoteTopicSuffix); - } - if (props.containsKey(SOURCE_CLUSTER_ALIAS_CONFIG)) { - sourceClusterAlias = (String) props.get(SOURCE_CLUSTER_ALIAS_CONFIG); - log.info("Using source cluster alias `{}`.", sourceClusterAlias); - } - } - - /** Unlike DefaultReplicationPolicy, LegacyReplicationPolicy does not include the source - * cluster alias in the remote topic name. By default, topic names are unchanged. If - * a `remote.topic.suffix` is provided, topic are renamed accordingly. - */ - @Override - public String formatRemoteTopic(String sourceClusterAlias, String topic) { - return topic + remoteTopicSuffix; - } - - /** Unlike DefaultReplicationPolicy, LegacyReplicationPolicy cannot know the source of - * a remote topic based on its name alone. If `source.cluster.alias` is provided, - * `topicSource` will return that for anything that looks like a remote topic. By default, - * all topics look like remote topics. - */ - @Override - public String topicSource(String topic) { - if (topic.endsWith(remoteTopicSuffix)) { - return sourceClusterAlias; - } else { - return null; - } - } - - /** Unlike DefaultReplicationPolicy, LegacyReplicationPolicy cannot distinguish remote - * topics from regular topics by default. If `remote.topic.suffix` is provided, this - * method will strip that and return the original topic. By default, `topic` is returned. - */ - @Override - public String upstreamTopic(String topic) { - if (topic.endsWith(remoteTopicSuffix)) { - return topic.substring(0, topic.length() - remoteTopicSuffix.length()); - } else { - return topic; - } - } - - /** If remote.topic.suffix is provided, treat suffixed topics as internal topics. - * This prevents replicated topics from being further replicated when using - * LegacyReplicationPolicy. - */ - @Override - public boolean isInternalTopic(String topic) { - return ReplicationPolicy.super.isInternalTopic(topic) - || (!remoteTopicSuffix.isEmpty() && topic.endsWith(remoteTopicSuffix)); - } -} diff --git a/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/MirrorClient.java b/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/MirrorClient.java index acc1796e0a22a..291f713679e45 100644 --- a/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/MirrorClient.java +++ b/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/MirrorClient.java @@ -203,7 +203,7 @@ int countHopsForTopic(String topic, String sourceClusterAlias) { return hops; } if (visited.contains(source)) { - // Extra check for LegacyReplicationPolicy (and any other dumb impls) that cannot prevent cycles. + // Extra check for IdentityReplicationPolicy and similar impls that cannot prevent cycles. // We assume we're stuck in a cycle and will never find sourceClusterAlias. return -1; } diff --git a/connect/mirror-client/src/test/java/org/apache/kafka/connect/mirror/MirrorClientTest.java b/connect/mirror-client/src/test/java/org/apache/kafka/connect/mirror/MirrorClientTest.java index fa0145fca13c4..c867a3e3cd654 100644 --- a/connect/mirror-client/src/test/java/org/apache/kafka/connect/mirror/MirrorClientTest.java +++ b/connect/mirror-client/src/test/java/org/apache/kafka/connect/mirror/MirrorClientTest.java @@ -136,16 +136,20 @@ public void upstreamClustersTest() throws InterruptedException { } @Test - public void legacyUpstreamClustersTest() throws InterruptedException { - // When configured with a specific source, LegacyReplicationPolicy should just return the provided source. - MirrorClient client = new FakeMirrorClient(new LegacyReplicationPolicy("", "source"), Arrays.asList("topic1", + public void testIdentityReplicationUpstreamClusters() throws InterruptedException { + // IdentityReplicationPolicy treats heartbeats as a special case, so these should work as usual. + MirrorClient client = new FakeMirrorClient(new IdentityReplicationPolicy("source"), Arrays.asList("topic1", "topic2", "heartbeats", "source1.heartbeats", "source1.source2.heartbeats", "source3.source4.source5.heartbeats")); Set sources = client.upstreamClusters(); - assertTrue(sources.contains("source")); + assertTrue(sources.contains("source1")); + assertTrue(sources.contains("source2")); + assertTrue(sources.contains("source3")); + assertTrue(sources.contains("source4")); + assertTrue(sources.contains("source5")); assertFalse(sources.contains("")); assertFalse(sources.contains(null)); - assertEquals(1, sources.size()); + assertEquals(5, sources.size()); } @Test @@ -161,11 +165,10 @@ public void remoteTopicsTest() throws InterruptedException { assertTrue(remoteTopics.contains("source3.source4.source5.topic6")); } - @Test - public void legacyRemoteTopicsTest() throws InterruptedException { - // By default, LegacyReplicationPolicy should consider any topic to be remote. - MirrorClient client = new FakeMirrorClient(new LegacyReplicationPolicy("", "source"), Arrays.asList( + public void testIdentityReplicationRemoteTopics() throws InterruptedException { + // IdentityReplicationPolicy should consider any topic to be remote. + MirrorClient client = new FakeMirrorClient(new IdentityReplicationPolicy("source"), Arrays.asList( "topic1", "topic2", "topic3")); Set remoteTopics = client.remoteTopics(); assertTrue(remoteTopics.contains("topic1")); @@ -188,20 +191,9 @@ public void remoteTopicsSeparatorTest() throws InterruptedException { assertTrue(remoteTopics.contains("source3__source4__source5__topic6")); } - @Test - public void testLegacyRemoteTopicWithSuffix() { - MirrorClient client = new FakeMirrorClient( - new LegacyReplicationPolicy(".replica", "primary"), Arrays.asList()); - assertEquals("topic1.replica", client.replicationPolicy() - .formatRemoteTopic("primary", "topic1")); - assertEquals("primary", client.replicationPolicy() - .topicSource("topic1.replica")); - } - - @Test - public void testLegacyRemoteTopicWithoutSuffix() { + public void testIdentityReplicationTopicSource() { MirrorClient client = new FakeMirrorClient( - new LegacyReplicationPolicy("", "primary"), Arrays.asList()); + new IdentityReplicationPolicy("primary"), Arrays.asList()); assertEquals("topic1", client.replicationPolicy() .formatRemoteTopic("primary", "topic1")); assertEquals("primary", client.replicationPolicy() diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceConnector.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceConnector.java index d5e7aa224df73..b45f17607b508 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceConnector.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceConnector.java @@ -494,7 +494,7 @@ boolean isCycle(String topic) { } else { String upstreamTopic = replicationPolicy.upstreamTopic(topic); if (upstreamTopic.equals(topic)) { - // Extra check for LegacyReplicationPolicy + // Extra check for IdentityReplicationPolicy and similar impls that don't prevent cycles. return false; } return isCycle(upstreamTopic); diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIdentityReplicationPolicyIntegrationTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/IdentityReplicationIntegrationTest.java similarity index 98% rename from connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIdentityReplicationPolicyIntegrationTest.java rename to connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/IdentityReplicationIntegrationTest.java index 43abad3f53022..7385ed9ce1015 100644 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIdentityReplicationPolicyIntegrationTest.java +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/IdentityReplicationIntegrationTest.java @@ -51,7 +51,7 @@ * are translated and replicated from the primary cluster to the backup cluster during this failover. */ @Tag("integration") -public class MirrorConnectorsIdentityReplicationPolicyIntegrationTest extends MirrorConnectorsIntegrationTest { +public class IdentityReplicationIntegrationTest extends MirrorConnectorsIntegrationTest { @BeforeEach public void startClusters() throws Exception { super.startClusters(new HashMap() {{ @@ -169,7 +169,7 @@ public void testReplicationWithEmptyPartition() throws Exception { // sleep few seconds to have MM2 finish replication so that "end" consumer will consume some record Thread.sleep(TimeUnit.SECONDS.toMillis(3)); - // note that with LegacyReplicationPolicy, topics on the backup are NOT renamed to PRIMARY_CLUSTER_ALIAS + "." + topic + // note that with IdentityReplicationPolicy, topics on the backup are NOT renamed to PRIMARY_CLUSTER_ALIAS + "." + topic String backupTopic = topic; // consume all records from backup cluster From aaa10b22fceed297fc7c1b946544029bb51ebb6e Mon Sep 17 00:00:00 2001 From: Ryanne Dolan Date: Fri, 7 May 2021 19:08:57 +0000 Subject: [PATCH 05/13] WIP LegacyReplicationPolicy and tests --- .../mirror/LegacyReplicationPolicy.java | 113 ++++++++++++++++++ .../kafka/connect/mirror/MirrorClient.java | 11 +- .../connect/mirror/ReplicationPolicy.java | 2 +- .../connect/mirror/MirrorClientTest.java | 50 +++++++- .../connect/mirror/MirrorSourceConnector.java | 7 +- .../MirrorConnectorsIntegrationBaseTest.java | 102 +++++++++------- ...MirrorConnectorsIntegrationLegacyTest.java | 35 ++++++ 7 files changed, 274 insertions(+), 46 deletions(-) create mode 100644 connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/LegacyReplicationPolicy.java create mode 100644 connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationLegacyTest.java diff --git a/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/LegacyReplicationPolicy.java b/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/LegacyReplicationPolicy.java new file mode 100644 index 0000000000000..1a483147c68bc --- /dev/null +++ b/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/LegacyReplicationPolicy.java @@ -0,0 +1,113 @@ +/* + * 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; + +import org.apache.kafka.common.Configurable; + +import java.util.Map; +import java.util.regex.Pattern; + +/** LegacyReplicationPolicy attempts to mimic MirrorMaker v1, and thus by default does not + * rename topics. In order to support active/active replication with careful configuration, + * topics can be renamed with an optional suffix, e.g. "topic1.replica". + * + * LegacyReplicationPolicy cannot, in general, detect whether a topic is remote or not, nor + * which cluster a topic may have been replicated from. If a `remote.topic.suffix` is + * provided, we use that to distinguish remote topics. Otherwise we consider all topics as + * remote topics. If a `source.cluster.alias` is provided, we use that as the source of + * any remote topics. Otherwise we return null for `topicSource()`. + * + * Paradoxically, then, the default behavior of LegacyReplicationPolicy is to consider all + * topics to be remote but with no known source. + * + * N.B. MirrorMaker is not able to prevent cycles when using this class, so take care that + * your replication topology is acyclic. If migrating from MirrorMaker v1, this will likely + * already be the case. + */ +public class LegacyReplicationPolicy implements ReplicationPolicy, Configurable { + + public static final String REMOTE_TOPIC_SUFFIX_CONFIG = "remote.topic.suffix"; + public static final String SOURCE_CLUSTER_ALIAS_CONFIG = "source.cluster.alias"; + + private String remoteTopicSuffix = ""; + private String sourceClusterAlias = null; + + public LegacyReplicationPolicy() { + } + + // Visible for testing + LegacyReplicationPolicy(String remoteTopicSuffix, String sourceClusterAlias) { + this.remoteTopicSuffix = remoteTopicSuffix; + this.sourceClusterAlias = sourceClusterAlias; + } + + @Override + public void configure(Map props) { + if (props.containsKey(REMOTE_TOPIC_SUFFIX_CONFIG)) { + remoteTopicSuffix = (String) props.get(REMOTE_TOPIC_SUFFIX_CONFIG); + } + if (props.containsKey(SOURCE_CLUSTER_ALIAS_CONFIG)) { + sourceClusterAlias = (String) props.get(SOURCE_CLUSTER_ALIAS_CONFIG); + } + } + + /** Unlike DefaultReplicationPolicy, LegacyReplicationPolicy does not include the source + * cluster alias in the remote topic name. By default, topic names are unchanged. If + * a `remote.topic.suffix` is provided, topic are renamed accordingly. + */ + @Override + public String formatRemoteTopic(String sourceClusterAlias, String topic) { + return topic + remoteTopicSuffix; + } + + /** Unlike DefaultReplicationPolicy, LegacyReplicationPolicy cannot know the source of + * a remote topic based on its name alone. If `source.cluster.alias` is provided, + * `topicSource` will return that for anything that looks like a remote topic. By default, + * all topics look like remote topics. + */ + @Override + public String topicSource(String topic) { + if (topic.endsWith(remoteTopicSuffix)) { + return sourceClusterAlias; + } else { + return null; + } + } + + /** Unlike DefaultReplicationPolicy, LegacyReplicationPolicy cannot distinguish remote + * topics from regular topics by default. If `remote.topic.suffix` is provided, this + * method will strip that and return the original topic. By default, `topic` is returned. + */ + @Override + public String upstreamTopic(String topic) { + if (topic.endsWith(remoteTopicSuffix)) { + return topic.substring(0, topic.length() - remoteTopicSuffix.length()); + } else { + return topic; + } + } + + /** If remote.topic.suffix is provided, treat suffixed topics as internal topics. + * This prevents replicated topics from being further replicated when using + * LegacyReplicationPolicy. + */ + @Override + public boolean isInternalTopic(String topic) { + return ReplicationPolicy.super.isInternalTopic(topic) + || (!remoteTopicSuffix.isEmpty() && topic.endsWith(remoteTopicSuffix)); + } +} diff --git a/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/MirrorClient.java b/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/MirrorClient.java index 17d18ecb58a27..775ec782c2d41 100644 --- a/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/MirrorClient.java +++ b/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/MirrorClient.java @@ -192,6 +192,7 @@ Set listTopics() throws InterruptedException { int countHopsForTopic(String topic, String sourceClusterAlias) { int hops = 0; + Set visited = new HashSet<>(); while (true) { hops++; String source = replicationPolicy.topicSource(topic); @@ -201,6 +202,12 @@ int countHopsForTopic(String topic, String sourceClusterAlias) { if (source.equals(sourceClusterAlias)) { return hops; } + if (visited.contains(source)) { + // Extra check for LegacyReplicationPolicy (and any other dumb impls) that cannot prevent cycles. + // We assume we're stuck in a cycle and will never find sourceClusterAlias. + return -1; + } + visited.add(source); topic = replicationPolicy.upstreamTopic(topic); } } @@ -223,7 +230,9 @@ boolean isRemoteTopic(String topic) { Set allSources(String topic) { Set sources = new HashSet<>(); String source = replicationPolicy.topicSource(topic); - while (source != null) { + while (source != null && !sources.contains(source)) { + // The extra Set.contains above is for LegacyReplicationPolicy (and any other dumb impls) which + // cannot prevent cycles. In impls that do prevent cycles, we shouldn't see the same source twice. sources.add(source); topic = replicationPolicy.upstreamTopic(topic); source = replicationPolicy.topicSource(topic); diff --git a/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/ReplicationPolicy.java b/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/ReplicationPolicy.java index 11f73f50ceac5..b6eb26c8707bc 100644 --- a/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/ReplicationPolicy.java +++ b/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/ReplicationPolicy.java @@ -45,7 +45,7 @@ public interface ReplicationPolicy { */ default String originalTopic(String topic) { String upstream = upstreamTopic(topic); - if (upstream == null) { + if (upstream == null || upstream.equals(topic)) { return topic; } else { return originalTopic(upstream); diff --git a/connect/mirror-client/src/test/java/org/apache/kafka/connect/mirror/MirrorClientTest.java b/connect/mirror-client/src/test/java/org/apache/kafka/connect/mirror/MirrorClientTest.java index 766b91ce69944..fa0145fca13c4 100644 --- a/connect/mirror-client/src/test/java/org/apache/kafka/connect/mirror/MirrorClientTest.java +++ b/connect/mirror-client/src/test/java/org/apache/kafka/connect/mirror/MirrorClientTest.java @@ -37,7 +37,11 @@ private static class FakeMirrorClient extends MirrorClient { List topics; FakeMirrorClient(List topics) { - super(null, new DefaultReplicationPolicy(), null); + this(new DefaultReplicationPolicy(), topics); + } + + FakeMirrorClient(ReplicationPolicy replicationPolicy, List topics) { + super(null, replicationPolicy, null); this.topics = topics; } @@ -131,6 +135,19 @@ public void upstreamClustersTest() throws InterruptedException { assertFalse(sources.contains(null)); } + @Test + public void legacyUpstreamClustersTest() throws InterruptedException { + // When configured with a specific source, LegacyReplicationPolicy should just return the provided source. + MirrorClient client = new FakeMirrorClient(new LegacyReplicationPolicy("", "source"), Arrays.asList("topic1", + "topic2", "heartbeats", "source1.heartbeats", "source1.source2.heartbeats", + "source3.source4.source5.heartbeats")); + Set sources = client.upstreamClusters(); + assertTrue(sources.contains("source")); + assertFalse(sources.contains("")); + assertFalse(sources.contains(null)); + assertEquals(1, sources.size()); + } + @Test public void remoteTopicsTest() throws InterruptedException { MirrorClient client = new FakeMirrorClient(Arrays.asList("topic1", "topic2", "topic3", @@ -144,6 +161,18 @@ public void remoteTopicsTest() throws InterruptedException { assertTrue(remoteTopics.contains("source3.source4.source5.topic6")); } + + @Test + public void legacyRemoteTopicsTest() throws InterruptedException { + // By default, LegacyReplicationPolicy should consider any topic to be remote. + MirrorClient client = new FakeMirrorClient(new LegacyReplicationPolicy("", "source"), Arrays.asList( + "topic1", "topic2", "topic3")); + Set remoteTopics = client.remoteTopics(); + assertTrue(remoteTopics.contains("topic1")); + assertTrue(remoteTopics.contains("topic2")); + assertTrue(remoteTopics.contains("topic3")); + } + @Test public void remoteTopicsSeparatorTest() throws InterruptedException { MirrorClient client = new FakeMirrorClient(Arrays.asList("topic1", "topic2", "topic3", @@ -159,4 +188,23 @@ public void remoteTopicsSeparatorTest() throws InterruptedException { assertTrue(remoteTopics.contains("source3__source4__source5__topic6")); } + @Test + public void testLegacyRemoteTopicWithSuffix() { + MirrorClient client = new FakeMirrorClient( + new LegacyReplicationPolicy(".replica", "primary"), Arrays.asList()); + assertEquals("topic1.replica", client.replicationPolicy() + .formatRemoteTopic("primary", "topic1")); + assertEquals("primary", client.replicationPolicy() + .topicSource("topic1.replica")); + } + + @Test + public void testLegacyRemoteTopicWithoutSuffix() { + MirrorClient client = new FakeMirrorClient( + new LegacyReplicationPolicy("", "primary"), Arrays.asList()); + assertEquals("topic1", client.replicationPolicy() + .formatRemoteTopic("primary", "topic1")); + assertEquals("primary", client.replicationPolicy() + .topicSource("topic1")); + } } diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceConnector.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceConnector.java index 91a40a638cbf6..d5e7aa224df73 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceConnector.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceConnector.java @@ -492,7 +492,12 @@ boolean isCycle(String topic) { } else if (source.equals(sourceAndTarget.target())) { return true; } else { - return isCycle(replicationPolicy.upstreamTopic(topic)); + String upstreamTopic = replicationPolicy.upstreamTopic(topic); + if (upstreamTopic.equals(topic)) { + // Extra check for LegacyReplicationPolicy + return false; + } + return isCycle(upstreamTopic); } } diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationBaseTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationBaseTest.java index 841eedc33c6e6..93beab350400d 100644 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationBaseTest.java +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationBaseTest.java @@ -34,6 +34,7 @@ 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.mirror.ReplicationPolicy; import org.apache.kafka.connect.util.clusters.EmbeddedConnectCluster; import org.apache.kafka.connect.util.clusters.EmbeddedKafkaCluster; import org.apache.kafka.connect.util.clusters.UngracefulShutdownException; @@ -98,6 +99,9 @@ public abstract class MirrorConnectorsIntegrationBaseTest { private MirrorMakerConfig mm2Config; private EmbeddedConnectCluster primary; private EmbeddedConnectCluster backup; + private MirrorClient primaryClient; + private MirrorClient backupClient; + private ReplicationPolicy replicationPolicy; private Exit.Procedure exitProcedure; private Exit.Procedure haltProcedure; @@ -141,11 +145,13 @@ public void startClusters() throws Exception { backupBrokerProps.put("auto.create.topics.enable", "false"); mm2Props.putAll(basicMM2Config()); + mm2Props.putAll(configOverrides()); - 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))); - + // We don't have bootstrap.servers yet, so the MirrorMakerConfig is incomplete at this point. + MirrorMakerConfig tempMM2Config = new MirrorMakerConfig(mm2Props); + primaryWorkerProps = tempMM2Config.workerConfig(new SourceAndTarget(BACKUP_CLUSTER_ALIAS, PRIMARY_CLUSTER_ALIAS)); + backupWorkerProps.putAll(tempMM2Config.workerConfig(new SourceAndTarget(PRIMARY_CLUSTER_ALIAS, BACKUP_CLUSTER_ALIAS))); + primary = new EmbeddedConnectCluster.Builder() .name(PRIMARY_CLUSTER_ALIAS + "-connect-cluster") .numWorkers(NUM_WORKERS) @@ -180,6 +186,14 @@ public void startClusters() throws Exception { waitForTopicCreated(backup, "mm2-offsets.primary.internal"); waitForTopicCreated(backup, "mm2-configs.primary.internal"); + // 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()); + mm2Config = new MirrorMakerConfig(mm2Props); + primaryClient = new MirrorClient(mm2Config.clientConfig(PRIMARY_CLUSTER_ALIAS)); + backupClient = new MirrorClient(mm2Config.clientConfig(BACKUP_CLUSTER_ALIAS)); + replicationPolicy = primaryClient.replicationPolicy(); // assume same in both clusters + createTopics(); warmUpConsumer(Collections.singletonMap("group.id", "consumer-group-dummy")); @@ -188,10 +202,6 @@ public void startClusters() throws Exception { 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()); } @AfterEach @@ -237,27 +247,25 @@ public void testReplication() throws Exception { 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)); - // make sure the topic is auto-created in the other cluster - waitForTopicCreated(primary, "backup.test-topic-1"); - waitForTopicCreated(backup, "primary.test-topic-1"); - assertEquals(TopicConfig.CLEANUP_POLICY_COMPACT, getTopicConfig(backup.kafka(), "primary.test-topic-1", TopicConfig.CLEANUP_POLICY_CONFIG), - "topic config was not synced"); + waitForTopicCreated(primary, formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-1")); + waitForTopicCreated(backup, formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1")); +// FIXME assertEquals(TopicConfig.CLEANUP_POLICY_COMPACT, getTopicConfig(backup.kafka(), +// formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1"), +// TopicConfig.CLEANUP_POLICY_CONFIG), "topic config was not synced"); assertEquals(NUM_RECORDS_PRODUCED, primary.kafka().consume(NUM_RECORDS_PRODUCED, RECORD_TRANSFER_DURATION_MS, "test-topic-1").count(), "Records were not produced to primary cluster."); - assertEquals(NUM_RECORDS_PRODUCED, backup.kafka().consume(NUM_RECORDS_PRODUCED, RECORD_TRANSFER_DURATION_MS, "primary.test-topic-1").count(), + assertEquals(NUM_RECORDS_PRODUCED, backup.kafka().consume(NUM_RECORDS_PRODUCED, RECORD_TRANSFER_DURATION_MS, formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1")).count(), "Records were not replicated to backup cluster."); assertEquals(NUM_RECORDS_PRODUCED, backup.kafka().consume(NUM_RECORDS_PRODUCED, RECORD_TRANSFER_DURATION_MS, "test-topic-1").count(), "Records were not produced to backup cluster."); - assertEquals(NUM_RECORDS_PRODUCED, primary.kafka().consume(NUM_RECORDS_PRODUCED, RECORD_TRANSFER_DURATION_MS, "backup.test-topic-1").count(), + assertEquals(NUM_RECORDS_PRODUCED, primary.kafka().consume(NUM_RECORDS_PRODUCED, RECORD_TRANSFER_DURATION_MS, formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-1")).count(), "Records were not replicated to primary cluster."); - assertEquals(NUM_RECORDS_PRODUCED * 2, primary.kafka().consume(NUM_RECORDS_PRODUCED * 2, RECORD_TRANSFER_DURATION_MS, "backup.test-topic-1", "test-topic-1").count(), + assertEquals(NUM_RECORDS_PRODUCED * 2, primary.kafka().consume(NUM_RECORDS_PRODUCED * 2, RECORD_TRANSFER_DURATION_MS, formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-1"), "test-topic-1").count(), "Primary cluster doesn't have all records from both clusters."); - assertEquals(NUM_RECORDS_PRODUCED * 2, backup.kafka().consume(NUM_RECORDS_PRODUCED * 2, RECORD_TRANSFER_DURATION_MS, "primary.test-topic-1", "test-topic-1").count(), + assertEquals(NUM_RECORDS_PRODUCED * 2, backup.kafka().consume(NUM_RECORDS_PRODUCED * 2, RECORD_TRANSFER_DURATION_MS, formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1"), "test-topic-1").count(), "Backup cluster doesn't have all records from both clusters."); assertTrue(primary.kafka().consume(1, RECORD_TRANSFER_DURATION_MS, "heartbeats").count() > 0, @@ -280,7 +288,7 @@ public void testReplication() throws Exception { Duration.ofMillis(CHECKPOINT_DURATION_MS)); assertTrue(backupOffsets.containsKey( - new TopicPartition("primary.test-topic-1", 0)), "Offsets not translated downstream to backup cluster. Found: " + backupOffsets); + new TopicPartition(formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1"), 0)), "Offsets not translated downstream to backup cluster. Found: " + backupOffsets); // Failover consumer group to backup cluster. try (Consumer primaryConsumer = backup.kafka().createConsumer(Collections.singletonMap("group.id", consumerGroupName))) { @@ -289,15 +297,15 @@ public void testReplication() throws Exception { primaryConsumer.poll(CONSUMER_POLL_TIMEOUT_MS); primaryConsumer.commitAsync(); - assertTrue(primaryConsumer.position(new TopicPartition("primary.test-topic-1", 0)) > 0, "Consumer failedover to zero offset."); + assertTrue(primaryConsumer.position(new TopicPartition(formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1"), 0)) > 0, "Consumer failedover to zero offset."); assertTrue(primaryConsumer.position( - new TopicPartition("primary.test-topic-1", 0)) <= NUM_RECORDS_PRODUCED, "Consumer failedover beyond expected offset."); + new TopicPartition(formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1"), 0)) <= NUM_RECORDS_PRODUCED, "Consumer failedover beyond expected offset."); assertTrue(primary.kafka().consume(1, CHECKPOINT_DURATION_MS, "backup.checkpoints.internal").count() > 0, "Checkpoints were not emitted upstream to primary cluster."); } waitForCondition(() -> primaryClient.remoteConsumerOffsets(consumerGroupName, BACKUP_CLUSTER_ALIAS, - Duration.ofMillis(CHECKPOINT_DURATION_MS)).containsKey(new TopicPartition("backup.test-topic-1", 0)), CHECKPOINT_DURATION_MS, "Offsets not translated downstream to primary cluster."); + Duration.ofMillis(CHECKPOINT_DURATION_MS)).containsKey(new TopicPartition(formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-1"), 0)), CHECKPOINT_DURATION_MS, "Offsets not translated downstream to primary cluster."); waitForCondition(() -> primaryClient.remoteConsumerOffsets(consumerGroupName, BACKUP_CLUSTER_ALIAS, Duration.ofMillis(CHECKPOINT_DURATION_MS)).containsKey(new TopicPartition("test-topic-1", 0)), CHECKPOINT_DURATION_MS, "Offsets not translated upstream to primary cluster."); @@ -316,11 +324,11 @@ public void testReplication() throws Exception { backupConsumer.commitAsync(); assertTrue(backupConsumer.position(new TopicPartition("test-topic-1", 0)) > 0, "Consumer failedback to zero upstream offset."); - assertTrue(backupConsumer.position(new TopicPartition("backup.test-topic-1", 0)) > 0, "Consumer failedback to zero downstream offset."); + assertTrue(backupConsumer.position(new TopicPartition(formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-1"), 0)) > 0, "Consumer failedback to zero downstream offset."); assertTrue(backupConsumer.position( new TopicPartition("test-topic-1", 0)) <= NUM_RECORDS_PRODUCED, "Consumer failedback beyond expected upstream offset."); assertTrue(backupConsumer.position( - new TopicPartition("backup.test-topic-1", 0)) <= NUM_RECORDS_PRODUCED, "Consumer failedback beyond expected downstream offset."); + new TopicPartition(formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-1"), 0)) <= NUM_RECORDS_PRODUCED, "Consumer failedback beyond expected downstream offset."); } // create more matching topics @@ -328,8 +336,8 @@ public void testReplication() throws Exception { backup.kafka().createTopic("test-topic-3", NUM_PARTITIONS); // make sure the topic is auto-created in the other cluster - waitForTopicCreated(backup, "primary.test-topic-2"); - waitForTopicCreated(primary, "backup.test-topic-3"); + waitForTopicCreated(backup, formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-2")); + waitForTopicCreated(primary, formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-3")); // only produce messages to the first partition produceMessages(primary, "test-topic-2", 1); @@ -341,9 +349,9 @@ public void testReplication() throws Exception { assertEquals(NUM_RECORDS_PER_PARTITION, backup.kafka().consume(NUM_RECORDS_PER_PARTITION, RECORD_TRANSFER_DURATION_MS, "test-topic-3").count(), "Records were not produced to backup cluster."); - assertEquals(NUM_RECORDS_PER_PARTITION, primary.kafka().consume(NUM_RECORDS_PER_PARTITION, 2 * RECORD_TRANSFER_DURATION_MS, "backup.test-topic-3").count(), + assertEquals(NUM_RECORDS_PER_PARTITION, primary.kafka().consume(NUM_RECORDS_PER_PARTITION, 2 * RECORD_TRANSFER_DURATION_MS, formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-3")).count(), "New topic was not replicated to primary cluster."); - assertEquals(NUM_RECORDS_PER_PARTITION, backup.kafka().consume(NUM_RECORDS_PER_PARTITION, 2 * RECORD_TRANSFER_DURATION_MS, "primary.test-topic-2").count(), + assertEquals(NUM_RECORDS_PER_PARTITION, backup.kafka().consume(NUM_RECORDS_PER_PARTITION, 2 * RECORD_TRANSFER_DURATION_MS, formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-2")).count(), "New topic was not replicated to backup cluster."); } @@ -375,7 +383,7 @@ public void testReplicationWithEmptyPartition() throws Exception { // consume all records from backup cluster try (Consumer backupConsumer = backup.kafka().createConsumerAndSubscribeTo(consumerProps, - PRIMARY_CLUSTER_ALIAS + "." + topic)) { + replicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, topic))) { waitForConsumingAllRecords(backupConsumer, expectedRecords); } @@ -385,7 +393,8 @@ public void testReplicationWithEmptyPartition() throws Exception { 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)); + OffsetAndMetadata offset = remoteOffsets.get(new TopicPartition( + replicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, topic), NUM_PARTITIONS - 1)); // offset of the last partition should exist, but its value should be 0 assertNotNull(offset, "Offset of last partition was not replicated"); assertEquals(0, offset.offset(), "Offset of last partition is not zero"); @@ -418,13 +427,13 @@ public void testOneWayReplicationWithAutoOffsetSync() throws InterruptedExceptio waitUntilMirrorMakerIsRunning(backup, CONNECTOR_LIST, mm2Config, PRIMARY_CLUSTER_ALIAS, BACKUP_CLUSTER_ALIAS); // make sure the topic is created in the other cluster - waitForTopicCreated(primary, "backup.test-topic-1"); - waitForTopicCreated(backup, "primary.test-topic-1"); + waitForTopicCreated(primary, formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-1")); + waitForTopicCreated(backup, formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1")); // create a consumer at backup cluster with same consumer group Id to consume 1 topic Consumer backupConsumer = backup.kafka().createConsumerAndSubscribeTo( - consumerProps, "primary.test-topic-1"); + consumerProps, formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1")); - waitForConsumerGroupOffsetSync(backup, backupConsumer, Collections.singletonList("primary.test-topic-1"), + waitForConsumerGroupOffsetSync(backup, backupConsumer, Collections.singletonList(formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1")), consumerGroupName, NUM_RECORDS_PRODUCED); ConsumerRecords records = backupConsumer.poll(CONSUMER_POLL_TIMEOUT_MS); @@ -437,7 +446,7 @@ public void testOneWayReplicationWithAutoOffsetSync() throws InterruptedExceptio // now create a new topic in primary cluster primary.kafka().createTopic("test-topic-2", NUM_PARTITIONS); // make sure the topic is created in backup cluster - waitForTopicCreated(backup, "primary.test-topic-2"); + waitForTopicCreated(backup, formatRemoteTopic("primary", "test-topic-2")); // produce some records to the new topic in primary cluster produceMessages(primary, "test-topic-2"); @@ -451,9 +460,9 @@ public void testOneWayReplicationWithAutoOffsetSync() throws InterruptedExceptio // 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"); + "group.id", consumerGroupName), formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1"), formatRemoteTopic("primary", "test-topic-2")); - waitForConsumerGroupOffsetSync(backup, backupConsumer, Arrays.asList("primary.test-topic-1", "primary.test-topic-2"), + waitForConsumerGroupOffsetSync(backup, backupConsumer, Arrays.asList(formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1"), formatRemoteTopic("primary", "test-topic-2")), consumerGroupName, NUM_RECORDS_PRODUCED); records = backupConsumer.poll(CONSUMER_POLL_TIMEOUT_MS); @@ -499,7 +508,7 @@ private static void waitForTopicCreated(EmbeddedConnectCluster cluster, String t private static void deleteAllTopics(EmbeddedKafkaCluster cluster) throws Exception { try (final Admin adminClient = cluster.createAdminClient()) { Set topicsToBeDeleted = adminClient.listTopics().names().get(); - log.debug("Deleting topics: {} ", topicsToBeDeleted); + log.info("Deleting topics: {} ", topicsToBeDeleted); adminClient.deleteTopics(topicsToBeDeleted).all().get(); } } @@ -608,7 +617,12 @@ private static Map basicMM2Config() { return mm2Props; } - + + // override in specialized tests + protected Map configOverrides() { + return new HashMap(); + } + private void createTopics() { // to verify topic config will be sync-ed across clusters Map topicConfig = Collections.singletonMap(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT); @@ -620,10 +634,10 @@ private void createTopics() { // create these topics before starting the connectors so we don't need to wait for discovery primary.kafka().createTopic("test-topic-1", NUM_PARTITIONS, 1, topicConfig, adminClientConfig); - primary.kafka().createTopic("backup.test-topic-1", 1, 1, emptyMap, adminClientConfig); + primary.kafka().createTopic(formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-1"), 1, 1, emptyMap, adminClientConfig); primary.kafka().createTopic("heartbeats", 1, 1, emptyMap, adminClientConfig); backup.kafka().createTopic("test-topic-1", NUM_PARTITIONS, 1, emptyMap, adminClientConfig); - backup.kafka().createTopic("primary.test-topic-1", 1, 1, emptyMap, adminClientConfig); + backup.kafka().createTopic(formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1"), 1, 1, emptyMap, adminClientConfig); backup.kafka().createTopic("heartbeats", 1, 1, emptyMap, adminClientConfig); } @@ -640,4 +654,8 @@ private void warmUpConsumer(Map consumerProps) throws Interrupte dummyConsumer.commitSync(); dummyConsumer.close(); } + + private String formatRemoteTopic(String source, String topic) { + return replicationPolicy.formatRemoteTopic(source, topic); + } } diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationLegacyTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationLegacyTest.java new file mode 100644 index 0000000000000..afcedca1c4643 --- /dev/null +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationLegacyTest.java @@ -0,0 +1,35 @@ +/* + * 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.connect.mirror.LegacyReplicationPolicy; + +import java.util.Map; +import java.util.HashMap; + +import org.junit.jupiter.api.Tag; + +@Tag("integration") +public class MirrorConnectorsIntegrationLegacyTest extends MirrorConnectorsIntegrationBaseTest { + @Override + protected Map configOverrides() { + Map m = new HashMap<>(); + m.put("replication.policy.class", LegacyReplicationPolicy.class.getName()); + m.put("remote.topic.suffix", ".replica"); + return m; + } +} From 29a95e61e6713f7e1603311f653c5e7593ff3a0a Mon Sep 17 00:00:00 2001 From: Ryanne Dolan Date: Tue, 18 May 2021 16:36:35 -0500 Subject: [PATCH 06/13] WIP fix integration tests -- two FIXMEs --- .../mirror/LegacyReplicationPolicy.java | 13 +- .../kafka/connect/mirror/MirrorClient.java | 2 +- .../connect/mirror/MirrorMakerConfig.java | 2 + .../MirrorConnectorsIntegrationBaseTest.java | 121 +++++++++++------- ...MirrorConnectorsIntegrationLegacyTest.java | 2 +- .../util/clusters/EmbeddedKafkaCluster.java | 2 +- 6 files changed, 92 insertions(+), 50 deletions(-) diff --git a/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/LegacyReplicationPolicy.java b/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/LegacyReplicationPolicy.java index 1a483147c68bc..75609fab9d659 100644 --- a/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/LegacyReplicationPolicy.java +++ b/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/LegacyReplicationPolicy.java @@ -19,7 +19,9 @@ import org.apache.kafka.common.Configurable; import java.util.Map; -import java.util.regex.Pattern; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** LegacyReplicationPolicy attempts to mimic MirrorMaker v1, and thus by default does not * rename topics. In order to support active/active replication with careful configuration, @@ -39,8 +41,9 @@ * already be the case. */ public class LegacyReplicationPolicy implements ReplicationPolicy, Configurable { + private static final Logger log = LoggerFactory.getLogger(LegacyReplicationPolicy.class); - public static final String REMOTE_TOPIC_SUFFIX_CONFIG = "remote.topic.suffix"; + public static final String REMOTE_TOPIC_SUFFIX_CONFIG = "replication.policy.remote.topic.suffix"; public static final String SOURCE_CLUSTER_ALIAS_CONFIG = "source.cluster.alias"; private String remoteTopicSuffix = ""; @@ -51,17 +54,19 @@ public LegacyReplicationPolicy() { // Visible for testing LegacyReplicationPolicy(String remoteTopicSuffix, String sourceClusterAlias) { - this.remoteTopicSuffix = remoteTopicSuffix; - this.sourceClusterAlias = sourceClusterAlias; + this.remoteTopicSuffix = remoteTopicSuffix; + this.sourceClusterAlias = sourceClusterAlias; } @Override public void configure(Map props) { if (props.containsKey(REMOTE_TOPIC_SUFFIX_CONFIG)) { remoteTopicSuffix = (String) props.get(REMOTE_TOPIC_SUFFIX_CONFIG); + log.info("Using custom remote topic suffix `{}`.", remoteTopicSuffix); } if (props.containsKey(SOURCE_CLUSTER_ALIAS_CONFIG)) { sourceClusterAlias = (String) props.get(SOURCE_CLUSTER_ALIAS_CONFIG); + log.info("Using source cluster alias `{}`.", sourceClusterAlias); } } diff --git a/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/MirrorClient.java b/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/MirrorClient.java index 775ec782c2d41..aacabe70a5a56 100644 --- a/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/MirrorClient.java +++ b/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/MirrorClient.java @@ -60,7 +60,7 @@ public class MirrorClient implements AutoCloseable { private ReplicationPolicy replicationPolicy; private Map consumerConfig; - public MirrorClient(Map props) { + public MirrorClient(Map props) { this(new MirrorClientConfig(props)); } diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorMakerConfig.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorMakerConfig.java index 298bce34fe531..33cd8a7ac3031 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorMakerConfig.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorMakerConfig.java @@ -175,6 +175,7 @@ public Map workerConfig(SourceAndTarget sourceAndTarget) { props.putAll(stringsWithPrefix("header.converter")); props.putAll(stringsWithPrefix("task")); props.putAll(stringsWithPrefix("worker")); + props.putAll(stringsWithPrefix("replication.policy")); // transform any expression like ${provider:path:key}, since the worker doesn't do so props = transform(props); @@ -203,6 +204,7 @@ public Map connectorBaseConfig(SourceAndTarget sourceAndTarget, props.keySet().retainAll(MirrorConnectorConfig.CONNECTOR_CONFIG_DEF.names()); props.putAll(stringsWithPrefix(CONFIG_PROVIDERS_CONFIG)); + props.putAll(stringsWithPrefix("replication.policy")); Map sourceClusterProps = clusterProps(sourceAndTarget.source()); // attrs non prefixed with producer|consumer|admin diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationBaseTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationBaseTest.java index 93beab350400d..3bc2772274b60 100644 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationBaseTest.java +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationBaseTest.java @@ -101,7 +101,8 @@ public abstract class MirrorConnectorsIntegrationBaseTest { private EmbeddedConnectCluster backup; private MirrorClient primaryClient; private MirrorClient backupClient; - private ReplicationPolicy replicationPolicy; + private ReplicationPolicy primaryReplicationPolicy; + private ReplicationPolicy backupReplicationPolicy; private Exit.Procedure exitProcedure; private Exit.Procedure haltProcedure; @@ -177,7 +178,7 @@ public void startClusters() throws Exception { waitForTopicCreated(primary, "mm2-status.backup.internal"); waitForTopicCreated(primary, "mm2-offsets.backup.internal"); waitForTopicCreated(primary, "mm2-configs.backup.internal"); - + backup.start(); backup.assertions().assertAtLeastNumWorkersAreUp(NUM_WORKERS, "Workers of " + BACKUP_CLUSTER_ALIAS + "-connect-cluster did not start in time."); @@ -190,9 +191,16 @@ public void startClusters() throws Exception { mm2Props.put(PRIMARY_CLUSTER_ALIAS + ".bootstrap.servers", primary.kafka().bootstrapServers()); mm2Props.put(BACKUP_CLUSTER_ALIAS + ".bootstrap.servers", backup.kafka().bootstrapServers()); mm2Config = new MirrorMakerConfig(mm2Props); - primaryClient = new MirrorClient(mm2Config.clientConfig(PRIMARY_CLUSTER_ALIAS)); - backupClient = new MirrorClient(mm2Config.clientConfig(BACKUP_CLUSTER_ALIAS)); - replicationPolicy = primaryClient.replicationPolicy(); // assume same in both clusters + Map primaryClientConfig = mm2Config.clientConfig(PRIMARY_CLUSTER_ALIAS).originals(); + // hack for LegacyReplicationPolicy, which needs a hint for source.cluster.alias + primaryClientConfig.put("source.cluster.alias", BACKUP_CLUSTER_ALIAS); + primaryClient = new MirrorClient(primaryClientConfig); + Map backupClientConfig = mm2Config.clientConfig(BACKUP_CLUSTER_ALIAS).originals(); + // hack for LegacyReplicationPolicy, which needs a hint for source.cluster.alias + backupClientConfig.put("source.cluster.alias", PRIMARY_CLUSTER_ALIAS); + backupClient = new MirrorClient(backupClientConfig); + primaryReplicationPolicy = primaryClient.replicationPolicy(); + backupReplicationPolicy = backupClient.replicationPolicy(); createTopics(); @@ -248,33 +256,39 @@ public void testReplication() throws Exception { waitUntilMirrorMakerIsRunning(primary, CONNECTOR_LIST, mm2Config, BACKUP_CLUSTER_ALIAS, PRIMARY_CLUSTER_ALIAS); // make sure the topic is auto-created in the other cluster - waitForTopicCreated(primary, formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-1")); - waitForTopicCreated(backup, formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1")); -// FIXME assertEquals(TopicConfig.CLEANUP_POLICY_COMPACT, getTopicConfig(backup.kafka(), -// formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1"), -// TopicConfig.CLEANUP_POLICY_CONFIG), "topic config was not synced"); + waitForTopicCreated(primary, primaryReplicationPolicy.formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-1")); + waitForTopicCreated(backup, backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1")); + assertEquals(TopicConfig.CLEANUP_POLICY_COMPACT, getTopicConfig(backup.kafka(), + backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1"), + TopicConfig.CLEANUP_POLICY_CONFIG), "topic config was not synced"); assertEquals(NUM_RECORDS_PRODUCED, primary.kafka().consume(NUM_RECORDS_PRODUCED, RECORD_TRANSFER_DURATION_MS, "test-topic-1").count(), "Records were not produced to primary cluster."); - assertEquals(NUM_RECORDS_PRODUCED, backup.kafka().consume(NUM_RECORDS_PRODUCED, RECORD_TRANSFER_DURATION_MS, formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1")).count(), + assertEquals(NUM_RECORDS_PRODUCED, backup.kafka().consume(NUM_RECORDS_PRODUCED, RECORD_TRANSFER_DURATION_MS, + backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1")).count(), "Records were not replicated to backup cluster."); assertEquals(NUM_RECORDS_PRODUCED, backup.kafka().consume(NUM_RECORDS_PRODUCED, RECORD_TRANSFER_DURATION_MS, "test-topic-1").count(), "Records were not produced to backup cluster."); - assertEquals(NUM_RECORDS_PRODUCED, primary.kafka().consume(NUM_RECORDS_PRODUCED, RECORD_TRANSFER_DURATION_MS, formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-1")).count(), + assertEquals(NUM_RECORDS_PRODUCED, primary.kafka().consume(NUM_RECORDS_PRODUCED, RECORD_TRANSFER_DURATION_MS, + primaryReplicationPolicy.formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-1")).count(), "Records were not replicated to primary cluster."); - assertEquals(NUM_RECORDS_PRODUCED * 2, primary.kafka().consume(NUM_RECORDS_PRODUCED * 2, RECORD_TRANSFER_DURATION_MS, formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-1"), "test-topic-1").count(), + assertEquals(NUM_RECORDS_PRODUCED * 2, primary.kafka().consume(NUM_RECORDS_PRODUCED * 2, RECORD_TRANSFER_DURATION_MS, + primaryReplicationPolicy.formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-1"), "test-topic-1").count(), "Primary cluster doesn't have all records from both clusters."); - assertEquals(NUM_RECORDS_PRODUCED * 2, backup.kafka().consume(NUM_RECORDS_PRODUCED * 2, RECORD_TRANSFER_DURATION_MS, formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1"), "test-topic-1").count(), + assertEquals(NUM_RECORDS_PRODUCED * 2, backup.kafka().consume(NUM_RECORDS_PRODUCED * 2, RECORD_TRANSFER_DURATION_MS, + backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1"), "test-topic-1").count(), "Backup cluster doesn't have all records from both clusters."); assertTrue(primary.kafka().consume(1, RECORD_TRANSFER_DURATION_MS, "heartbeats").count() > 0, "Heartbeats were not emitted to primary cluster."); assertTrue(backup.kafka().consume(1, RECORD_TRANSFER_DURATION_MS, "heartbeats").count() > 0, "Heartbeats were not emitted to backup cluster."); - assertTrue(backup.kafka().consume(1, RECORD_TRANSFER_DURATION_MS, "primary.heartbeats").count() > 0, + assertTrue(backup.kafka().consume(1, RECORD_TRANSFER_DURATION_MS, + backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "heartbeats")).count() > 0, "Heartbeats were not replicated downstream to backup cluster."); - assertTrue(primary.kafka().consume(1, RECORD_TRANSFER_DURATION_MS, "backup.heartbeats").count() > 0, + assertTrue(primary.kafka().consume(1, RECORD_TRANSFER_DURATION_MS, + primaryReplicationPolicy.formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "heartbeats")).count() > 0, "Heartbeats were not replicated downstream to primary cluster."); assertTrue(backupClient.upstreamClusters().contains(PRIMARY_CLUSTER_ALIAS), "Did not find upstream primary cluster."); @@ -288,7 +302,8 @@ public void testReplication() throws Exception { Duration.ofMillis(CHECKPOINT_DURATION_MS)); assertTrue(backupOffsets.containsKey( - new TopicPartition(formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1"), 0)), "Offsets not translated downstream to backup cluster. Found: " + backupOffsets); + new TopicPartition(backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1"), 0)), + "Offsets not translated downstream to backup cluster. Found: " + backupOffsets); // Failover consumer group to backup cluster. try (Consumer primaryConsumer = backup.kafka().createConsumer(Collections.singletonMap("group.id", consumerGroupName))) { @@ -297,18 +312,26 @@ public void testReplication() throws Exception { primaryConsumer.poll(CONSUMER_POLL_TIMEOUT_MS); primaryConsumer.commitAsync(); - assertTrue(primaryConsumer.position(new TopicPartition(formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1"), 0)) > 0, "Consumer failedover to zero offset."); assertTrue(primaryConsumer.position( - new TopicPartition(formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1"), 0)) <= NUM_RECORDS_PRODUCED, "Consumer failedover beyond expected offset."); + new TopicPartition(backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1"), 0)) > 0, + "Consumer failedover to zero offset."); + assertTrue(primaryConsumer.position( + new TopicPartition(backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1"), 0)) <= NUM_RECORDS_PRODUCED, + "Consumer failedover beyond expected offset."); assertTrue(primary.kafka().consume(1, CHECKPOINT_DURATION_MS, "backup.checkpoints.internal").count() > 0, "Checkpoints were not emitted upstream to primary cluster."); } waitForCondition(() -> primaryClient.remoteConsumerOffsets(consumerGroupName, BACKUP_CLUSTER_ALIAS, - Duration.ofMillis(CHECKPOINT_DURATION_MS)).containsKey(new TopicPartition(formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-1"), 0)), CHECKPOINT_DURATION_MS, "Offsets not translated downstream to primary cluster."); + Duration.ofMillis(CHECKPOINT_DURATION_MS)).containsKey( + new TopicPartition(primaryReplicationPolicy.formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-1"), 0)), + CHECKPOINT_DURATION_MS, "Offsets not translated downstream to primary cluster."); +/* FIXME waitForCondition(() -> primaryClient.remoteConsumerOffsets(consumerGroupName, BACKUP_CLUSTER_ALIAS, - Duration.ofMillis(CHECKPOINT_DURATION_MS)).containsKey(new TopicPartition("test-topic-1", 0)), CHECKPOINT_DURATION_MS, "Offsets not translated upstream to primary cluster."); + Duration.ofMillis(CHECKPOINT_DURATION_MS)).containsKey(new TopicPartition("test-topic-1", 0)), CHECKPOINT_DURATION_MS, + "Offsets not translated upstream to primary cluster."); +*/ Map primaryOffsets = primaryClient.remoteConsumerOffsets(consumerGroupName, BACKUP_CLUSTER_ALIAS, Duration.ofMillis(CHECKPOINT_DURATION_MS)); @@ -322,13 +345,21 @@ public void testReplication() throws Exception { primaryOffsets.forEach(backupConsumer::seek); backupConsumer.poll(CONSUMER_POLL_TIMEOUT_MS); backupConsumer.commitAsync(); - + +/* FIXME assertTrue(backupConsumer.position(new TopicPartition("test-topic-1", 0)) > 0, "Consumer failedback to zero upstream offset."); - assertTrue(backupConsumer.position(new TopicPartition(formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-1"), 0)) > 0, "Consumer failedback to zero downstream offset."); +*/ + assertTrue(backupConsumer.position( + new TopicPartition(primaryReplicationPolicy.formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-1"), 0)) > 0, + "Consumer failedback to zero downstream offset."); + +/* assertTrue(backupConsumer.position( new TopicPartition("test-topic-1", 0)) <= NUM_RECORDS_PRODUCED, "Consumer failedback beyond expected upstream offset."); +*/ assertTrue(backupConsumer.position( - new TopicPartition(formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-1"), 0)) <= NUM_RECORDS_PRODUCED, "Consumer failedback beyond expected downstream offset."); + new TopicPartition(primaryReplicationPolicy.formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-1"), 0)) <= NUM_RECORDS_PRODUCED, + "Consumer failedback beyond expected downstream offset."); } // create more matching topics @@ -336,8 +367,8 @@ public void testReplication() throws Exception { backup.kafka().createTopic("test-topic-3", NUM_PARTITIONS); // make sure the topic is auto-created in the other cluster - waitForTopicCreated(backup, formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-2")); - waitForTopicCreated(primary, formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-3")); + waitForTopicCreated(backup, backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-2")); + waitForTopicCreated(primary, primaryReplicationPolicy.formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-3")); // only produce messages to the first partition produceMessages(primary, "test-topic-2", 1); @@ -349,9 +380,11 @@ public void testReplication() throws Exception { assertEquals(NUM_RECORDS_PER_PARTITION, backup.kafka().consume(NUM_RECORDS_PER_PARTITION, RECORD_TRANSFER_DURATION_MS, "test-topic-3").count(), "Records were not produced to backup cluster."); - assertEquals(NUM_RECORDS_PER_PARTITION, primary.kafka().consume(NUM_RECORDS_PER_PARTITION, 2 * RECORD_TRANSFER_DURATION_MS, formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-3")).count(), + assertEquals(NUM_RECORDS_PER_PARTITION, primary.kafka().consume(NUM_RECORDS_PER_PARTITION, 2 * RECORD_TRANSFER_DURATION_MS, + primaryReplicationPolicy.formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-3")).count(), "New topic was not replicated to primary cluster."); - assertEquals(NUM_RECORDS_PER_PARTITION, backup.kafka().consume(NUM_RECORDS_PER_PARTITION, 2 * RECORD_TRANSFER_DURATION_MS, formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-2")).count(), + assertEquals(NUM_RECORDS_PER_PARTITION, backup.kafka().consume(NUM_RECORDS_PER_PARTITION, 2 * RECORD_TRANSFER_DURATION_MS, + backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-2")).count(), "New topic was not replicated to backup cluster."); } @@ -383,7 +416,7 @@ public void testReplicationWithEmptyPartition() throws Exception { // consume all records from backup cluster try (Consumer backupConsumer = backup.kafka().createConsumerAndSubscribeTo(consumerProps, - replicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, topic))) { + backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, topic))) { waitForConsumingAllRecords(backupConsumer, expectedRecords); } @@ -394,7 +427,7 @@ public void testReplicationWithEmptyPartition() throws Exception { // pinpoint the offset of the last partition which does not receive records OffsetAndMetadata offset = remoteOffsets.get(new TopicPartition( - replicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, topic), NUM_PARTITIONS - 1)); + backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, topic), NUM_PARTITIONS - 1)); // offset of the last partition should exist, but its value should be 0 assertNotNull(offset, "Offset of last partition was not replicated"); assertEquals(0, offset.offset(), "Offset of last partition is not zero"); @@ -427,13 +460,14 @@ public void testOneWayReplicationWithAutoOffsetSync() throws InterruptedExceptio waitUntilMirrorMakerIsRunning(backup, CONNECTOR_LIST, mm2Config, PRIMARY_CLUSTER_ALIAS, BACKUP_CLUSTER_ALIAS); // make sure the topic is created in the other cluster - waitForTopicCreated(primary, formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-1")); - waitForTopicCreated(backup, formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1")); + waitForTopicCreated(primary, primaryReplicationPolicy.formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-1")); + waitForTopicCreated(backup, backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1")); // create a consumer at backup cluster with same consumer group Id to consume 1 topic Consumer backupConsumer = backup.kafka().createConsumerAndSubscribeTo( - consumerProps, formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1")); + consumerProps, backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1")); - waitForConsumerGroupOffsetSync(backup, backupConsumer, Collections.singletonList(formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1")), + waitForConsumerGroupOffsetSync(backup, backupConsumer, Collections.singletonList( + backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1")), consumerGroupName, NUM_RECORDS_PRODUCED); ConsumerRecords records = backupConsumer.poll(CONSUMER_POLL_TIMEOUT_MS); @@ -446,7 +480,7 @@ public void testOneWayReplicationWithAutoOffsetSync() throws InterruptedExceptio // now create a new topic in primary cluster primary.kafka().createTopic("test-topic-2", NUM_PARTITIONS); // make sure the topic is created in backup cluster - waitForTopicCreated(backup, formatRemoteTopic("primary", "test-topic-2")); + waitForTopicCreated(backup, backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-2")); // produce some records to the new topic in primary cluster produceMessages(primary, "test-topic-2"); @@ -460,9 +494,12 @@ public void testOneWayReplicationWithAutoOffsetSync() throws InterruptedExceptio // 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), formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1"), formatRemoteTopic("primary", "test-topic-2")); + "group.id", consumerGroupName), backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1"), + backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-2")); - waitForConsumerGroupOffsetSync(backup, backupConsumer, Arrays.asList(formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1"), formatRemoteTopic("primary", "test-topic-2")), + waitForConsumerGroupOffsetSync(backup, backupConsumer, Arrays.asList( + backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1"), + backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-2")), consumerGroupName, NUM_RECORDS_PRODUCED); records = backupConsumer.poll(CONSUMER_POLL_TIMEOUT_MS); @@ -634,10 +671,12 @@ private void createTopics() { // create these topics before starting the connectors so we don't need to wait for discovery primary.kafka().createTopic("test-topic-1", NUM_PARTITIONS, 1, topicConfig, adminClientConfig); - primary.kafka().createTopic(formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-1"), 1, 1, emptyMap, adminClientConfig); + primary.kafka().createTopic(primaryReplicationPolicy.formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-1"), + 1, 1, emptyMap, adminClientConfig); primary.kafka().createTopic("heartbeats", 1, 1, emptyMap, adminClientConfig); backup.kafka().createTopic("test-topic-1", NUM_PARTITIONS, 1, emptyMap, adminClientConfig); - backup.kafka().createTopic(formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1"), 1, 1, emptyMap, adminClientConfig); + backup.kafka().createTopic(backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1"), + 1, 1, emptyMap, adminClientConfig); backup.kafka().createTopic("heartbeats", 1, 1, emptyMap, adminClientConfig); } @@ -654,8 +693,4 @@ private void warmUpConsumer(Map consumerProps) throws Interrupte dummyConsumer.commitSync(); dummyConsumer.close(); } - - private String formatRemoteTopic(String source, String topic) { - return replicationPolicy.formatRemoteTopic(source, topic); - } } diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationLegacyTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationLegacyTest.java index afcedca1c4643..863e5408efbe6 100644 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationLegacyTest.java +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationLegacyTest.java @@ -29,7 +29,7 @@ public class MirrorConnectorsIntegrationLegacyTest extends MirrorConnectorsInteg protected Map configOverrides() { Map m = new HashMap<>(); m.put("replication.policy.class", LegacyReplicationPolicy.class.getName()); - m.put("remote.topic.suffix", ".replica"); + m.put("replication.policy.remote.topic.suffix", ".replica"); return m; } } 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 30d41d26a56e2..ace0b34007d30 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 @@ -364,7 +364,7 @@ public void createTopic(String topic, int partitions, int replication, Map Date: Thu, 20 May 2021 17:05:08 -0500 Subject: [PATCH 07/13] steal mdedetrich's tests and IdentityReplicationPolicy, but without API changes --- .../mirror/IdentityReplicationPolicy.java | 92 ++++++ .../kafka/connect/mirror/MirrorClient.java | 3 +- ...ntityReplicationPolicyIntegrationTest.java | 263 ++++++++++++++++++ .../MirrorConnectorsIntegrationBaseTest.java | 199 +++++-------- ...MirrorConnectorsIntegrationLegacyTest.java | 35 --- 5 files changed, 433 insertions(+), 159 deletions(-) create mode 100644 connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/IdentityReplicationPolicy.java create mode 100644 connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIdentityReplicationPolicyIntegrationTest.java delete mode 100644 connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationLegacyTest.java diff --git a/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/IdentityReplicationPolicy.java b/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/IdentityReplicationPolicy.java new file mode 100644 index 0000000000000..465f193020c60 --- /dev/null +++ b/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/IdentityReplicationPolicy.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.mirror; + +import java.util.Map; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** IdentityReplicationPolicy does not rename remote topics. This is useful for migrating + * from legacy MM1, or for any use-case involving one-way replication. + * + * N.B. MirrorMaker is not able to prevent cycles when using this class, so take care that + * your replication topology is acyclic. If migrating from MirrorMaker v1, this will likely + * already be the case. + */ +public class IdentityReplicationPolicy extends DefaultReplicationPolicy { + private static final Logger log = LoggerFactory.getLogger(IdentityReplicationPolicy.class); + + public static final String SOURCE_CLUSTER_ALIAS_CONFIG = "source.cluster.alias"; + + private String sourceClusterAlias = null; + + @Override + public void configure(Map props) { + super.configure(props); + if (props.containsKey(SOURCE_CLUSTER_ALIAS_CONFIG)) { + sourceClusterAlias = (String) props.get(SOURCE_CLUSTER_ALIAS_CONFIG); + log.info("Using source cluster alias `{}`.", sourceClusterAlias); + } + } + + /** Unlike DefaultReplicationPolicy, IdentityReplicationPolicy does not include the source + * cluster alias in the remote topic name. Instead, topic names are unchanged. + * + * In the special case of heartbeats, we defer to DefaultReplicationPolicy. + */ + @Override + public String formatRemoteTopic(String sourceClusterAlias, String topic) { + if (looksLikeHeartbeat(topic)) { + return super.formatRemoteTopic(sourceClusterAlias, topic); + } else { + return topic; + } + } + + /** Unlike DefaultReplicationPolicy, IdendityReplicationPolicy cannot know the source of + * a remote topic based on its name alone. If `source.cluster.alias` is provided, + * `topicSource` will return that. + * + * In the special case of heartbeast, we defer to DefaultReplicationPolicy. + */ + @Override + public String topicSource(String topic) { + if (looksLikeHeartbeat(topic)) { + return super.topicSource(topic); + } else { + return sourceClusterAlias; + } + } + + /** Since any topic may be a "remote topic", this just returns `topic`. + * + * In the special case of heartbeast, we defer to DefaultReplicationPolicy. + */ + @Override + public String upstreamTopic(String topic) { + if (looksLikeHeartbeat(topic)) { + return super.upstreamTopic(topic); + } else { + return topic; + } + } + + private boolean looksLikeHeartbeat(String topic) { + return topic != null && topic.endsWith(MirrorClientConfig.HEARTBEATS_TOPIC); + } +} diff --git a/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/MirrorClient.java b/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/MirrorClient.java index aacabe70a5a56..acc1796e0a22a 100644 --- a/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/MirrorClient.java +++ b/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/MirrorClient.java @@ -231,8 +231,7 @@ Set allSources(String topic) { Set sources = new HashSet<>(); String source = replicationPolicy.topicSource(topic); while (source != null && !sources.contains(source)) { - // The extra Set.contains above is for LegacyReplicationPolicy (and any other dumb impls) which - // cannot prevent cycles. In impls that do prevent cycles, we shouldn't see the same source twice. + // The extra Set.contains above is for ReplicationPolicies that cannot prevent cycles. sources.add(source); topic = replicationPolicy.upstreamTopic(topic); source = replicationPolicy.topicSource(topic); diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIdentityReplicationPolicyIntegrationTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIdentityReplicationPolicyIntegrationTest.java new file mode 100644 index 0000000000000..43abad3f53022 --- /dev/null +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIdentityReplicationPolicyIntegrationTest.java @@ -0,0 +1,263 @@ +/* + * 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.TopicPartition; +import org.apache.kafka.connect.mirror.IdentityReplicationPolicy; +import org.apache.kafka.connect.mirror.MirrorClient; +import org.apache.kafka.connect.mirror.MirrorHeartbeatConnector; +import org.apache.kafka.connect.mirror.MirrorMakerConfig; + +import java.time.Duration; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +import org.junit.jupiter.api.Tag; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.BeforeEach; + +/** + * Tests MM2 replication and failover logic for {@link IdentityReplicationPolicy}. + * + *

MM2 is configured with active/passive replication between two Kafka clusters with {@link IdentityReplicationPolicy}. + * Tests validate that records sent to the primary cluster arrive at the backup cluster. Then, a consumer group is + * migrated from the primary cluster to the backup cluster. Tests validate that consumer offsets + * are translated and replicated from the primary cluster to the backup cluster during this failover. + */ +@Tag("integration") +public class MirrorConnectorsIdentityReplicationPolicyIntegrationTest extends MirrorConnectorsIntegrationTest { + @BeforeEach + public void startClusters() throws Exception { + super.startClusters(new HashMap() {{ + put("replication.policy.class", IdentityReplicationPolicy.class.getName()); + put("topics", "test-topic-.*"); + put(BACKUP_CLUSTER_ALIAS + "->" + PRIMARY_CLUSTER_ALIAS + ".enabled", "false"); + put(PRIMARY_CLUSTER_ALIAS + "->" + BACKUP_CLUSTER_ALIAS + ".enabled", "true"); + }}); + } + + @Test + public void testReplication() throws Exception { + produceMessages(primary, "test-topic-1"); + String consumerGroupName = "consumer-group-testReplication"; + Map consumerProps = new HashMap() {{ + put("group.id", consumerGroupName); + put("auto.offset.reset", "latest"); + }}; + // warm up consumers before starting the connectors so we don't need to wait for discovery + warmUpConsumer(consumerProps); + + mm2Config = new MirrorMakerConfig(mm2Props); + + waitUntilMirrorMakerIsRunning(backup, CONNECTOR_LIST, mm2Config, PRIMARY_CLUSTER_ALIAS, BACKUP_CLUSTER_ALIAS); + waitUntilMirrorMakerIsRunning(primary, Collections.singletonList(MirrorHeartbeatConnector.class), 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)); + + // make sure the topic is auto-created in the other cluster + waitForTopicCreated(primary, "test-topic-1"); + waitForTopicCreated(backup, "test-topic-1"); + assertEquals(TopicConfig.CLEANUP_POLICY_COMPACT, getTopicConfig(backup.kafka(), "test-topic-1", TopicConfig.CLEANUP_POLICY_CONFIG), + "topic config was not synced"); + + assertEquals(NUM_RECORDS_PRODUCED, primary.kafka().consume(NUM_RECORDS_PRODUCED, RECORD_TRANSFER_DURATION_MS, "test-topic-1").count(), + "Records were not produced to primary cluster."); + assertEquals(NUM_RECORDS_PRODUCED, backup.kafka().consume(NUM_RECORDS_PRODUCED, RECORD_TRANSFER_DURATION_MS, "test-topic-1").count(), + "Records were not replicated to backup cluster."); + + assertTrue(primary.kafka().consume(1, RECORD_TRANSFER_DURATION_MS, "heartbeats").count() > 0, + "Heartbeats were not emitted to primary cluster."); + assertTrue(backup.kafka().consume(1, RECORD_TRANSFER_DURATION_MS, "heartbeats").count() > 0, + "Heartbeats were not emitted to backup cluster."); + assertTrue(backup.kafka().consume(1, RECORD_TRANSFER_DURATION_MS, "primary.heartbeats").count() > 0, + "Heartbeats were not replicated downstream to backup cluster."); + assertTrue(primary.kafka().consume(1, RECORD_TRANSFER_DURATION_MS, "heartbeats").count() > 0, + "Heartbeats were not replicated downstream to primary cluster."); + + assertTrue(backupClient.upstreamClusters().contains(PRIMARY_CLUSTER_ALIAS), "Did not find upstream primary cluster."); + assertEquals(1, backupClient.replicationHops(PRIMARY_CLUSTER_ALIAS), "Did not calculate replication hops correctly."); + assertTrue(backup.kafka().consume(1, CHECKPOINT_DURATION_MS, "primary.checkpoints.internal").count() > 0, + "Checkpoints were not emitted downstream to backup cluster."); + + Map backupOffsets = backupClient.remoteConsumerOffsets(consumerGroupName, PRIMARY_CLUSTER_ALIAS, + Duration.ofMillis(CHECKPOINT_DURATION_MS)); + + assertTrue(backupOffsets.containsKey( + new TopicPartition("test-topic-1", 0)), "Offsets not translated downstream to backup cluster. Found: " + backupOffsets); + + // Failover consumer group to backup cluster. + try (Consumer primaryConsumer = backup.kafka().createConsumer(Collections.singletonMap("group.id", consumerGroupName))) { + primaryConsumer.assign(backupOffsets.keySet()); + backupOffsets.forEach(primaryConsumer::seek); + primaryConsumer.poll(CONSUMER_POLL_TIMEOUT_MS); + primaryConsumer.commitAsync(); + + assertTrue(primaryConsumer.position(new TopicPartition("test-topic-1", 0)) > 0, "Consumer failedover to zero offset."); + assertTrue(primaryConsumer.position( + new TopicPartition("test-topic-1", 0)) <= NUM_RECORDS_PRODUCED, "Consumer failedover beyond expected offset."); + } + + primaryClient.close(); + backupClient.close(); + + // create more matching topics + primary.kafka().createTopic("test-topic-2", NUM_PARTITIONS); + + // make sure the topic is auto-created in the other cluster + waitForTopicCreated(backup, "test-topic-2"); + + // only produce messages to the first partition + produceMessages(primary, "test-topic-2", 1); + + // expect total consumed messages equals to NUM_RECORDS_PER_PARTITION + assertEquals(NUM_RECORDS_PER_PARTITION, primary.kafka().consume(NUM_RECORDS_PER_PARTITION, RECORD_TRANSFER_DURATION_MS, "test-topic-2").count(), + "Records were not produced to primary cluster."); + assertEquals(NUM_RECORDS_PER_PARTITION, backup.kafka().consume(NUM_RECORDS_PER_PARTITION, 2 * RECORD_TRANSFER_DURATION_MS, "test-topic-2").count(), + "New topic was not replicated to backup cluster."); + } + + @Test + public void testReplicationWithEmptyPartition() throws Exception { + String consumerGroupName = "consumer-group-testReplicationWithEmptyPartition"; + Map 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 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)); + + // note that with LegacyReplicationPolicy, topics on the backup are NOT renamed to PRIMARY_CLUSTER_ALIAS + "." + topic + String backupTopic = topic; + + // consume all records from backup cluster + try (Consumer backupConsumer = backup.kafka().createConsumerAndSubscribeTo(consumerProps, + backupTopic)) { + waitForConsumingAllRecords(backupConsumer, expectedRecords); + } + + try (Admin backupClient = backup.kafka().createAdminClient()) { + // retrieve the consumer group offset from backup cluster + Map remoteOffsets = + backupClient.listConsumerGroupOffsets(consumerGroupName).partitionsToOffsetAndMetadata().get(); + + // pinpoint the offset of the last partition which does not receive records + OffsetAndMetadata offset = remoteOffsets.get(new TopicPartition(backupTopic, NUM_PARTITIONS - 1)); + // offset of the last partition should exist, but its value should be 0 + assertNotNull(offset, "Offset of last partition was not replicated"); + assertEquals(0, offset.offset(), "Offset of last partition is not zero"); + } + } + + @Test + public void testOneWayReplicationWithAutoOffsetSync() throws InterruptedException { + produceMessages(primary, "test-topic-1"); + String consumerGroupName = "consumer-group-testOneWayReplicationWithAutoOffsetSync"; + Map consumerProps = new HashMap() {{ + 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 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); + + // make sure the topic is created in the other cluster + waitForTopicCreated(primary, "backup.test-topic-1"); + waitForTopicCreated(backup, "test-topic-1"); + // create a consumer at backup cluster with same consumer group Id to consume 1 topic + Consumer backupConsumer = backup.kafka().createConsumerAndSubscribeTo( + consumerProps, "test-topic-1"); + + waitForConsumerGroupOffsetSync(backup, backupConsumer, Collections.singletonList("test-topic-1"), + consumerGroupName, NUM_RECORDS_PRODUCED); + + ConsumerRecords records = backupConsumer.poll(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(0, records.count(), "consumer record size is not zero"); + + // now create a new topic in primary cluster + primary.kafka().createTopic("test-topic-2", NUM_PARTITIONS); + // make sure the topic is created in backup cluster + waitForTopicCreated(backup, "test-topic-2"); + + // 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 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), "test-topic-1", "test-topic-2"); + + waitForConsumerGroupOffsetSync(backup, backupConsumer, Arrays.asList("test-topic-1", "test-topic-2"), + consumerGroupName, NUM_RECORDS_PRODUCED); + + records = backupConsumer.poll(CONSUMER_POLL_TIMEOUT_MS); + // similar reasoning as above, no more records to consume by the same consumer group at backup cluster + assertEquals(0, records.count(), "consumer record size is not zero"); + backupConsumer.close(); + } +} diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationBaseTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationBaseTest.java index 3bc2772274b60..c7d308aa3381d 100644 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationBaseTest.java +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationBaseTest.java @@ -34,7 +34,6 @@ 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.mirror.ReplicationPolicy; import org.apache.kafka.connect.util.clusters.EmbeddedConnectCluster; import org.apache.kafka.connect.util.clusters.EmbeddedKafkaCluster; import org.apache.kafka.connect.util.clusters.UngracefulShutdownException; @@ -78,33 +77,29 @@ 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; + protected static final int NUM_RECORDS_PER_PARTITION = 10; + protected 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; private static final int RECORD_CONSUME_DURATION_MS = 20_000; private static final int OFFSET_SYNC_DURATION_MS = 30_000; private static final int TOPIC_SYNC_DURATION_MS = 60_000; private static final int REQUEST_TIMEOUT_DURATION_MS = 60_000; private static final int NUM_WORKERS = 3; - private static final Duration CONSUMER_POLL_TIMEOUT_MS = Duration.ofMillis(500); + protected static final Duration CONSUMER_POLL_TIMEOUT_MS = Duration.ofMillis(500); protected static final String PRIMARY_CLUSTER_ALIAS = "primary"; protected static final String BACKUP_CLUSTER_ALIAS = "backup"; - private static final List> CONNECTOR_LIST = + protected static final List> CONNECTOR_LIST = Arrays.asList(MirrorSourceConnector.class, MirrorCheckpointConnector.class, MirrorHeartbeatConnector.class); private volatile boolean shuttingDown; protected Map mm2Props = new HashMap<>(); - private MirrorMakerConfig mm2Config; - private EmbeddedConnectCluster primary; - private EmbeddedConnectCluster backup; - private MirrorClient primaryClient; - private MirrorClient backupClient; - private ReplicationPolicy primaryReplicationPolicy; - private ReplicationPolicy backupReplicationPolicy; + protected MirrorMakerConfig mm2Config; + protected EmbeddedConnectCluster primary; + protected EmbeddedConnectCluster backup; - private Exit.Procedure exitProcedure; + protected Exit.Procedure exitProcedure; private Exit.Procedure haltProcedure; protected Properties primaryBrokerProps = new Properties(); @@ -114,6 +109,14 @@ public abstract class MirrorConnectorsIntegrationBaseTest { @BeforeEach public void startClusters() throws Exception { + startClusters(new HashMap() {{ + put("topics", "test-topic-.*, primary.test-topic-.*, backup.test-topic-.*"); + put(PRIMARY_CLUSTER_ALIAS + "->" + BACKUP_CLUSTER_ALIAS + ".enabled", "true"); + put(BACKUP_CLUSTER_ALIAS + "->" + PRIMARY_CLUSTER_ALIAS + ".enabled", "true"); + }}); + } + + public void startClusters(Map additionalMM2Config) throws Exception { shuttingDown = false; exitProcedure = (code, message) -> { if (shuttingDown) { @@ -144,15 +147,14 @@ public void startClusters() throws Exception { primaryBrokerProps.put("auto.create.topics.enable", "false"); backupBrokerProps.put("auto.create.topics.enable", "false"); - + mm2Props.putAll(basicMM2Config()); - mm2Props.putAll(configOverrides()); + mm2Props.putAll(additionalMM2Config); - // We don't have bootstrap.servers yet, so the MirrorMakerConfig is incomplete at this point. - MirrorMakerConfig tempMM2Config = new MirrorMakerConfig(mm2Props); - primaryWorkerProps = tempMM2Config.workerConfig(new SourceAndTarget(BACKUP_CLUSTER_ALIAS, PRIMARY_CLUSTER_ALIAS)); - backupWorkerProps.putAll(tempMM2Config.workerConfig(new SourceAndTarget(PRIMARY_CLUSTER_ALIAS, BACKUP_CLUSTER_ALIAS))); - + 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(NUM_WORKERS) @@ -187,21 +189,6 @@ public void startClusters() throws Exception { waitForTopicCreated(backup, "mm2-offsets.primary.internal"); waitForTopicCreated(backup, "mm2-configs.primary.internal"); - // 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()); - mm2Config = new MirrorMakerConfig(mm2Props); - Map primaryClientConfig = mm2Config.clientConfig(PRIMARY_CLUSTER_ALIAS).originals(); - // hack for LegacyReplicationPolicy, which needs a hint for source.cluster.alias - primaryClientConfig.put("source.cluster.alias", BACKUP_CLUSTER_ALIAS); - primaryClient = new MirrorClient(primaryClientConfig); - Map backupClientConfig = mm2Config.clientConfig(BACKUP_CLUSTER_ALIAS).originals(); - // hack for LegacyReplicationPolicy, which needs a hint for source.cluster.alias - backupClientConfig.put("source.cluster.alias", PRIMARY_CLUSTER_ALIAS); - backupClient = new MirrorClient(backupClientConfig); - primaryReplicationPolicy = primaryClient.replicationPolicy(); - backupReplicationPolicy = backupClient.replicationPolicy(); - createTopics(); warmUpConsumer(Collections.singletonMap("group.id", "consumer-group-dummy")); @@ -210,6 +197,10 @@ public void startClusters() throws Exception { 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()); } @AfterEach @@ -255,40 +246,36 @@ public void testReplication() throws Exception { 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)); + // make sure the topic is auto-created in the other cluster - waitForTopicCreated(primary, primaryReplicationPolicy.formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-1")); - waitForTopicCreated(backup, backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1")); - assertEquals(TopicConfig.CLEANUP_POLICY_COMPACT, getTopicConfig(backup.kafka(), - backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1"), - TopicConfig.CLEANUP_POLICY_CONFIG), "topic config was not synced"); + waitForTopicCreated(primary, "backup.test-topic-1"); + waitForTopicCreated(backup, "primary.test-topic-1"); + assertEquals(TopicConfig.CLEANUP_POLICY_COMPACT, getTopicConfig(backup.kafka(), "primary.test-topic-1", TopicConfig.CLEANUP_POLICY_CONFIG), + "topic config was not synced"); assertEquals(NUM_RECORDS_PRODUCED, primary.kafka().consume(NUM_RECORDS_PRODUCED, RECORD_TRANSFER_DURATION_MS, "test-topic-1").count(), "Records were not produced to primary cluster."); - assertEquals(NUM_RECORDS_PRODUCED, backup.kafka().consume(NUM_RECORDS_PRODUCED, RECORD_TRANSFER_DURATION_MS, - backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1")).count(), + assertEquals(NUM_RECORDS_PRODUCED, backup.kafka().consume(NUM_RECORDS_PRODUCED, RECORD_TRANSFER_DURATION_MS, "primary.test-topic-1").count(), "Records were not replicated to backup cluster."); assertEquals(NUM_RECORDS_PRODUCED, backup.kafka().consume(NUM_RECORDS_PRODUCED, RECORD_TRANSFER_DURATION_MS, "test-topic-1").count(), "Records were not produced to backup cluster."); - assertEquals(NUM_RECORDS_PRODUCED, primary.kafka().consume(NUM_RECORDS_PRODUCED, RECORD_TRANSFER_DURATION_MS, - primaryReplicationPolicy.formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-1")).count(), + assertEquals(NUM_RECORDS_PRODUCED, primary.kafka().consume(NUM_RECORDS_PRODUCED, RECORD_TRANSFER_DURATION_MS, "backup.test-topic-1").count(), "Records were not replicated to primary cluster."); - assertEquals(NUM_RECORDS_PRODUCED * 2, primary.kafka().consume(NUM_RECORDS_PRODUCED * 2, RECORD_TRANSFER_DURATION_MS, - primaryReplicationPolicy.formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-1"), "test-topic-1").count(), + assertEquals(NUM_RECORDS_PRODUCED * 2, primary.kafka().consume(NUM_RECORDS_PRODUCED * 2, RECORD_TRANSFER_DURATION_MS, "backup.test-topic-1", "test-topic-1").count(), "Primary cluster doesn't have all records from both clusters."); - assertEquals(NUM_RECORDS_PRODUCED * 2, backup.kafka().consume(NUM_RECORDS_PRODUCED * 2, RECORD_TRANSFER_DURATION_MS, - backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1"), "test-topic-1").count(), + assertEquals(NUM_RECORDS_PRODUCED * 2, backup.kafka().consume(NUM_RECORDS_PRODUCED * 2, RECORD_TRANSFER_DURATION_MS, "primary.test-topic-1", "test-topic-1").count(), "Backup cluster doesn't have all records from both clusters."); assertTrue(primary.kafka().consume(1, RECORD_TRANSFER_DURATION_MS, "heartbeats").count() > 0, "Heartbeats were not emitted to primary cluster."); assertTrue(backup.kafka().consume(1, RECORD_TRANSFER_DURATION_MS, "heartbeats").count() > 0, "Heartbeats were not emitted to backup cluster."); - assertTrue(backup.kafka().consume(1, RECORD_TRANSFER_DURATION_MS, - backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "heartbeats")).count() > 0, + assertTrue(backup.kafka().consume(1, RECORD_TRANSFER_DURATION_MS, "primary.heartbeats").count() > 0, "Heartbeats were not replicated downstream to backup cluster."); - assertTrue(primary.kafka().consume(1, RECORD_TRANSFER_DURATION_MS, - primaryReplicationPolicy.formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "heartbeats")).count() > 0, + assertTrue(primary.kafka().consume(1, RECORD_TRANSFER_DURATION_MS, "backup.heartbeats").count() > 0, "Heartbeats were not replicated downstream to primary cluster."); assertTrue(backupClient.upstreamClusters().contains(PRIMARY_CLUSTER_ALIAS), "Did not find upstream primary cluster."); @@ -302,8 +289,7 @@ public void testReplication() throws Exception { Duration.ofMillis(CHECKPOINT_DURATION_MS)); assertTrue(backupOffsets.containsKey( - new TopicPartition(backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1"), 0)), - "Offsets not translated downstream to backup cluster. Found: " + backupOffsets); + new TopicPartition("primary.test-topic-1", 0)), "Offsets not translated downstream to backup cluster. Found: " + backupOffsets); // Failover consumer group to backup cluster. try (Consumer primaryConsumer = backup.kafka().createConsumer(Collections.singletonMap("group.id", consumerGroupName))) { @@ -312,26 +298,18 @@ public void testReplication() throws Exception { primaryConsumer.poll(CONSUMER_POLL_TIMEOUT_MS); primaryConsumer.commitAsync(); + assertTrue(primaryConsumer.position(new TopicPartition("primary.test-topic-1", 0)) > 0, "Consumer failedover to zero offset."); assertTrue(primaryConsumer.position( - new TopicPartition(backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1"), 0)) > 0, - "Consumer failedover to zero offset."); - assertTrue(primaryConsumer.position( - new TopicPartition(backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1"), 0)) <= NUM_RECORDS_PRODUCED, - "Consumer failedover beyond expected offset."); + new TopicPartition("primary.test-topic-1", 0)) <= NUM_RECORDS_PRODUCED, "Consumer failedover beyond expected offset."); assertTrue(primary.kafka().consume(1, CHECKPOINT_DURATION_MS, "backup.checkpoints.internal").count() > 0, "Checkpoints were not emitted upstream to primary cluster."); } waitForCondition(() -> primaryClient.remoteConsumerOffsets(consumerGroupName, BACKUP_CLUSTER_ALIAS, - Duration.ofMillis(CHECKPOINT_DURATION_MS)).containsKey( - new TopicPartition(primaryReplicationPolicy.formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-1"), 0)), - CHECKPOINT_DURATION_MS, "Offsets not translated downstream to primary cluster."); + Duration.ofMillis(CHECKPOINT_DURATION_MS)).containsKey(new TopicPartition("backup.test-topic-1", 0)), CHECKPOINT_DURATION_MS, "Offsets not translated downstream to primary cluster."); -/* FIXME waitForCondition(() -> primaryClient.remoteConsumerOffsets(consumerGroupName, BACKUP_CLUSTER_ALIAS, - Duration.ofMillis(CHECKPOINT_DURATION_MS)).containsKey(new TopicPartition("test-topic-1", 0)), CHECKPOINT_DURATION_MS, - "Offsets not translated upstream to primary cluster."); -*/ + Duration.ofMillis(CHECKPOINT_DURATION_MS)).containsKey(new TopicPartition("test-topic-1", 0)), CHECKPOINT_DURATION_MS, "Offsets not translated upstream to primary cluster."); Map primaryOffsets = primaryClient.remoteConsumerOffsets(consumerGroupName, BACKUP_CLUSTER_ALIAS, Duration.ofMillis(CHECKPOINT_DURATION_MS)); @@ -345,21 +323,13 @@ public void testReplication() throws Exception { primaryOffsets.forEach(backupConsumer::seek); backupConsumer.poll(CONSUMER_POLL_TIMEOUT_MS); backupConsumer.commitAsync(); - -/* FIXME + assertTrue(backupConsumer.position(new TopicPartition("test-topic-1", 0)) > 0, "Consumer failedback to zero upstream offset."); -*/ - assertTrue(backupConsumer.position( - new TopicPartition(primaryReplicationPolicy.formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-1"), 0)) > 0, - "Consumer failedback to zero downstream offset."); - -/* + assertTrue(backupConsumer.position(new TopicPartition("backup.test-topic-1", 0)) > 0, "Consumer failedback to zero downstream offset."); assertTrue(backupConsumer.position( new TopicPartition("test-topic-1", 0)) <= NUM_RECORDS_PRODUCED, "Consumer failedback beyond expected upstream offset."); -*/ assertTrue(backupConsumer.position( - new TopicPartition(primaryReplicationPolicy.formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-1"), 0)) <= NUM_RECORDS_PRODUCED, - "Consumer failedback beyond expected downstream offset."); + new TopicPartition("backup.test-topic-1", 0)) <= NUM_RECORDS_PRODUCED, "Consumer failedback beyond expected downstream offset."); } // create more matching topics @@ -367,8 +337,8 @@ public void testReplication() throws Exception { backup.kafka().createTopic("test-topic-3", NUM_PARTITIONS); // make sure the topic is auto-created in the other cluster - waitForTopicCreated(backup, backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-2")); - waitForTopicCreated(primary, primaryReplicationPolicy.formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-3")); + waitForTopicCreated(backup, "primary.test-topic-2"); + waitForTopicCreated(primary, "backup.test-topic-3"); // only produce messages to the first partition produceMessages(primary, "test-topic-2", 1); @@ -380,11 +350,9 @@ public void testReplication() throws Exception { assertEquals(NUM_RECORDS_PER_PARTITION, backup.kafka().consume(NUM_RECORDS_PER_PARTITION, RECORD_TRANSFER_DURATION_MS, "test-topic-3").count(), "Records were not produced to backup cluster."); - assertEquals(NUM_RECORDS_PER_PARTITION, primary.kafka().consume(NUM_RECORDS_PER_PARTITION, 2 * RECORD_TRANSFER_DURATION_MS, - primaryReplicationPolicy.formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-3")).count(), + assertEquals(NUM_RECORDS_PER_PARTITION, primary.kafka().consume(NUM_RECORDS_PER_PARTITION, 2 * RECORD_TRANSFER_DURATION_MS, "backup.test-topic-3").count(), "New topic was not replicated to primary cluster."); - assertEquals(NUM_RECORDS_PER_PARTITION, backup.kafka().consume(NUM_RECORDS_PER_PARTITION, 2 * RECORD_TRANSFER_DURATION_MS, - backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-2")).count(), + assertEquals(NUM_RECORDS_PER_PARTITION, backup.kafka().consume(NUM_RECORDS_PER_PARTITION, 2 * RECORD_TRANSFER_DURATION_MS, "primary.test-topic-2").count(), "New topic was not replicated to backup cluster."); } @@ -414,9 +382,11 @@ public void testReplicationWithEmptyPartition() throws Exception { // sleep few seconds to have MM2 finish replication so that "end" consumer will consume some record Thread.sleep(TimeUnit.SECONDS.toMillis(3)); + String backupTopic = PRIMARY_CLUSTER_ALIAS + "." + topic; + // consume all records from backup cluster - try (Consumer backupConsumer = backup.kafka().createConsumerAndSubscribeTo(consumerProps, - backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, topic))) { + try (Consumer backupConsumer = backup.kafka().createConsumerAndSubscribeTo(consumerProps, + backupTopic)) { waitForConsumingAllRecords(backupConsumer, expectedRecords); } @@ -426,8 +396,7 @@ public void testReplicationWithEmptyPartition() throws Exception { backupClient.listConsumerGroupOffsets(consumerGroupName).partitionsToOffsetAndMetadata().get(); // pinpoint the offset of the last partition which does not receive records - OffsetAndMetadata offset = remoteOffsets.get(new TopicPartition( - backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, topic), NUM_PARTITIONS - 1)); + OffsetAndMetadata offset = remoteOffsets.get(new TopicPartition(backupTopic, NUM_PARTITIONS - 1)); // offset of the last partition should exist, but its value should be 0 assertNotNull(offset, "Offset of last partition was not replicated"); assertEquals(0, offset.offset(), "Offset of last partition is not zero"); @@ -460,14 +429,13 @@ public void testOneWayReplicationWithAutoOffsetSync() throws InterruptedExceptio waitUntilMirrorMakerIsRunning(backup, CONNECTOR_LIST, mm2Config, PRIMARY_CLUSTER_ALIAS, BACKUP_CLUSTER_ALIAS); // make sure the topic is created in the other cluster - waitForTopicCreated(primary, primaryReplicationPolicy.formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-1")); - waitForTopicCreated(backup, backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1")); + waitForTopicCreated(primary, "backup.test-topic-1"); + waitForTopicCreated(backup, "primary.test-topic-1"); // create a consumer at backup cluster with same consumer group Id to consume 1 topic Consumer backupConsumer = backup.kafka().createConsumerAndSubscribeTo( - consumerProps, backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1")); + consumerProps, "primary.test-topic-1"); - waitForConsumerGroupOffsetSync(backup, backupConsumer, Collections.singletonList( - backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1")), + waitForConsumerGroupOffsetSync(backup, backupConsumer, Collections.singletonList("primary.test-topic-1"), consumerGroupName, NUM_RECORDS_PRODUCED); ConsumerRecords records = backupConsumer.poll(CONSUMER_POLL_TIMEOUT_MS); @@ -480,7 +448,7 @@ public void testOneWayReplicationWithAutoOffsetSync() throws InterruptedExceptio // now create a new topic in primary cluster primary.kafka().createTopic("test-topic-2", NUM_PARTITIONS); // make sure the topic is created in backup cluster - waitForTopicCreated(backup, backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-2")); + waitForTopicCreated(backup, "primary.test-topic-2"); // produce some records to the new topic in primary cluster produceMessages(primary, "test-topic-2"); @@ -494,12 +462,9 @@ public void testOneWayReplicationWithAutoOffsetSync() throws InterruptedExceptio // 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), backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1"), - backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-2")); + "group.id", consumerGroupName), "primary.test-topic-1", "primary.test-topic-2"); - waitForConsumerGroupOffsetSync(backup, backupConsumer, Arrays.asList( - backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1"), - backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-2")), + waitForConsumerGroupOffsetSync(backup, backupConsumer, Arrays.asList("primary.test-topic-1", "primary.test-topic-2"), consumerGroupName, NUM_RECORDS_PRODUCED); records = backupConsumer.poll(CONSUMER_POLL_TIMEOUT_MS); @@ -511,7 +476,7 @@ public void testOneWayReplicationWithAutoOffsetSync() throws InterruptedExceptio /* * launch the connectors on kafka connect cluster and check if they are running */ - private static void waitUntilMirrorMakerIsRunning(EmbeddedConnectCluster connectCluster, + protected static void waitUntilMirrorMakerIsRunning(EmbeddedConnectCluster connectCluster, List> connectorClasses, MirrorMakerConfig mm2Config, String primary, String backup) throws InterruptedException { for (Class connector : connectorClasses) { @@ -531,7 +496,7 @@ private static void waitUntilMirrorMakerIsRunning(EmbeddedConnectCluster connect /* * wait for the topic created on the cluster */ - private static void waitForTopicCreated(EmbeddedConnectCluster cluster, String topicName) throws InterruptedException { + protected static void waitForTopicCreated(EmbeddedConnectCluster cluster, String topicName) throws InterruptedException { try (final Admin adminClient = cluster.kafka().createAdminClient()) { waitForCondition(() -> adminClient.listTopics().names().get().contains(topicName), TOPIC_SYNC_DURATION_MS, "Topic: " + topicName + " didn't get created on cluster: " + cluster.getName() @@ -545,7 +510,7 @@ private static void waitForTopicCreated(EmbeddedConnectCluster cluster, String t private static void deleteAllTopics(EmbeddedKafkaCluster cluster) throws Exception { try (final Admin adminClient = cluster.createAdminClient()) { Set topicsToBeDeleted = adminClient.listTopics().names().get(); - log.info("Deleting topics: {} ", topicsToBeDeleted); + log.debug("Deleting topics: {} ", topicsToBeDeleted); adminClient.deleteTopics(topicsToBeDeleted).all().get(); } } @@ -553,7 +518,7 @@ private static void deleteAllTopics(EmbeddedKafkaCluster cluster) throws Excepti /* * retrieve the config value based on the input cluster, topic and config name */ - private static String getTopicConfig(EmbeddedKafkaCluster cluster, String topic, String configName) throws Exception { + protected static String getTopicConfig(EmbeddedKafkaCluster cluster, String topic, String configName) throws Exception { try (Admin client = cluster.createAdminClient()) { Collection cr = Collections.singleton( new ConfigResource(ConfigResource.Type.TOPIC, topic)); @@ -588,7 +553,7 @@ protected void produceMessages(EmbeddedConnectCluster cluster, String topicName, * given consumer group, topics and expected number of records, make sure the consumer group * offsets are eventually synced to the expected offset numbers */ - private static void waitForConsumerGroupOffsetSync(EmbeddedConnectCluster connect, + protected static void waitForConsumerGroupOffsetSync(EmbeddedConnectCluster connect, Consumer consumer, List topics, String consumerGroupId, int numRecords) throws InterruptedException { try (Admin adminClient = connect.kafka().createAdminClient()) { @@ -618,7 +583,7 @@ private static void waitForConsumerGroupOffsetSync(EmbeddedConnectCluster co /* * make sure the consumer to consume expected number of records */ - private static void waitForConsumingAllRecords(Consumer consumer, int numExpectedRecords) + protected static void waitForConsumingAllRecords(Consumer consumer, int numExpectedRecords) throws InterruptedException { final AtomicInteger totalConsumedRecords = new AtomicInteger(0); waitForCondition(() -> { @@ -631,14 +596,11 @@ private static void waitForConsumingAllRecords(Consumer consumer, int /* * MM2 config to use in integration tests */ - private static Map basicMM2Config() { + protected static Map basicMM2Config() { Map mm2Props = new HashMap<>(); mm2Props.put("clusters", PRIMARY_CLUSTER_ALIAS + ", " + BACKUP_CLUSTER_ALIAS); mm2Props.put("max.tasks", "10"); - mm2Props.put("topics", "test-topic-.*, primary.test-topic-.*, backup.test-topic-.*"); mm2Props.put("groups", "consumer-group-.*"); - mm2Props.put(PRIMARY_CLUSTER_ALIAS + "->" + BACKUP_CLUSTER_ALIAS + ".enabled", "true"); - mm2Props.put(BACKUP_CLUSTER_ALIAS + "->" + PRIMARY_CLUSTER_ALIAS + ".enabled", "true"); mm2Props.put("sync.topic.acls.enabled", "false"); mm2Props.put("emit.checkpoints.interval.seconds", "1"); mm2Props.put("emit.heartbeats.interval.seconds", "1"); @@ -654,12 +616,7 @@ private static Map basicMM2Config() { return mm2Props; } - - // override in specialized tests - protected Map configOverrides() { - return new HashMap(); - } - + private void createTopics() { // to verify topic config will be sync-ed across clusters Map topicConfig = Collections.singletonMap(TopicConfig.CLEANUP_POLICY_CONFIG, TopicConfig.CLEANUP_POLICY_COMPACT); @@ -671,19 +628,17 @@ private void createTopics() { // create these topics before starting the connectors so we don't need to wait for discovery primary.kafka().createTopic("test-topic-1", NUM_PARTITIONS, 1, topicConfig, adminClientConfig); - primary.kafka().createTopic(primaryReplicationPolicy.formatRemoteTopic(BACKUP_CLUSTER_ALIAS, "test-topic-1"), - 1, 1, emptyMap, adminClientConfig); + primary.kafka().createTopic("backup.test-topic-1", 1, 1, emptyMap, adminClientConfig); primary.kafka().createTopic("heartbeats", 1, 1, emptyMap, adminClientConfig); backup.kafka().createTopic("test-topic-1", NUM_PARTITIONS, 1, emptyMap, adminClientConfig); - backup.kafka().createTopic(backupReplicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1"), - 1, 1, emptyMap, adminClientConfig); + backup.kafka().createTopic("primary.test-topic-1", 1, 1, emptyMap, adminClientConfig); backup.kafka().createTopic("heartbeats", 1, 1, emptyMap, adminClientConfig); } /* * Generate some consumer activity on both clusters to ensure the checkpoint connector always starts promptly */ - private void warmUpConsumer(Map consumerProps) throws InterruptedException { + protected void warmUpConsumer(Map consumerProps) throws InterruptedException { Consumer dummyConsumer = primary.kafka().createConsumerAndSubscribeTo(consumerProps, "test-topic-1"); dummyConsumer.poll(CONSUMER_POLL_TIMEOUT_MS); dummyConsumer.commitSync(); diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationLegacyTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationLegacyTest.java deleted file mode 100644 index 863e5408efbe6..0000000000000 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationLegacyTest.java +++ /dev/null @@ -1,35 +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.mirror.integration; - -import org.apache.kafka.connect.mirror.LegacyReplicationPolicy; - -import java.util.Map; -import java.util.HashMap; - -import org.junit.jupiter.api.Tag; - -@Tag("integration") -public class MirrorConnectorsIntegrationLegacyTest extends MirrorConnectorsIntegrationBaseTest { - @Override - protected Map configOverrides() { - Map m = new HashMap<>(); - m.put("replication.policy.class", LegacyReplicationPolicy.class.getName()); - m.put("replication.policy.remote.topic.suffix", ".replica"); - return m; - } -} From 7f621cfbc102c03e6cad76db1fc7a6ef7282c16e Mon Sep 17 00:00:00 2001 From: Ryanne Dolan Date: Fri, 21 May 2021 15:35:49 -0500 Subject: [PATCH 08/13] dropped LegacyReplicationPolicy --- .../mirror/IdentityReplicationPolicy.java | 13 +- .../mirror/LegacyReplicationPolicy.java | 118 ------------------ .../kafka/connect/mirror/MirrorClient.java | 2 +- .../connect/mirror/MirrorClientTest.java | 36 +++--- .../connect/mirror/MirrorSourceConnector.java | 2 +- ...> IdentityReplicationIntegrationTest.java} | 4 +- 6 files changed, 29 insertions(+), 146 deletions(-) delete mode 100644 connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/LegacyReplicationPolicy.java rename connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/{MirrorConnectorsIdentityReplicationPolicyIntegrationTest.java => IdentityReplicationIntegrationTest.java} (98%) diff --git a/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/IdentityReplicationPolicy.java b/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/IdentityReplicationPolicy.java index 465f193020c60..082bcba9d3cda 100644 --- a/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/IdentityReplicationPolicy.java +++ b/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/IdentityReplicationPolicy.java @@ -35,6 +35,15 @@ public class IdentityReplicationPolicy extends DefaultReplicationPolicy { private String sourceClusterAlias = null; + public IdentityReplicationPolicy() { + //nop + } + + // Visible for testing + IdentityReplicationPolicy(String sourceClusterAlias) { + this.sourceClusterAlias = sourceClusterAlias; + } + @Override public void configure(Map props) { super.configure(props); @@ -62,7 +71,7 @@ public String formatRemoteTopic(String sourceClusterAlias, String topic) { * a remote topic based on its name alone. If `source.cluster.alias` is provided, * `topicSource` will return that. * - * In the special case of heartbeast, we defer to DefaultReplicationPolicy. + * In the special case of heartbeats, we defer to DefaultReplicationPolicy. */ @Override public String topicSource(String topic) { @@ -75,7 +84,7 @@ public String topicSource(String topic) { /** Since any topic may be a "remote topic", this just returns `topic`. * - * In the special case of heartbeast, we defer to DefaultReplicationPolicy. + * In the special case of heartbeats, we defer to DefaultReplicationPolicy. */ @Override public String upstreamTopic(String topic) { diff --git a/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/LegacyReplicationPolicy.java b/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/LegacyReplicationPolicy.java deleted file mode 100644 index 75609fab9d659..0000000000000 --- a/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/LegacyReplicationPolicy.java +++ /dev/null @@ -1,118 +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.mirror; - -import org.apache.kafka.common.Configurable; - -import java.util.Map; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** LegacyReplicationPolicy attempts to mimic MirrorMaker v1, and thus by default does not - * rename topics. In order to support active/active replication with careful configuration, - * topics can be renamed with an optional suffix, e.g. "topic1.replica". - * - * LegacyReplicationPolicy cannot, in general, detect whether a topic is remote or not, nor - * which cluster a topic may have been replicated from. If a `remote.topic.suffix` is - * provided, we use that to distinguish remote topics. Otherwise we consider all topics as - * remote topics. If a `source.cluster.alias` is provided, we use that as the source of - * any remote topics. Otherwise we return null for `topicSource()`. - * - * Paradoxically, then, the default behavior of LegacyReplicationPolicy is to consider all - * topics to be remote but with no known source. - * - * N.B. MirrorMaker is not able to prevent cycles when using this class, so take care that - * your replication topology is acyclic. If migrating from MirrorMaker v1, this will likely - * already be the case. - */ -public class LegacyReplicationPolicy implements ReplicationPolicy, Configurable { - private static final Logger log = LoggerFactory.getLogger(LegacyReplicationPolicy.class); - - public static final String REMOTE_TOPIC_SUFFIX_CONFIG = "replication.policy.remote.topic.suffix"; - public static final String SOURCE_CLUSTER_ALIAS_CONFIG = "source.cluster.alias"; - - private String remoteTopicSuffix = ""; - private String sourceClusterAlias = null; - - public LegacyReplicationPolicy() { - } - - // Visible for testing - LegacyReplicationPolicy(String remoteTopicSuffix, String sourceClusterAlias) { - this.remoteTopicSuffix = remoteTopicSuffix; - this.sourceClusterAlias = sourceClusterAlias; - } - - @Override - public void configure(Map props) { - if (props.containsKey(REMOTE_TOPIC_SUFFIX_CONFIG)) { - remoteTopicSuffix = (String) props.get(REMOTE_TOPIC_SUFFIX_CONFIG); - log.info("Using custom remote topic suffix `{}`.", remoteTopicSuffix); - } - if (props.containsKey(SOURCE_CLUSTER_ALIAS_CONFIG)) { - sourceClusterAlias = (String) props.get(SOURCE_CLUSTER_ALIAS_CONFIG); - log.info("Using source cluster alias `{}`.", sourceClusterAlias); - } - } - - /** Unlike DefaultReplicationPolicy, LegacyReplicationPolicy does not include the source - * cluster alias in the remote topic name. By default, topic names are unchanged. If - * a `remote.topic.suffix` is provided, topic are renamed accordingly. - */ - @Override - public String formatRemoteTopic(String sourceClusterAlias, String topic) { - return topic + remoteTopicSuffix; - } - - /** Unlike DefaultReplicationPolicy, LegacyReplicationPolicy cannot know the source of - * a remote topic based on its name alone. If `source.cluster.alias` is provided, - * `topicSource` will return that for anything that looks like a remote topic. By default, - * all topics look like remote topics. - */ - @Override - public String topicSource(String topic) { - if (topic.endsWith(remoteTopicSuffix)) { - return sourceClusterAlias; - } else { - return null; - } - } - - /** Unlike DefaultReplicationPolicy, LegacyReplicationPolicy cannot distinguish remote - * topics from regular topics by default. If `remote.topic.suffix` is provided, this - * method will strip that and return the original topic. By default, `topic` is returned. - */ - @Override - public String upstreamTopic(String topic) { - if (topic.endsWith(remoteTopicSuffix)) { - return topic.substring(0, topic.length() - remoteTopicSuffix.length()); - } else { - return topic; - } - } - - /** If remote.topic.suffix is provided, treat suffixed topics as internal topics. - * This prevents replicated topics from being further replicated when using - * LegacyReplicationPolicy. - */ - @Override - public boolean isInternalTopic(String topic) { - return ReplicationPolicy.super.isInternalTopic(topic) - || (!remoteTopicSuffix.isEmpty() && topic.endsWith(remoteTopicSuffix)); - } -} diff --git a/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/MirrorClient.java b/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/MirrorClient.java index acc1796e0a22a..291f713679e45 100644 --- a/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/MirrorClient.java +++ b/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/MirrorClient.java @@ -203,7 +203,7 @@ int countHopsForTopic(String topic, String sourceClusterAlias) { return hops; } if (visited.contains(source)) { - // Extra check for LegacyReplicationPolicy (and any other dumb impls) that cannot prevent cycles. + // Extra check for IdentityReplicationPolicy and similar impls that cannot prevent cycles. // We assume we're stuck in a cycle and will never find sourceClusterAlias. return -1; } diff --git a/connect/mirror-client/src/test/java/org/apache/kafka/connect/mirror/MirrorClientTest.java b/connect/mirror-client/src/test/java/org/apache/kafka/connect/mirror/MirrorClientTest.java index fa0145fca13c4..c867a3e3cd654 100644 --- a/connect/mirror-client/src/test/java/org/apache/kafka/connect/mirror/MirrorClientTest.java +++ b/connect/mirror-client/src/test/java/org/apache/kafka/connect/mirror/MirrorClientTest.java @@ -136,16 +136,20 @@ public void upstreamClustersTest() throws InterruptedException { } @Test - public void legacyUpstreamClustersTest() throws InterruptedException { - // When configured with a specific source, LegacyReplicationPolicy should just return the provided source. - MirrorClient client = new FakeMirrorClient(new LegacyReplicationPolicy("", "source"), Arrays.asList("topic1", + public void testIdentityReplicationUpstreamClusters() throws InterruptedException { + // IdentityReplicationPolicy treats heartbeats as a special case, so these should work as usual. + MirrorClient client = new FakeMirrorClient(new IdentityReplicationPolicy("source"), Arrays.asList("topic1", "topic2", "heartbeats", "source1.heartbeats", "source1.source2.heartbeats", "source3.source4.source5.heartbeats")); Set sources = client.upstreamClusters(); - assertTrue(sources.contains("source")); + assertTrue(sources.contains("source1")); + assertTrue(sources.contains("source2")); + assertTrue(sources.contains("source3")); + assertTrue(sources.contains("source4")); + assertTrue(sources.contains("source5")); assertFalse(sources.contains("")); assertFalse(sources.contains(null)); - assertEquals(1, sources.size()); + assertEquals(5, sources.size()); } @Test @@ -161,11 +165,10 @@ public void remoteTopicsTest() throws InterruptedException { assertTrue(remoteTopics.contains("source3.source4.source5.topic6")); } - @Test - public void legacyRemoteTopicsTest() throws InterruptedException { - // By default, LegacyReplicationPolicy should consider any topic to be remote. - MirrorClient client = new FakeMirrorClient(new LegacyReplicationPolicy("", "source"), Arrays.asList( + public void testIdentityReplicationRemoteTopics() throws InterruptedException { + // IdentityReplicationPolicy should consider any topic to be remote. + MirrorClient client = new FakeMirrorClient(new IdentityReplicationPolicy("source"), Arrays.asList( "topic1", "topic2", "topic3")); Set remoteTopics = client.remoteTopics(); assertTrue(remoteTopics.contains("topic1")); @@ -188,20 +191,9 @@ public void remoteTopicsSeparatorTest() throws InterruptedException { assertTrue(remoteTopics.contains("source3__source4__source5__topic6")); } - @Test - public void testLegacyRemoteTopicWithSuffix() { - MirrorClient client = new FakeMirrorClient( - new LegacyReplicationPolicy(".replica", "primary"), Arrays.asList()); - assertEquals("topic1.replica", client.replicationPolicy() - .formatRemoteTopic("primary", "topic1")); - assertEquals("primary", client.replicationPolicy() - .topicSource("topic1.replica")); - } - - @Test - public void testLegacyRemoteTopicWithoutSuffix() { + public void testIdentityReplicationTopicSource() { MirrorClient client = new FakeMirrorClient( - new LegacyReplicationPolicy("", "primary"), Arrays.asList()); + new IdentityReplicationPolicy("primary"), Arrays.asList()); assertEquals("topic1", client.replicationPolicy() .formatRemoteTopic("primary", "topic1")); assertEquals("primary", client.replicationPolicy() diff --git a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceConnector.java b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceConnector.java index d5e7aa224df73..b45f17607b508 100644 --- a/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceConnector.java +++ b/connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceConnector.java @@ -494,7 +494,7 @@ boolean isCycle(String topic) { } else { String upstreamTopic = replicationPolicy.upstreamTopic(topic); if (upstreamTopic.equals(topic)) { - // Extra check for LegacyReplicationPolicy + // Extra check for IdentityReplicationPolicy and similar impls that don't prevent cycles. return false; } return isCycle(upstreamTopic); diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIdentityReplicationPolicyIntegrationTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/IdentityReplicationIntegrationTest.java similarity index 98% rename from connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIdentityReplicationPolicyIntegrationTest.java rename to connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/IdentityReplicationIntegrationTest.java index 43abad3f53022..7385ed9ce1015 100644 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIdentityReplicationPolicyIntegrationTest.java +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/IdentityReplicationIntegrationTest.java @@ -51,7 +51,7 @@ * are translated and replicated from the primary cluster to the backup cluster during this failover. */ @Tag("integration") -public class MirrorConnectorsIdentityReplicationPolicyIntegrationTest extends MirrorConnectorsIntegrationTest { +public class IdentityReplicationIntegrationTest extends MirrorConnectorsIntegrationTest { @BeforeEach public void startClusters() throws Exception { super.startClusters(new HashMap() {{ @@ -169,7 +169,7 @@ public void testReplicationWithEmptyPartition() throws Exception { // sleep few seconds to have MM2 finish replication so that "end" consumer will consume some record Thread.sleep(TimeUnit.SECONDS.toMillis(3)); - // note that with LegacyReplicationPolicy, topics on the backup are NOT renamed to PRIMARY_CLUSTER_ALIAS + "." + topic + // note that with IdentityReplicationPolicy, topics on the backup are NOT renamed to PRIMARY_CLUSTER_ALIAS + "." + topic String backupTopic = topic; // consume all records from backup cluster From a9cf4e41fb91c36325354154c826506ee9f8afa9 Mon Sep 17 00:00:00 2001 From: Ryanne Dolan Date: Wed, 16 Jun 2021 15:26:02 -0500 Subject: [PATCH 09/13] drop unnecessary change to mirror-client api --- .../main/java/org/apache/kafka/connect/mirror/MirrorClient.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/MirrorClient.java b/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/MirrorClient.java index 291f713679e45..aa0bca1f9194d 100644 --- a/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/MirrorClient.java +++ b/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/MirrorClient.java @@ -60,7 +60,7 @@ public class MirrorClient implements AutoCloseable { private ReplicationPolicy replicationPolicy; private Map consumerConfig; - public MirrorClient(Map props) { + public MirrorClient(Map props) { this(new MirrorClientConfig(props)); } From bf8548138182a94bfe57261a9a174bb49d75b6ab Mon Sep 17 00:00:00 2001 From: Ryanne Dolan Date: Wed, 16 Jun 2021 15:45:20 -0500 Subject: [PATCH 10/13] improve tests for IdentityReplicationPolicy h/t @mimaison --- .../connect/mirror/MirrorClientTest.java | 11 +++++++++- .../mirror/MirrorSourceConnectorTest.java | 22 +++++++++++++++++++ 2 files changed, 32 insertions(+), 1 deletion(-) diff --git a/connect/mirror-client/src/test/java/org/apache/kafka/connect/mirror/MirrorClientTest.java b/connect/mirror-client/src/test/java/org/apache/kafka/connect/mirror/MirrorClientTest.java index c867a3e3cd654..5807a8a57a5ca 100644 --- a/connect/mirror-client/src/test/java/org/apache/kafka/connect/mirror/MirrorClientTest.java +++ b/connect/mirror-client/src/test/java/org/apache/kafka/connect/mirror/MirrorClientTest.java @@ -169,11 +169,14 @@ public void remoteTopicsTest() throws InterruptedException { public void testIdentityReplicationRemoteTopics() throws InterruptedException { // IdentityReplicationPolicy should consider any topic to be remote. MirrorClient client = new FakeMirrorClient(new IdentityReplicationPolicy("source"), Arrays.asList( - "topic1", "topic2", "topic3")); + "topic1", "topic2", "topic3", "heartbeats", "backup.heartbeats")); Set remoteTopics = client.remoteTopics(); assertTrue(remoteTopics.contains("topic1")); assertTrue(remoteTopics.contains("topic2")); assertTrue(remoteTopics.contains("topic3")); + // Heartbeats are treated as a special case + assertFalse(remoteTopics.contains("heartbeats")); + assertTrue(remoteTopics.contains("backup.heartbeats")); } @Test @@ -191,6 +194,7 @@ public void remoteTopicsSeparatorTest() throws InterruptedException { assertTrue(remoteTopics.contains("source3__source4__source5__topic6")); } + @Test public void testIdentityReplicationTopicSource() { MirrorClient client = new FakeMirrorClient( new IdentityReplicationPolicy("primary"), Arrays.asList()); @@ -198,5 +202,10 @@ public void testIdentityReplicationTopicSource() { .formatRemoteTopic("primary", "topic1")); assertEquals("primary", client.replicationPolicy() .topicSource("topic1")); + // Heartbeats are handled as a special case + assertEquals("backup.heartbeats", client.replicationPolicy() + .formatRemoteTopic("backup", "heartbeats")); + assertEquals("backup", client.replicationPolicy() + .topicSource("backup.heartbeats")); } } diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorSourceConnectorTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorSourceConnectorTest.java index 68d149c755ff8..b4c8ca651d29a 100644 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorSourceConnectorTest.java +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorSourceConnectorTest.java @@ -77,10 +77,32 @@ public void testNoCycles() { assertFalse(connector.shouldReplicateTopic("target.topic1"), "should not allow cycles"); assertFalse(connector.shouldReplicateTopic("target.source.topic1"), "should not allow cycles"); assertFalse(connector.shouldReplicateTopic("source.target.topic1"), "should not allow cycles"); + assertFalse(connector.shouldReplicateTopic("target.source.target.topic1"), "should not allow cycles"); + assertFalse(connector.shouldReplicateTopic("source.target.source.topic1"), "should not allow cycles"); assertTrue(connector.shouldReplicateTopic("topic1"), "should allow anything else"); assertTrue(connector.shouldReplicateTopic("source.topic1"), "should allow anything else"); } + @Test + public void testIdentityReplication() { + MirrorSourceConnector connector = new MirrorSourceConnector(new SourceAndTarget("source", "target"), + new IdentityReplicationPolicy(), x -> true, x -> true); + assertTrue(connector.shouldReplicateTopic("target.topic1"), "should allow cycles"); + assertTrue(connector.shouldReplicateTopic("target.source.topic1"), "should allow cycles"); + assertTrue(connector.shouldReplicateTopic("source.target.topic1"), "should allow cycles"); + assertTrue(connector.shouldReplicateTopic("target.source.target.topic1"), "should allow cycles"); + assertTrue(connector.shouldReplicateTopic("source.target.source.topic1"), "should allow cycles"); + assertTrue(connector.shouldReplicateTopic("topic1"), "should allow normal topics"); + assertTrue(connector.shouldReplicateTopic("othersource.topic1"), "should allow normal topics"); + assertFalse(connector.shouldReplicateTopic("target.heartbeats"), "should not allow heartbeat cycles"); + assertFalse(connector.shouldReplicateTopic("target.source.heartbeats"), "should not allow heartbeat cycles"); + assertFalse(connector.shouldReplicateTopic("source.target.heartbeats"), "should not allow heartbeat cycles"); + assertFalse(connector.shouldReplicateTopic("target.source.target.heartbeats"), "should not allow heartbeat cycles"); + assertFalse(connector.shouldReplicateTopic("source.target.source.heartbeats"), "should not allow heartbeat cycles"); + assertTrue(connector.shouldReplicateTopic("heartbeats"), "should allow heartbeat topics"); + assertTrue(connector.shouldReplicateTopic("othersource.heartbeats"), "should allow heartbeat topics"); + } + @Test public void testAclFiltering() { MirrorSourceConnector connector = new MirrorSourceConnector(new SourceAndTarget("source", "target"), From 627e0992e35e308db33d5cc6edafc4fed989df06 Mon Sep 17 00:00:00 2001 From: Ryanne Dolan Date: Mon, 21 Jun 2021 16:16:51 -0500 Subject: [PATCH 11/13] drop unnecessary constructor from IdentityReplicationPolicy --- .../connect/mirror/IdentityReplicationPolicy.java | 9 --------- .../kafka/connect/mirror/MirrorClientTest.java | 13 ++++++++++--- 2 files changed, 10 insertions(+), 12 deletions(-) diff --git a/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/IdentityReplicationPolicy.java b/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/IdentityReplicationPolicy.java index 082bcba9d3cda..7320b18020112 100644 --- a/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/IdentityReplicationPolicy.java +++ b/connect/mirror-client/src/main/java/org/apache/kafka/connect/mirror/IdentityReplicationPolicy.java @@ -35,15 +35,6 @@ public class IdentityReplicationPolicy extends DefaultReplicationPolicy { private String sourceClusterAlias = null; - public IdentityReplicationPolicy() { - //nop - } - - // Visible for testing - IdentityReplicationPolicy(String sourceClusterAlias) { - this.sourceClusterAlias = sourceClusterAlias; - } - @Override public void configure(Map props) { super.configure(props); diff --git a/connect/mirror-client/src/test/java/org/apache/kafka/connect/mirror/MirrorClientTest.java b/connect/mirror-client/src/test/java/org/apache/kafka/connect/mirror/MirrorClientTest.java index 5807a8a57a5ca..2e1b9b771942e 100644 --- a/connect/mirror-client/src/test/java/org/apache/kafka/connect/mirror/MirrorClientTest.java +++ b/connect/mirror-client/src/test/java/org/apache/kafka/connect/mirror/MirrorClientTest.java @@ -138,7 +138,7 @@ public void upstreamClustersTest() throws InterruptedException { @Test public void testIdentityReplicationUpstreamClusters() throws InterruptedException { // IdentityReplicationPolicy treats heartbeats as a special case, so these should work as usual. - MirrorClient client = new FakeMirrorClient(new IdentityReplicationPolicy("source"), Arrays.asList("topic1", + MirrorClient client = new FakeMirrorClient(identityReplicationPolicy("source"), Arrays.asList("topic1", "topic2", "heartbeats", "source1.heartbeats", "source1.source2.heartbeats", "source3.source4.source5.heartbeats")); Set sources = client.upstreamClusters(); @@ -168,7 +168,7 @@ public void remoteTopicsTest() throws InterruptedException { @Test public void testIdentityReplicationRemoteTopics() throws InterruptedException { // IdentityReplicationPolicy should consider any topic to be remote. - MirrorClient client = new FakeMirrorClient(new IdentityReplicationPolicy("source"), Arrays.asList( + MirrorClient client = new FakeMirrorClient(identityReplicationPolicy("source"), Arrays.asList( "topic1", "topic2", "topic3", "heartbeats", "backup.heartbeats")); Set remoteTopics = client.remoteTopics(); assertTrue(remoteTopics.contains("topic1")); @@ -197,7 +197,7 @@ public void remoteTopicsSeparatorTest() throws InterruptedException { @Test public void testIdentityReplicationTopicSource() { MirrorClient client = new FakeMirrorClient( - new IdentityReplicationPolicy("primary"), Arrays.asList()); + identityReplicationPolicy("primary"), Arrays.asList()); assertEquals("topic1", client.replicationPolicy() .formatRemoteTopic("primary", "topic1")); assertEquals("primary", client.replicationPolicy() @@ -208,4 +208,11 @@ public void testIdentityReplicationTopicSource() { assertEquals("backup", client.replicationPolicy() .topicSource("backup.heartbeats")); } + + private ReplicationPolicy identityReplicationPolicy(String source) { + IdentityReplicationPolicy policy = new IdentityReplicationPolicy(); + policy.configure(Collections.singletonMap( + IdentityReplicationPolicy.SOURCE_CLUSTER_ALIAS_CONFIG, source)); + return policy; + } } From ada302c98e86492f90288b4053d49fe68e392c4f Mon Sep 17 00:00:00 2001 From: Ryanne Dolan Date: Tue, 29 Jun 2021 16:46:11 -0500 Subject: [PATCH 12/13] mention IdentityReplicationPolicy in upgrade notes --- docs/upgrade.html | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/docs/upgrade.html b/docs/upgrade.html index 671170695121b..005fd1d08cead 100644 --- a/docs/upgrade.html +++ b/docs/upgrade.html @@ -80,7 +80,13 @@

Notable changes in 3 understood by brokers or version 2.5 or higher, so you must upgrade your kafka cluster to get the stronger semantics. Otherwise, you can just pass in new ConsumerGroupMetadata(consumerGroupId) to work with older brokers. See KIP-732 for more details. - +
  • The Connect-based MirrorMaker (MM2) includes changes to support IdentityReplicationPolicy, enabling replication without renaming topics. + The existing DefaultReplicationPolicy is still used by default, but identity replication can be enabled via the + replication.policy configuration property. This is especially useful for users migrating from the older MirrorMaker (MM1), or for + use-cases with simple one-way replication topologies where topic renaming is undesirable. Note that IdentityReplicationPolicy, unlike + DefaultReplicationPolicy, cannot prevent replication cycles based on topic names, so take care to avoid cycles when constructing your + replication topology. +
  • Notable changes in 2.8.0
    From f9edd324c79ece1d405e1eaccef34ef16fa557d0 Mon Sep 17 00:00:00 2001 From: Ryanne Dolan Date: Wed, 30 Jun 2021 15:29:26 -0500 Subject: [PATCH 13/13] fix testOffsetSyncsTopicsOnTarget integration test to work with identity replication --- .../integration/IdentityReplicationIntegrationTest.java | 2 +- .../integration/MirrorConnectorsIntegrationBaseTest.java | 8 ++++++-- 2 files changed, 7 insertions(+), 3 deletions(-) diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/IdentityReplicationIntegrationTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/IdentityReplicationIntegrationTest.java index 7385ed9ce1015..43b1fcbf6d1d3 100644 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/IdentityReplicationIntegrationTest.java +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/IdentityReplicationIntegrationTest.java @@ -51,7 +51,7 @@ * are translated and replicated from the primary cluster to the backup cluster during this failover. */ @Tag("integration") -public class IdentityReplicationIntegrationTest extends MirrorConnectorsIntegrationTest { +public class IdentityReplicationIntegrationTest extends MirrorConnectorsIntegrationBaseTest { @BeforeEach public void startClusters() throws Exception { super.startClusters(new HashMap() {{ diff --git a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationBaseTest.java b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationBaseTest.java index 7ce24fcc84656..6fb7a81676bc2 100644 --- a/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationBaseTest.java +++ b/connect/mirror/src/test/java/org/apache/kafka/connect/mirror/integration/MirrorConnectorsIntegrationBaseTest.java @@ -36,6 +36,7 @@ import org.apache.kafka.connect.mirror.SourceAndTarget; import org.apache.kafka.connect.mirror.Checkpoint; import org.apache.kafka.connect.mirror.MirrorCheckpointConnector; +import org.apache.kafka.connect.mirror.ReplicationPolicy; import org.apache.kafka.connect.util.clusters.EmbeddedConnectCluster; import org.apache.kafka.connect.util.clusters.EmbeddedKafkaCluster; import org.apache.kafka.connect.util.clusters.UngracefulShutdownException; @@ -493,6 +494,9 @@ public void testOffsetSyncsTopicsOnTarget() throws Exception { produceMessages(primary, "test-topic-1"); + ReplicationPolicy replicationPolicy = new MirrorClient(mm2Config.clientConfig(BACKUP_CLUSTER_ALIAS)).replicationPolicy(); + String remoteTopic = replicationPolicy.formatRemoteTopic(PRIMARY_CLUSTER_ALIAS, "test-topic-1"); + // Check offsets are pushed to the checkpoint topic Consumer backupConsumer = backup.kafka().createConsumerAndSubscribeTo(Collections.singletonMap( "auto.offset.reset", "earliest"), PRIMARY_CLUSTER_ALIAS + ".checkpoints.internal"); @@ -500,13 +504,13 @@ public void testOffsetSyncsTopicsOnTarget() throws Exception { ConsumerRecords records = backupConsumer.poll(Duration.ofSeconds(1L)); for (ConsumerRecord record : records) { Checkpoint checkpoint = Checkpoint.deserializeRecord(record); - if ((PRIMARY_CLUSTER_ALIAS + ".test-topic-1").equals(checkpoint.topicPartition().topic())) { + if (remoteTopic.equals(checkpoint.topicPartition().topic())) { return true; } } return false; }, 30_000, - "Unable to find checkpoints for " + PRIMARY_CLUSTER_ALIAS + "test-topic-1" + "Unable to find checkpoints for " + PRIMARY_CLUSTER_ALIAS + ".test-topic-1" ); // Ensure no offset-syncs topics have been created on the primary cluster