showuon commented on code in PR #11778:
URL: https://github.com/apache/kafka/pull/11778#discussion_r889851112
##########
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:
I agree we put default to 5 for `fencableProducer` for now. And improve it
for customization to <= 5 value if necessary in the future.
##########
connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaConfigBackingStore.java:
##########
@@ -309,16 +342,71 @@ 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
Review Comment:
This line should be removed.
##########
connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaConfigBackingStore.java:
##########
@@ -547,6 +663,27 @@ KafkaBasedLog<String, byte[]>
setupAndCreateKafkaBasedLog(String topic, final Wo
return createKafkaBasedLog(topic, producerProps, consumerProps, new
ConsumeCallback(), topicDescription, adminSupplier);
}
+ private void maybeSendFencably(String key, byte[] value) {
+ if (!usesFencableWriter) {
+ configLog.send(key, value);
+ return;
+ }
+
+ if (fencableProducer == null) {
+ throw new IllegalStateException("Cannot produce to config topic
without claiming write privileges first");
+ }
+
+ try {
+ fencableProducer.beginTransaction();
+ fencableProducer.send(new ProducerRecord<>(topic, key, value));
+ fencableProducer.commitTransaction();
+ } catch (ProducerFencedException e) {
+ log.warn("Failed to write to config topic as producer was fenced
out", e);
+ fencableProducer = null;
Review Comment:
Should we close the `fencableProducer` when `ProducerFencedException`
thrown? Looks like we only recreate one afterwards, right? And what about other
exceptions? Should we also close `fencableProducer` in those cases?
--
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]