KAFKA-14752: Kafka examples improvements - producer changes (#13515)

KAFKA-14752: Kafka examples improvements - producer changes

Reviewers: Luke Chen <showuon@gmail.com>, Christo Lolov <christololov@gmail.com>
This commit is contained in:
Federico Valeri 2023-05-08 04:15:52 +02:00 committed by GitHub
parent 6e7144ac24
commit 78090bb4cd
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
4 changed files with 133 additions and 97 deletions

View File

@ -66,7 +66,9 @@ public class ExactlyOnceMessageProcessor extends Thread {
// It is recommended to have a relatively short txn timeout in order to clear pending offsets faster. // It is recommended to have a relatively short txn timeout in order to clear pending offsets faster.
final int transactionTimeoutMs = 10000; final int transactionTimeoutMs = 10000;
// A unique transactional.id must be provided in order to properly use EOS. // A unique transactional.id must be provided in order to properly use EOS.
producer = new Producer(outputTopic, true, transactionalId, true, -1, transactionTimeoutMs, null).get(); producer = new Producer(
"processor-producer", KafkaProperties.KAFKA_SERVER_URL + ":" + KafkaProperties.KAFKA_SERVER_PORT, outputTopic, true, transactionalId, true, -1, transactionTimeoutMs, null)
.createKafkaProducer();
// Consumer must be in read_committed mode, which means it won't be able to read uncommitted data. // Consumer must be in read_committed mode, which means it won't be able to read uncommitted data.
// Consumer could optionally configure groupInstanceId to avoid unnecessary rebalances. // Consumer could optionally configure groupInstanceId to avoid unnecessary rebalances.
this.groupInstanceId = "Txn-consumer-" + instanceIdx; this.groupInstanceId = "Txn-consumer-" + instanceIdx;

View File

@ -26,7 +26,8 @@ public class KafkaConsumerProducerDemo {
public static void main(String[] args) throws InterruptedException { public static void main(String[] args) throws InterruptedException {
boolean isAsync = args.length == 0 || !args[0].trim().equalsIgnoreCase("sync"); boolean isAsync = args.length == 0 || !args[0].trim().equalsIgnoreCase("sync");
CountDownLatch latch = new CountDownLatch(2); CountDownLatch latch = new CountDownLatch(2);
Producer producerThread = new Producer(KafkaProperties.TOPIC, isAsync, null, false, 10000, -1, latch); Producer producerThread = new Producer(
"producer", KafkaProperties.KAFKA_SERVER_URL + ":" + KafkaProperties.KAFKA_SERVER_PORT, KafkaProperties.TOPIC, isAsync, null, false, 10000, -1, latch);
producerThread.start(); producerThread.start();
Consumer consumerThread = new Consumer( Consumer consumerThread = new Consumer(

View File

@ -91,7 +91,8 @@ public class KafkaExactlyOnceDemo {
CountDownLatch prePopulateLatch = new CountDownLatch(1); CountDownLatch prePopulateLatch = new CountDownLatch(1);
/* Stage 2: pre-populate records */ /* Stage 2: pre-populate records */
Producer producerThread = new Producer(INPUT_TOPIC, false, null, true, numRecords, -1, prePopulateLatch); Producer producerThread = new Producer(
"producer", KafkaProperties.KAFKA_SERVER_URL + ":" + KafkaProperties.KAFKA_SERVER_PORT, INPUT_TOPIC, false, null, true, numRecords, -1, prePopulateLatch);
producerThread.start(); producerThread.start();
if (!prePopulateLatch.await(5, TimeUnit.MINUTES)) { if (!prePopulateLatch.await(5, TimeUnit.MINUTES)) {

View File

@ -21,133 +21,165 @@ import org.apache.kafka.clients.producer.KafkaProducer;
import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.clients.producer.ProducerConfig;
import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.clients.producer.ProducerRecord;
import org.apache.kafka.clients.producer.RecordMetadata; import org.apache.kafka.clients.producer.RecordMetadata;
import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.errors.AuthorizationException;
import org.apache.kafka.common.errors.FencedInstanceIdException;
import org.apache.kafka.common.errors.OutOfOrderSequenceException;
import org.apache.kafka.common.errors.ProducerFencedException;
import org.apache.kafka.common.errors.RetriableException;
import org.apache.kafka.common.errors.SerializationException;
import org.apache.kafka.common.errors.UnsupportedVersionException;
import org.apache.kafka.common.serialization.IntegerSerializer; import org.apache.kafka.common.serialization.IntegerSerializer;
import org.apache.kafka.common.serialization.StringSerializer; import org.apache.kafka.common.serialization.StringSerializer;
import java.util.Properties; import java.util.Properties;
import java.util.UUID;
import java.util.concurrent.CountDownLatch; import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutionException;
import java.util.concurrent.Future;
/** /**
* Demo producer that demonstrate two modes of KafkaProducer. * A simple producer thread supporting two send modes:
* If the user uses the Async mode: The messages will be printed to stdout upon successful completion * - Async mode (default): records are sent without waiting for the response.
* If the user uses the sync mode (isAsync = false): Each send loop will block until completion. * - Sync mode: each send operation blocks waiting for the response.
*/ */
public class Producer extends Thread { public class Producer extends Thread {
private final KafkaProducer<Integer, String> producer; private final String bootstrapServers;
private final String topic; private final String topic;
private final Boolean isAsync; private final boolean isAsync;
private int numRecords; private final String transactionalId;
private final boolean enableIdempotency;
private final int numRecords;
private final int transactionTimeoutMs;
private final CountDownLatch latch; private final CountDownLatch latch;
private volatile boolean closed;
public Producer(final String topic, public Producer(String threadName,
final Boolean isAsync, String bootstrapServers,
final String transactionalId, String topic,
final boolean enableIdempotency, boolean isAsync,
final int numRecords, String transactionalId,
final int transactionTimeoutMs, boolean enableIdempotency,
final CountDownLatch latch) { int numRecords,
Properties props = new Properties(); int transactionTimeoutMs,
props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, KafkaProperties.KAFKA_SERVER_URL + ":" + KafkaProperties.KAFKA_SERVER_PORT); CountDownLatch latch) {
props.put(ProducerConfig.CLIENT_ID_CONFIG, "DemoProducer"); super(threadName);
props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, IntegerSerializer.class.getName()); this.bootstrapServers = bootstrapServers;
props.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName());
if (transactionTimeoutMs > 0) {
props.put(ProducerConfig.TRANSACTION_TIMEOUT_CONFIG, transactionTimeoutMs);
}
if (transactionalId != null) {
props.put(ProducerConfig.TRANSACTIONAL_ID_CONFIG, transactionalId);
}
props.put(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, enableIdempotency);
producer = new KafkaProducer<>(props);
this.topic = topic; this.topic = topic;
this.isAsync = isAsync; this.isAsync = isAsync;
this.transactionalId = transactionalId;
this.enableIdempotency = enableIdempotency;
this.numRecords = numRecords; this.numRecords = numRecords;
this.transactionTimeoutMs = transactionTimeoutMs;
this.latch = latch; this.latch = latch;
} }
KafkaProducer<Integer, String> get() {
return producer;
}
@Override @Override
public void run() { public void run() {
int messageKey = 0; int key = 0;
int recordsSent = 0; int sentRecords = 0;
try { // the producer instance is thread safe
while (recordsSent < numRecords) { try (KafkaProducer<Integer, String> producer = createKafkaProducer()) {
final long currentTimeMs = System.currentTimeMillis(); while (!closed && sentRecords < numRecords) {
produceOnce(messageKey, recordsSent, currentTimeMs); if (isAsync) {
messageKey += 2; asyncSend(producer, key, "test" + key);
recordsSent += 1; } else {
syncSend(producer, key, "test" + key);
}
key++;
sentRecords++;
} }
} catch (Exception e) { } catch (Throwable e) {
System.out.println("Producer encountered exception:" + e); Utils.printOut("Unhandled exception");
} finally { e.printStackTrace();
System.out.println("Producer sent " + numRecords + " records successfully"); }
this.producer.close(); Utils.printOut("Sent %d records", sentRecords);
shutdown();
}
public void shutdown() {
if (!closed) {
closed = true;
latch.countDown(); latch.countDown();
} }
} }
private void produceOnce(final int messageKey, final int recordsSent, final long currentTimeMs) throws ExecutionException, InterruptedException { public KafkaProducer<Integer, String> createKafkaProducer() {
String messageStr = "Message_" + messageKey; Properties props = new Properties();
// bootstrap server config is required for producer to connect to brokers
if (isAsync) { // Send asynchronously props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServers);
sendAsync(messageKey, messageStr, currentTimeMs); // client id is not required, but it's good to track the source of requests beyond just ip/port
return; // by allowing a logical application name to be included in server-side request logging
props.put(ProducerConfig.CLIENT_ID_CONFIG, "client-" + UUID.randomUUID());
// key and value are just byte arrays, so we need to set appropriate serializers
props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, IntegerSerializer.class);
props.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, StringSerializer.class);
if (transactionTimeoutMs > 0) {
// max time before the transaction coordinator proactively aborts the ongoing transaction
props.put(ProducerConfig.TRANSACTION_TIMEOUT_CONFIG, transactionTimeoutMs);
} }
Future<RecordMetadata> future = send(messageKey, messageStr); if (transactionalId != null) {
future.get(); // the transactional id must be static and unique
System.out.println("Sent message: (" + messageKey + ", " + messageStr + ")"); // it is used to identify the same producer instance across process restarts
props.put(ProducerConfig.TRANSACTIONAL_ID_CONFIG, transactionalId);
}
// enable duplicates protection at the partition level
props.put(ProducerConfig.ENABLE_IDEMPOTENCE_CONFIG, enableIdempotency);
return new KafkaProducer<>(props);
} }
private void sendAsync(final int messageKey, final String messageStr, final long currentTimeMs) { private void asyncSend(KafkaProducer<Integer, String> producer, int key, String value) {
this.producer.send(new ProducerRecord<>(topic, // send the record asynchronously, setting a callback to be notified of the result
messageKey, // note that, even if you set a small batch.size with linger.ms=0, the send operation
messageStr), // will still be blocked when buffer.memory is full or metadata are not available
new DemoCallBack(currentTimeMs, messageKey, messageStr)); producer.send(new ProducerRecord<>(topic, key, value), new ProducerCallback(key, value));
} }
private Future<RecordMetadata> send(final int messageKey, final String messageStr) { private RecordMetadata syncSend(KafkaProducer<Integer, String> producer, int key, String value)
return producer.send(new ProducerRecord<>(topic, throws ExecutionException, InterruptedException {
messageKey, try {
messageStr)); // send the record and then call get, which blocks waiting for the ack from the broker
} RecordMetadata metadata = producer.send(new ProducerRecord<>(topic, key, value)).get();
} Utils.maybePrintRecord(numRecords, key, value, metadata);
return metadata;
class DemoCallBack implements Callback { } catch (AuthorizationException | UnsupportedVersionException | ProducerFencedException
| FencedInstanceIdException | OutOfOrderSequenceException | SerializationException e) {
private final long startTime; Utils.printErr(e.getMessage());
private final int key; // we can't recover from these exceptions
private final String message; shutdown();
} catch (KafkaException e) {
public DemoCallBack(long startTime, int key, String message) { Utils.printErr(e.getMessage());
this.startTime = startTime; }
this.key = key; return null;
this.message = message;
} }
/** class ProducerCallback implements Callback {
* A callback method the user can implement to provide asynchronous handling of request completion. This method will private final int key;
* be called when the record sent to the server has been acknowledged. When exception is not null in the callback, private final String value;
* metadata will contain the special -1 value for all fields except for topicPartition, which will be valid.
* public ProducerCallback(int key, String value) {
* @param metadata The metadata for the record that was sent (i.e. the partition and offset). An empty metadata this.key = key;
* with -1 value for all fields except for topicPartition will be returned if an error occurred. this.value = value;
* @param exception The exception thrown during processing of this record. Null if no error occurred. }
*/
public void onCompletion(RecordMetadata metadata, Exception exception) { /**
long elapsedTime = System.currentTimeMillis() - startTime; * A callback method the user can implement to provide asynchronous handling of request completion. This method will
if (metadata != null) { * be called when the record sent to the server has been acknowledged. When exception is not null in the callback,
System.out.println( * metadata will contain the special -1 value for all fields except for topicPartition, which will be valid.
"message(" + key + ", " + message + ") sent to partition(" + metadata.partition() + *
"), " + * @param metadata The metadata for the record that was sent (i.e. the partition and offset). An empty metadata
"offset(" + metadata.offset() + ") in " + elapsedTime + " ms"); * with -1 value for all fields except for topicPartition will be returned if an error occurred.
} else { * @param exception The exception thrown during processing of this record. Null if no error occurred.
exception.printStackTrace(); */
public void onCompletion(RecordMetadata metadata, Exception exception) {
if (exception != null) {
Utils.printErr(exception.getMessage());
if (!(exception instanceof RetriableException)) {
// we can't recover from these exceptions
shutdown();
}
} else {
Utils.maybePrintRecord(numRecords, key, value, metadata);
}
} }
} }
} }