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


##########
connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceTask.java:
##########
@@ -51,58 +48,52 @@
 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);
+            offsetSyncWriter.clearPendingOffsetSyncs();
+        }
+        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) {
+            offsetSyncWriter = new OffsetSyncWriter(config);
+            offsetSyncWriter.clearPendingOffsetSyncs();

Review Comment:
   Is `offsetSyncWriter.clearPendingOffsetSyncs` necessary if 
`offsetSyncWriter` is a new object?



##########
connect/mirror/src/main/java/org/apache/kafka/connect/mirror/OffsetSyncWriter.java:
##########
@@ -0,0 +1,133 @@
+/*
+ * 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);
+    private static final int MAX_OUTSTANDING_OFFSET_SYNCS = 10;
+
+    private final Map<TopicPartition, OffsetSync> delayedOffsetSyncs = new 
LinkedHashMap<>();
+    private final Map<TopicPartition, OffsetSync> pendingOffsetSyncs = new 
LinkedHashMap<>();
+    private Semaphore outstandingOffsetSyncs;

Review Comment:
   those variables can be `final`



##########
connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorConnectorConfig.java:
##########
@@ -109,9 +109,17 @@ public abstract class MirrorConnectorConfig extends 
AbstractConfig {
     public static final String TOPIC_FILTER_CLASS_DOC = "TopicFilter to use. 
Selects topics to replicate.";
     public static final Class<?> TOPIC_FILTER_CLASS_DEFAULT = 
DefaultTopicFilter.class;
 
-    public static final String OFFSET_SYNCS_TOPIC_LOCATION = 
"offset-syncs.topic.location";
+    public static final String OFFSET_SYNCS_TOPIC_CONFIG_PREFIX = 
"offset-syncs.topic.";
+    public static final String OFFSET_SYNCS_TOPIC_LOCATION = 
OFFSET_SYNCS_TOPIC_CONFIG_PREFIX + "location";
     public static final String OFFSET_SYNCS_TOPIC_LOCATION_DEFAULT = 
SOURCE_CLUSTER_ALIAS_DEFAULT;
     public static final String OFFSET_SYNCS_TOPIC_LOCATION_DOC = "The location 
(source/target) of the offset-syncs topic.";
+
+    public static final String EMIT_OFFSET_SYNCS_ENABLED = "emit.offset-syncs" 
+ MirrorCheckpointConfig.ENABLED_SUFFIX;

Review Comment:
   `MirrorCheckpointConfig` is subclass of `MirrorConnectorConfig`, so this 
reference is a bit dangerous to me. Maybe we can use constants instead?



##########
connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceTask.java:
##########
@@ -112,12 +103,15 @@ public void start(Map<String, String> props) {
 
     @Override
     public void commit() {
-        // Offset syncs which were not emitted immediately due to their offset 
spacing should be sent periodically
-        // This ensures that low-volume topics aren't left with persistent lag 
at the end of the topic
-        promoteDelayedOffsetSyncs();
-        // Publish any offset syncs that we've queued up, but have not yet 
been able to publish
-        // (likely because we previously reached our limit for number of 
outstanding syncs)
-        firePendingOffsetSyncs();
+        // Handle delayed and pending offset syncs only when 
emit.offset-syncs.enabled set to true
+        if (emitOffsetSyncEnabled) {

Review Comment:
   Maybe using null check is more suitable? `if (offsetSyncWriter != null)`



##########
connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceTask.java:
##########
@@ -51,58 +48,52 @@
 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);
+            offsetSyncWriter.clearPendingOffsetSyncs();

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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to