mimaison commented on code in PR #15999:
URL: https://github.com/apache/kafka/pull/15999#discussion_r1620924489


##########
connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointConfig.java:
##########
@@ -252,7 +252,13 @@ private static ConfigDef defineCheckpointConfig(ConfigDef 
baseConfig) {
                         ConfigDef.Type.CLASS,
                         TOPIC_FILTER_CLASS_DEFAULT,
                         ConfigDef.Importance.LOW,
-                        TOPIC_FILTER_CLASS_DOC);
+                        TOPIC_FILTER_CLASS_DOC)
+                .define(

Review Comment:
   This connector is not emitting offset-syncs so I'm not sure it makes sense 
to define this configuration on this connector.
   
   I see you use this for validation, but in standalone and distributed modes 
the value for this configuration on this connector is not necessarily the same 
as the value on MirrorSourceConnector. So if you start MirrorSourceConnector 
with `emit.offset-syncs.enabled=false` and then this connector with the value 
set to `true` it would still pass validation.
   
   When running in dedicated mode the configurations are shared, but maybe we 
can do the validation in the MirrorMakerConfig class.



##########
connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceTask.java:
##########
@@ -51,58 +48,50 @@
 public class MirrorSourceTask extends SourceTask {
 
     private static final Logger log = 
LoggerFactory.getLogger(MirrorSourceTask.class);
-
-    private static final int MAX_OUTSTANDING_OFFSET_SYNCS = 10;
-
     private KafkaConsumer<byte[], byte[]> consumer;
-    private KafkaProducer<byte[], byte[]> offsetProducer;
     private String sourceClusterAlias;
-    private String offsetSyncsTopic;
     private Duration pollTimeout;
-    private long maxOffsetLag;
     private Map<TopicPartition, PartitionState> partitionStates;
     private ReplicationPolicy replicationPolicy;
     private MirrorSourceMetrics metrics;
     private boolean stopping = false;
-    private final Map<TopicPartition, OffsetSync> delayedOffsetSyncs = new 
LinkedHashMap<>();
-    private final Map<TopicPartition, OffsetSync> pendingOffsetSyncs = new 
LinkedHashMap<>();
-    private Semaphore outstandingOffsetSyncs;
     private Semaphore consumerAccess;
+    private OffsetSyncWriter offsetSyncWriter;
+    private boolean emitOffsetSyncEnabled;

Review Comment:
   We don't really seem to use this field. It's only used where it's computed. 
Either we keep it and use it instead of checking if offsetSyncWriter is null, 
or we can delete it.



##########
connect/mirror/src/main/java/org/apache/kafka/connect/mirror/OffsetSyncWriter.java:
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.utils.Utils;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.concurrent.Semaphore;
+
+class OffsetSyncWriter implements AutoCloseable {
+    private static final Logger log = 
LoggerFactory.getLogger(OffsetSyncWriter.class);

Review Comment:
   `log` -> `LOG`



##########
connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceTask.java:
##########
@@ -51,58 +48,50 @@
 public class MirrorSourceTask extends SourceTask {
 
     private static final Logger log = 
LoggerFactory.getLogger(MirrorSourceTask.class);
-
-    private static final int MAX_OUTSTANDING_OFFSET_SYNCS = 10;
-
     private KafkaConsumer<byte[], byte[]> consumer;
-    private KafkaProducer<byte[], byte[]> offsetProducer;
     private String sourceClusterAlias;
-    private String offsetSyncsTopic;
     private Duration pollTimeout;
-    private long maxOffsetLag;
     private Map<TopicPartition, PartitionState> partitionStates;
     private ReplicationPolicy replicationPolicy;
     private MirrorSourceMetrics metrics;
     private boolean stopping = false;
-    private final Map<TopicPartition, OffsetSync> delayedOffsetSyncs = new 
LinkedHashMap<>();
-    private final Map<TopicPartition, OffsetSync> pendingOffsetSyncs = new 
LinkedHashMap<>();
-    private Semaphore outstandingOffsetSyncs;
     private Semaphore consumerAccess;
+    private OffsetSyncWriter offsetSyncWriter;
+    private boolean emitOffsetSyncEnabled;
 
     public MirrorSourceTask() {}
 
     // for testing
     MirrorSourceTask(KafkaConsumer<byte[], byte[]> consumer, 
MirrorSourceMetrics metrics, String sourceClusterAlias,
                      ReplicationPolicy replicationPolicy, long maxOffsetLag, 
KafkaProducer<byte[], byte[]> producer,
                      Semaphore outstandingOffsetSyncs, Map<TopicPartition, 
PartitionState> partitionStates,
-                     String offsetSyncsTopic) {
+                     String offsetSyncsTopic, boolean emitOffsetSyncEnabled) {
         this.consumer = consumer;
         this.metrics = metrics;
         this.sourceClusterAlias = sourceClusterAlias;
         this.replicationPolicy = replicationPolicy;
-        this.maxOffsetLag = maxOffsetLag;
         consumerAccess = new Semaphore(1);
-        this.offsetProducer = producer;
-        this.outstandingOffsetSyncs = outstandingOffsetSyncs;
+        if (emitOffsetSyncEnabled) {
+            this.offsetSyncWriter = new OffsetSyncWriter(producer, 
offsetSyncsTopic, outstandingOffsetSyncs, maxOffsetLag);
+        }
+        this.emitOffsetSyncEnabled = emitOffsetSyncEnabled;
         this.partitionStates = partitionStates;
-        this.offsetSyncsTopic = offsetSyncsTopic;
     }
 
     @Override
     public void start(Map<String, String> props) {
         MirrorSourceTaskConfig config = new MirrorSourceTaskConfig(props);
-        pendingOffsetSyncs.clear();
-        outstandingOffsetSyncs = new Semaphore(MAX_OUTSTANDING_OFFSET_SYNCS);
+        emitOffsetSyncEnabled = config.emitOffsetSyncEnabled();
         consumerAccess = new Semaphore(1);  // let one thread at a time access 
the consumer
         sourceClusterAlias = config.sourceClusterAlias();
         metrics = config.metrics();
         pollTimeout = config.consumerPollTimeout();
-        maxOffsetLag = config.maxOffsetLag();
         replicationPolicy = config.replicationPolicy();
         partitionStates = new HashMap<>();
-        offsetSyncsTopic = config.offsetSyncsTopic();
+        if (this.emitOffsetSyncEnabled) {

Review Comment:
   We can drop `this` here



##########
connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceConnector.java:
##########
@@ -162,7 +164,9 @@ public void start(Map<String, String> props) {
         sourceAdminClient = 
config.forwardingAdmin(config.sourceAdminConfig("replication-source-admin"));
         targetAdminClient = 
config.forwardingAdmin(config.targetAdminConfig("replication-target-admin"));
         useIncrementalAlterConfigs =  
!config.useIncrementalAlterConfigs().equals(MirrorSourceConfig.NEVER_USE_INCREMENTAL_ALTER_CONFIGS);
-        offsetSyncsAdminClient = 
config.forwardingAdmin(config.offsetSyncsTopicAdminConfig());
+        if (config.emitOffsetSyncEnabled()) {

Review Comment:
   This client is only used to create the topic in `createOffsetSyncsTopic()`. 
I wonder if we can only create it `createOffsetSyncsTopic()`, create the topic 
and directly close it instead of having it as a field. WDYT?



##########
connect/mirror/src/main/java/org/apache/kafka/connect/mirror/OffsetSyncWriter.java:
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.utils.Utils;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import java.util.concurrent.Semaphore;
+
+class OffsetSyncWriter implements AutoCloseable {

Review Comment:
   Can we add javadoc? Especially as the name is now confusing with 
`OffsetSyncStore`



-- 
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-unsubscr...@kafka.apache.org

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

Reply via email to