C0urante commented on code in PR #11779:
URL: https://github.com/apache/kafka/pull/11779#discussion_r891380187
##########
connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaConfigBackingStore.java:
##########
@@ -291,7 +328,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 "
+ + "for source connectors, or use a new Kafka
broker version.",
Review Comment:
Haha yep, caught and fixed this in an upstream PR that's since been merged.
Will pick up in the rebase.
https://github.com/apache/kafka/blob/a6c5a74fdbdce9a992b47706913c920902cda28c/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaConfigBackingStore.java#L323
##########
connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaConfigBackingStore.java:
##########
@@ -309,16 +355,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:
Yep, this got fixed in https://github.com/apache/kafka/pull/11778, which
just got merged. A rebase should take care of this.
--
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]