C0urante commented on code in PR #11778: URL: https://github.com/apache/kafka/pull/11778#discussion_r889731729
########## 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: Ack, done ########## 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: Good point, changed it to set to 5. Considered an alternative where we don't explicitly set anything and let the default (5) take effect unless the user overrides it in their worker config file, but we'd have to lower it to 5 if it were higher than that in order to prevent reordering, and there just doesn't seem to be enough value in permitting that kind of flexibility to do the extra work to add the necessary safeguards around it. ########## 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: 😎 -- 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