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


##########
connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorCheckpointConnector.java:
##########
@@ -105,6 +108,49 @@ public void stop() {
         Utils.closeQuietly(targetAdminClient, "target admin client");
     }
 
+    @Override
+    public Config validate(Map<String, String> props) {
+        List<ConfigValue> configValues = super.validate(props).configValues();
+        String emitCheckpointsValue = 
Optional.ofNullable(props.get(MirrorCheckpointConfig.EMIT_CHECKPOINTS_ENABLED)).orElse(Boolean.toString(MirrorCheckpointConfig.EMIT_CHECKPOINTS_ENABLED_DEFAULT));
+        String syncGroupOffsetsValue = 
Optional.ofNullable(props.get(MirrorCheckpointConfig.SYNC_GROUP_OFFSETS_ENABLED)).orElse(Boolean.toString(MirrorCheckpointConfig.SYNC_GROUP_OFFSETS_ENABLED_DEFAULT));
+        String emitOffsetSyncsValue = 
Optional.ofNullable(props.get(MirrorConnectorConfig.EMIT_OFFSET_SYNCS_ENABLED)).orElse(Boolean.toString(MirrorConnectorConfig.EMIT_OFFSET_SYNCS_ENABLED_DEFAULT));
+
+        if ("false".equals(emitCheckpointsValue) && 
"false".equals(syncGroupOffsetsValue)) {
+            ConfigValue syncGroupOffsets = configValues.stream().filter(prop 
-> MirrorCheckpointConfig.SYNC_GROUP_OFFSETS_ENABLED.equals(prop.name()))
+                    .findAny()
+                    .orElseGet(() -> {
+                        ConfigValue result = new 
ConfigValue(MirrorCheckpointConfig.SYNC_GROUP_OFFSETS_ENABLED);
+                        configValues.add(result);
+                        return result;
+                    });
+
+            ConfigValue emitCheckpoints = configValues.stream().filter(prop -> 
MirrorCheckpointConfig.EMIT_CHECKPOINTS_ENABLED.equals(prop.name()))
+                    .findAny()
+                    .orElseGet(() -> {
+                        ConfigValue result = new 
ConfigValue(MirrorCheckpointConfig.EMIT_CHECKPOINTS_ENABLED);
+                        configValues.add(result);
+                        return result;
+                    });
+
+            String errorMessage = "MirrorCheckpointConnector can't run with 
both" +
+                    MirrorCheckpointConfig.SYNC_GROUP_OFFSETS_ENABLED + ", " + 
MirrorCheckpointConfig.EMIT_CHECKPOINTS_ENABLED + "set to false";
+            syncGroupOffsets.addErrorMessage(errorMessage);
+            emitCheckpoints.addErrorMessage(errorMessage);
+        }
+        if ("false".equals(emitOffsetSyncsValue) && 
("true".equals(emitCheckpointsValue) || "true".equals(syncGroupOffsetsValue))) {
+            ConfigValue emitOffsetSyncs = configValues.stream().filter(prop -> 
MirrorConnectorConfig.EMIT_OFFSET_SYNCS_ENABLED.equals(prop.name()))
+                    .findAny()
+                    .orElseGet(() -> {
+                        ConfigValue result = new 
ConfigValue(MirrorConnectorConfig.EMIT_OFFSET_SYNCS_ENABLED);
+                        configValues.add(result);
+                        return result;
+                    });
+
+            emitOffsetSyncs.addErrorMessage("MirrorCheckpointConnector can't 
run while MirrorSourceConnector configured with" +
+                    MirrorConnectorConfig.EMIT_OFFSET_SYNCS_ENABLED + "set to 
false");
+        }
+        return new Config(configValues);

Review Comment:
   Can these validations live in the config class? In this case that would be 
`MirrorConnectConfig`. 
   
   Besides being in line with config validation it has the added benefit that 
we can use `.getBoolean(...)` instead of ` 
props.get(..prop.)).orElse(Boolean.toString(...default...)`.



##########
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);
+    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 final Semaphore outstandingOffsetSyncs;
+    private final KafkaProducer<byte[], byte[]> offsetProducer;
+    private final String offsetSyncsTopic;
+    protected final long maxOffsetLag;
+
+    public OffsetSyncWriter(MirrorSourceTaskConfig config) {
+        outstandingOffsetSyncs = new Semaphore(MAX_OUTSTANDING_OFFSET_SYNCS);
+        offsetSyncsTopic = config.offsetSyncsTopic();
+        offsetProducer = 
MirrorUtils.newProducer(config.offsetSyncsTopicProducerConfig());
+        maxOffsetLag = config.maxOffsetLag();
+    }
+
+    public OffsetSyncWriter(KafkaProducer<byte[], byte[]> producer,
+                            String offsetSyncsTopic,
+                            Semaphore outstandingOffsetSyncs,
+                            long maxOffsetLag) {
+        this.offsetProducer = producer;
+        this.offsetSyncsTopic = offsetSyncsTopic;
+        this.outstandingOffsetSyncs = outstandingOffsetSyncs;
+        this.maxOffsetLag = maxOffsetLag;
+    }
+
+    public void close() {
+        Utils.closeQuietly(offsetProducer, "offset producer");
+    }
+
+    // sends OffsetSync record to internal offsets topic
+    private void sendOffsetSync(OffsetSync offsetSync) {
+        ProducerRecord<byte[], byte[]> record = new 
ProducerRecord<>(offsetSyncsTopic, 0,
+                offsetSync.recordKey(), offsetSync.recordValue());
+        offsetProducer.send(record, (x, e) -> {
+            if (e != null) {
+                log.error("Failure sending offset sync.", e);
+            } else {
+                log.trace("Sync'd offsets for {}: {}=={}", 
offsetSync.topicPartition(),
+                        offsetSync.upstreamOffset(), 
offsetSync.downstreamOffset());
+            }
+            outstandingOffsetSyncs.release();
+        });
+    }
+
+    protected void firePendingOffsetSyncs() {
+        while (true) {
+            OffsetSync pendingOffsetSync;
+            synchronized (this) {
+                Iterator<OffsetSync> syncIterator = 
pendingOffsetSyncs.values().iterator();
+                if (!syncIterator.hasNext()) {
+                    // Nothing to sync
+                    log.trace("No more pending offset syncs");
+                    return;
+                }
+                pendingOffsetSync = syncIterator.next();
+                if (!outstandingOffsetSyncs.tryAcquire()) {
+                    // Too many outstanding syncs
+                    log.trace("Too many in-flight offset syncs; will try to 
send remaining offset syncs later");
+                    return;
+                }
+                syncIterator.remove();
+            }
+            // Publish offset sync outside of synchronized block; we may have 
to
+            // wait for producer metadata to update before Producer::send 
returns
+            sendOffsetSync(pendingOffsetSync);
+            log.trace("Dispatched offset sync for {}", 
pendingOffsetSync.topicPartition());
+        }
+    }
+
+    protected synchronized void promoteDelayedOffsetSyncs() {
+        pendingOffsetSyncs.putAll(delayedOffsetSyncs);
+        delayedOffsetSyncs.clear();
+    }
+
+    // updates partition state and queues up OffsetSync if necessary
+    protected void maybeQueueOffsetSyncs(MirrorSourceTask.PartitionState 
partitionState,

Review Comment:
   Why are `firePendingOffsetSyncs`, `promoteDelayedOffsetSyncs` and 
`maybeQueueOffsetSyncs` `protected`? Should these be `public`?



##########
connect/mirror/src/main/java/org/apache/kafka/connect/mirror/MirrorSourceConnector.java:
##########
@@ -257,6 +261,22 @@ public org.apache.kafka.common.config.Config 
validate(Map<String, String> props)
                 );
             }
         }
+        boolean offsetSyncsConfigured = configValues.stream()
+                .anyMatch(config -> 
config.name().startsWith(OFFSET_SYNCS_CLIENT_ROLE_PREFIX) || 
config.name().startsWith(OFFSET_SYNCS_TOPIC_CONFIG_PREFIX));
+        String emitOffsetSyncsValue = 
Optional.ofNullable(props.get(MirrorConnectorConfig.EMIT_OFFSET_SYNCS_ENABLED)).orElse(Boolean.toString(MirrorSourceConfig.EMIT_OFFSET_SYNCS_ENABLED_DEFAULT));
+
+        if ("false".equals(emitOffsetSyncsValue) && offsetSyncsConfigured) {
+            ConfigValue emitOffsetSyncs = configValues.stream().filter(prop -> 
MirrorConnectorConfig.EMIT_OFFSET_SYNCS_ENABLED.equals(prop.name()))
+                    .findAny()
+                    .orElseGet(() -> {
+                        ConfigValue result = new 
ConfigValue(MirrorConnectorConfig.EMIT_OFFSET_SYNCS_ENABLED);
+                        configValues.add(result);
+                        return result;
+                    });
+            emitOffsetSyncs.addErrorMessage("MirrorSourceConnector can't setup 
offset-syncs feature while" +
+                    MirrorConnectorConfig.EMIT_OFFSET_SYNCS_ENABLED + "set to 
false");
+        }

Review Comment:
   Same here, can this validation be performed after the config values are 
parsed?



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