You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by GitBox <gi...@apache.org> on 2022/11/23 11:52:47 UTC

[GitHub] [kafka] mimaison opened a new pull request, #12899: KAFKA-14413: Separate MirrorMaker configurations for each connector

mimaison opened a new pull request, #12899:
URL: https://github.com/apache/kafka/pull/12899

   This also adds 3 new Gradle tasks (`:connect:mirror:genMirrorSourceConfigDocs`, `:connect:mirror:genMirrorCheckpointConfigDocs` and `:connect:mirror:genMirrorHeartbeatConfigDocs`) to generate the documentation for the configuration of each connector.
   
   I plan to do updates to the documentation in a follow up PR.
   
   ### Committer Checklist (excluded from commit message)
   - [ ] Verify design and implementation 
   - [ ] Verify test coverage and CI build status
   - [ ] Verify documentation (including upgrade notes)
   


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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] showuon commented on a diff in pull request #12899: KAFKA-14413: Separate MirrorMaker configurations for each connector

Posted by GitBox <gi...@apache.org>.
showuon commented on code in PR #12899:
URL: https://github.com/apache/kafka/pull/12899#discussion_r1033079800


##########
connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorConnectorConfigTest.java:
##########
@@ -17,145 +17,34 @@
 package org.apache.kafka.connect.mirror;
 
 import org.apache.kafka.clients.CommonClientConfigs;
-import org.apache.kafka.common.TopicPartition;
-import org.apache.kafka.common.config.ConfigDef;
 import org.apache.kafka.common.config.ConfigException;
 import org.apache.kafka.common.metrics.JmxReporter;
 import org.apache.kafka.test.MockMetricsReporter;
 import org.junit.jupiter.api.Test;
 
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.List;
 import java.util.Map;
 import java.util.HashMap;
-import java.util.HashSet;
 
 import static org.apache.kafka.connect.mirror.TestUtils.makeProps;
 import static org.junit.jupiter.api.Assertions.assertTrue;
-import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.junit.jupiter.api.Assertions.assertThrows;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 
 public class MirrorConnectorConfigTest {
 
-    @Test
-    public void testTaskConfigTopicPartitions() {
-        List<TopicPartition> topicPartitions = Arrays.asList(new TopicPartition("topic-1", 2),
-            new TopicPartition("topic-3", 4), new TopicPartition("topic-5", 6));
-        MirrorConnectorConfig config = new MirrorConnectorConfig(makeProps());
-        Map<String, String> props = config.taskConfigForTopicPartitions(topicPartitions);
-        MirrorTaskConfig taskConfig = new MirrorTaskConfig(props);
-        assertEquals(taskConfig.taskTopicPartitions(), new HashSet<>(topicPartitions),
-                "Setting topic property configuration failed");
-    }
+    static class TestMirrorConnectorConfig extends MirrorConnectorConfig {
 
-    @Test
-    public void testTaskConfigConsumerGroups() {

Review Comment:
   Is this test deleted or moved to other file? I can't find it anywhere else.



##########
connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorConnectorConfigTest.java:
##########
@@ -17,145 +17,34 @@
 package org.apache.kafka.connect.mirror;
 
 import org.apache.kafka.clients.CommonClientConfigs;
-import org.apache.kafka.common.TopicPartition;
-import org.apache.kafka.common.config.ConfigDef;
 import org.apache.kafka.common.config.ConfigException;
 import org.apache.kafka.common.metrics.JmxReporter;
 import org.apache.kafka.test.MockMetricsReporter;
 import org.junit.jupiter.api.Test;
 
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.List;
 import java.util.Map;
 import java.util.HashMap;
-import java.util.HashSet;
 
 import static org.apache.kafka.connect.mirror.TestUtils.makeProps;
 import static org.junit.jupiter.api.Assertions.assertTrue;
-import static org.junit.jupiter.api.Assertions.assertFalse;
 import static org.junit.jupiter.api.Assertions.assertThrows;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 
 public class MirrorConnectorConfigTest {
 
-    @Test
-    public void testTaskConfigTopicPartitions() {
-        List<TopicPartition> topicPartitions = Arrays.asList(new TopicPartition("topic-1", 2),
-            new TopicPartition("topic-3", 4), new TopicPartition("topic-5", 6));
-        MirrorConnectorConfig config = new MirrorConnectorConfig(makeProps());
-        Map<String, String> props = config.taskConfigForTopicPartitions(topicPartitions);
-        MirrorTaskConfig taskConfig = new MirrorTaskConfig(props);
-        assertEquals(taskConfig.taskTopicPartitions(), new HashSet<>(topicPartitions),
-                "Setting topic property configuration failed");
-    }
+    static class TestMirrorConnectorConfig extends MirrorConnectorConfig {
 
-    @Test
-    public void testTaskConfigConsumerGroups() {
-        List<String> groups = Arrays.asList("consumer-1", "consumer-2", "consumer-3");
-        MirrorConnectorConfig config = new MirrorConnectorConfig(makeProps());
-        Map<String, String> props = config.taskConfigForConsumerGroups(groups);
-        MirrorTaskConfig taskConfig = new MirrorTaskConfig(props);
-        assertEquals(taskConfig.taskConsumerGroups(), new HashSet<>(groups),
-                "Setting consumer groups property configuration failed");
-    }
-
-    @Test
-    public void testTopicMatching() {
-        MirrorConnectorConfig config = new MirrorConnectorConfig(makeProps("topics", "topic1"));
-        assertTrue(config.topicFilter().shouldReplicateTopic("topic1"),
-                "topic1 replication property configuration failed");
-        assertFalse(config.topicFilter().shouldReplicateTopic("topic2"),
-                "topic2 replication property configuration failed");
-    }
-
-    @Test
-    public void testGroupMatching() {

Review Comment:
   Is this test deleted or moved to other file? I can't find it anywhere else.



##########
connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorConnectorConfigTest.java:
##########
@@ -260,100 +149,37 @@ public void testTargetAdminConfig() {
     public void testTargetAdminConfigWithSourcePrefix() {
         String prefix = MirrorConnectorConfig.TARGET_PREFIX + MirrorConnectorConfig.ADMIN_CLIENT_PREFIX;
         Map<String, String> connectorProps = makeProps(prefix + "connections.max.idle.ms", "10000");
-        MirrorConnectorConfig config = new MirrorConnectorConfig(connectorProps);
+        MirrorConnectorConfig config = new TestMirrorConnectorConfig(connectorProps);
         Map<String, Object> connectorAdminProps = config.targetAdminConfig();
         Map<String, Object> expectedAdminProps = new HashMap<>();
         expectedAdminProps.put("connections.max.idle.ms", "10000");
         assertEquals(expectedAdminProps, connectorAdminProps, prefix + " source connector admin props not matching");
     }
 
-    @Test
-    public void testOffsetSyncsTopic() {
-        // Invalid location
-        Map<String, String> connectorProps = makeProps("offset-syncs.topic.location", "something");
-        assertThrows(ConfigException.class, () -> new MirrorConnectorConfig(connectorProps));
-
-        connectorProps.put("offset-syncs.topic.location", "source");
-        MirrorConnectorConfig config = new MirrorConnectorConfig(connectorProps);
-        assertEquals("mm2-offset-syncs.target2.internal", config.offsetSyncsTopic());
-        connectorProps.put("offset-syncs.topic.location", "target");
-        config = new MirrorConnectorConfig(connectorProps);
-        assertEquals("mm2-offset-syncs.source1.internal", config.offsetSyncsTopic());
-        // Default to source
-        connectorProps.remove("offset-syncs.topic.location");
-        config = new MirrorConnectorConfig(connectorProps);
-        assertEquals("mm2-offset-syncs.target2.internal", config.offsetSyncsTopic());
-    }
-
-    @Test
-    public void testConsumerConfigsForOffsetSyncsTopic() {

Review Comment:
   ditto



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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] mimaison commented on pull request #12899: KAFKA-14413: Separate MirrorMaker configurations for each connector

Posted by GitBox <gi...@apache.org>.
mimaison commented on PR #12899:
URL: https://github.com/apache/kafka/pull/12899#issuecomment-1332205910

   Thanks @C0urante for the review. I've pushed an update.


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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] C0urante commented on a diff in pull request #12899: KAFKA-14413: Separate MirrorMaker configurations for each connector

Posted by GitBox <gi...@apache.org>.
C0urante commented on code in PR #12899:
URL: https://github.com/apache/kafka/pull/12899#discussion_r1034980195


##########
connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointConnector.java:
##########
@@ -45,7 +45,7 @@ public class MirrorCheckpointConnector extends SourceConnector {
     private static final Logger log = LoggerFactory.getLogger(MirrorCheckpointConnector.class);
 
     private Scheduler scheduler;
-    private MirrorConnectorConfig config;
+    private MirrorCheckpointConfig config;

Review Comment:
   Should we update the class Javadoc as well?
   ```java
   /** Replicate consumer group state between clusters. Emits checkpoint records.
    *
    *  @see MirrorCheckpointConfig for supported config properties.
    */
   
   ```



##########
connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointConfig.java:
##########
@@ -0,0 +1,255 @@
+/*
+ * 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.config.ConfigDef;
+import org.apache.kafka.common.utils.ConfigUtils;
+
+import java.time.Duration;
+import java.util.List;
+import java.util.Map;
+
+public class MirrorCheckpointConfig extends MirrorConnectorConfig {
+
+    public static final String TOPIC_FILTER_CLASS = "topic.filter.class";
+    private static final String TOPIC_FILTER_CLASS_DOC = "TopicFilter to use. Selects topics to replicate.";
+    public static final Class<?> TOPIC_FILTER_CLASS_DEFAULT = DefaultTopicFilter.class;

Review Comment:
   These and the `offset-syncs.topic.location`-related properties are defined both here and in the `MirrorSourceConfig` class. To reduce duplication, do you think it might make sense to put the constants for these properties (i.e., the property names and their docstrings) in the `MirrorConnectorConfig` class, along with the getter methods like `topicFilter` and `offsetSyncsTopicLocation`? We could leave it up to subclasses to actually add those properties to their respective `ConfigDef` objects so that they don't show up in the docs for connectors that don't use them, but it would help reduce the likelihood for typos and divergence in property names and docstrings.



##########
connect/mirror/src/test/java/org/apache/kafka/connect/mirror/MirrorSourceConnectorTest.java:
##########
@@ -333,4 +337,136 @@ public String upstreamTopic(String topic) {
                 new CustomReplicationPolicy(), new DefaultTopicFilter(), new DefaultConfigPropertyFilter());
         assertDoesNotThrow(() -> connector.isCycle(".b"));
     }
+
+    @Test
+    public void testTaskConfigTopicPartitions() {

Review Comment:
   Seems a little strange to move these tests to the `MirrorSourceConnectorTest` suite since they don't actually test against instances of a `MirrorSourceConnector` (and this might explain why @showuon had trouble finding where these tests were moved to). Did you consider creating new `MirrorSourceConfigTest` and `MirrorCheckpointConfigTest` classes?



##########
connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceConnector.java:
##########
@@ -71,7 +71,7 @@ public class MirrorSourceConnector extends SourceConnector {
     private static final AclBindingFilter ANY_TOPIC_ACL = new AclBindingFilter(ANY_TOPIC, AccessControlEntryFilter.ANY);
 
     private Scheduler scheduler;
-    private MirrorConnectorConfig config;
+    private MirrorSourceConfig config;

Review Comment:
   Same thought RE class Javadocs:
   ```java
    *  @see MirrorSourceConfig for supported config properties.
   ```



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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] mimaison commented on pull request #12899: KAFKA-14413: Separate MirrorMaker configurations for each connector

Posted by GitBox <gi...@apache.org>.
mimaison commented on PR #12899:
URL: https://github.com/apache/kafka/pull/12899#issuecomment-1328885642

   @showuon Thanks for the review and good catch! I double checked and all tests should be there now.


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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [kafka] mimaison merged pull request #12899: KAFKA-14413: Separate MirrorMaker configurations for each connector

Posted by GitBox <gi...@apache.org>.
mimaison merged PR #12899:
URL: https://github.com/apache/kafka/pull/12899


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

To unsubscribe, e-mail: jira-unsubscribe@kafka.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org