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/06/05 13:42:06 UTC

[GitHub] [kafka] showuon commented on a diff in pull request #11778: KAFKA-10000: Use transactional producer for config topic (KIP-618)

showuon commented on code in PR #11778:
URL: https://github.com/apache/kafka/pull/11778#discussion_r889690020


##########
connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaConfigBackingStore.java:
##########
@@ -291,7 +315,16 @@ public void start() {
         log.info("Starting KafkaConfigBackingStore");
         // Before startup, callbacks are *not* invoked. You can grab a snapshot after starting -- just take care that
         // updates can continue to occur in the background
-        configLog.start();
+        try {
+            configLog.start();
+        } catch (UnsupportedVersionException e) {
+            throw new ConnectException(
+                    "Enabling exactly-once support for source connectors requires a Kafka broker version that allows "
+                            + "admin clients to read consumer offsets. Disable the worker's exactly-once support "

Review Comment:
   nit: [Please] disable the worker's exactly-once support....



##########
connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaConfigBackingStore.java:
##########
@@ -606,7 +742,7 @@ public void onCompletion(Throwable error, ConsumerRecord<String, byte[]> record)
                         }
                         Object targetState = ((Map<String, Object>) value.value()).get("state");
                         if (!(targetState instanceof String)) {
-                            log.error("Invalid data for target state for connector '{}': 'state' field should be a Map but is {}",
+                            log.error("Invalid data for target state for connector '{}': 'state' field should be a String but is {}",

Review Comment:
   Nice catch!



##########
connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaConfigBackingStore.java:
##########
@@ -309,16 +342,70 @@ public void start() {
     @Override
     public void stop() {
         log.info("Closing KafkaConfigBackingStore");
-        try {
-            configLog.stop();
-        } finally {
-            if (ownTopicAdmin != null) {
-                ownTopicAdmin.close();
-            }
+
+        if (fencableProducer != null) {
+            Utils.closeQuietly(() -> fencableProducer.close(Duration.ZERO), "fencable producer for config topic");
         }
+        Utils.closeQuietly(ownTopicAdmin, "admin for config topic");
+        Utils.closeQuietly(configLog::stop, "KafkaBasedLog for config topic");
+
         log.info("Closed KafkaConfigBackingStore");
     }
 
+    @Override
+    public void claimWritePrivileges() {
+        if (usesFencableWriter && fencableProducer == null) {
+            try {
+                fencableProducer = createFencableProducer();
+                fencableProducer.initTransactions();
+            } catch (Exception e) {
+                if (fencableProducer != null) {
+                    Utils.closeQuietly(() -> fencableProducer.close(Duration.ZERO), "fencable producer for config topic");
+                    fencableProducer = null;
+                }
+                throw new ConnectException("Failed to create and initialize fencable producer for config topic", e);
+            }
+        }
+    }
+
+    private Map<String, Object> baseProducerProps(WorkerConfig workerConfig) {
+        Map<String, Object> producerProps = new HashMap<>(workerConfig.originals());
+        String kafkaClusterId = ConnectUtils.lookupKafkaClusterId(workerConfig);
+        producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName());
+        producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class.getName());
+        producerProps.put(ProducerConfig.DELIVERY_TIMEOUT_MS_CONFIG, Integer.MAX_VALUE);
+        ConnectUtils.addMetricsContextProperties(producerProps, workerConfig, kafkaClusterId);
+        return producerProps;
+    }
+
+    // Visible for testing
+    Map<String, Object> fencableProducerProps(DistributedConfig workerConfig) {
+        Map<String, Object> result = new HashMap<>(baseProducerProps(workerConfig));
+
+        // Always require producer acks to all to ensure durable writes
+        result.put(ProducerConfig.ACKS_CONFIG, "all");
+        // Don't allow more than one in-flight request to prevent reordering on retry (if enabled)
+        result.put(ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION, 1);

Review Comment:
   Is this expected even when we enable idempotence? I thought the idempotent producer can make sure the order when max in-flight <= 5? Could we set to 5 here?



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