Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

feat: add support for transaction #155

Closed
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
@@ -0,0 +1,26 @@
/*
* Copyright 2017-2020 original authors
*
* Licensed 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
*
* https://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 io.micronaut.configuration.kafka.annotation;


/**
* Parameter level annotation to indicate when send to is all or None.
*
* @author Michael Pollind
* @since 1.2
*/
public @interface KafkaTransaction {
}
Original file line number Diff line number Diff line change
Expand Up @@ -56,7 +56,11 @@
import org.apache.kafka.clients.producer.Producer;
import org.apache.kafka.clients.producer.ProducerRecord;
import org.apache.kafka.clients.producer.RecordMetadata;
import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.errors.AuthorizationException;
import org.apache.kafka.common.errors.OutOfOrderSequenceException;
import org.apache.kafka.common.errors.ProducerFencedException;
import org.apache.kafka.common.errors.WakeupException;
import org.apache.kafka.common.serialization.ByteArrayDeserializer;
import org.apache.kafka.common.serialization.StringDeserializer;
Expand All @@ -73,6 +77,7 @@
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentSkipListSet;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.regex.Pattern;

Expand Down Expand Up @@ -616,9 +621,11 @@ private void handleResultFlowable(
ConsumerRecord<?, ?> consumerRecord,
Flowable<?> resultFlowable,
boolean isBlocking) {
AtomicBoolean firstError = new AtomicBoolean(false);
Flowable<RecordMetadata> recordMetadataProducer = resultFlowable.subscribeOn(executorScheduler)
.flatMap((Function<Object, Publisher<RecordMetadata>>) o -> {
String[] destinationTopics = method.stringValues(SendTo.class);
boolean hasTransaction = method.hasAnnotation(KafkaTransaction.class);
if (ArrayUtils.isNotEmpty(destinationTopics)) {
Object key = consumerRecord.key();
Object value = o;
Expand All @@ -630,6 +637,50 @@ private void handleResultFlowable(
Argument.of((Class) (key != null ? key.getClass() : byte[].class)),
Argument.of(value.getClass())
);
//docs: https://kafka.apache.org/0110/javadoc/index.html?org/apache/kafka/clients/producer/KafkaProducer.html
if(hasTransaction) {
return Flowable.create(emitter -> {
try {
kafkaProducer.beginTransaction();
for (String destinationTopic : destinationTopics) {
ProducerRecord record = new ProducerRecord(
destinationTopic,
null,
key,
value,
consumerRecord.headers()
);
kafkaProducer.send(record, (metadata, exception) -> {
if (exception != null) {
emitter.onError(exception);
} else {
emitter.onNext(metadata);
}
});

}
kafkaProducer.commitTransaction();
} catch (ProducerFencedException | OutOfOrderSequenceException | AuthorizationException e) {
handleException(consumerBean, new KafkaListenerException(
"Unhandled exception [" + consumerRecord + "] with Kafka reactive consumer [" + method + "]: " + e.getMessage(),
e,
consumerBean,
kafkaConsumer,
consumerRecord
));
} catch (KafkaException e) {
handleException(consumerBean, new KafkaListenerException(
"Error transaction aborted [" + consumerRecord + "] with Kafka reactive consumer [" + method + "]: " + e.getMessage(),
e,
consumerBean,
kafkaConsumer,
consumerRecord
));
kafkaProducer.abortTransaction();
}
emitter.onComplete();
}, BackpressureStrategy.ERROR);
}

return Flowable.create(emitter -> {
for (String destinationTopic : destinationTopics) {
Expand Down Expand Up @@ -667,7 +718,7 @@ private void handleResultFlowable(
consumerRecord
));

if (kafkaListener.isTrue("redelivery")) {
if (kafkaListener.isTrue("redelivery") && !firstError.getAndSet(true)) {
if (LOG.isDebugEnabled()) {
LOG.debug("Attempting redelivery of record [{}] following error", consumerRecord);
}
Expand Down