Skip to content

Commit c757af5

Browse files
authored
KAFKA-14752: Kafka examples improvements - demo changes (#13517)
KAFKA-14752: Kafka examples improvements - demo changes Reviewers: Luke Chen <[email protected]>
1 parent 54a4067 commit c757af5

File tree

6 files changed

+121
-188
lines changed

6 files changed

+121
-188
lines changed

build.gradle

-1
Original file line numberDiff line numberDiff line change
@@ -1294,7 +1294,6 @@ project(':examples') {
12941294

12951295
dependencies {
12961296
implementation project(':clients')
1297-
implementation project(':server-common')
12981297
}
12991298

13001299
javadoc {

checkstyle/import-control-core.xml

-1
Original file line numberDiff line numberDiff line change
@@ -71,7 +71,6 @@
7171

7272
<subpackage name="examples">
7373
<allow pkg="org.apache.kafka.clients" />
74-
<allow pkg="org.apache.kafka.server.util" />
7574
</subpackage>
7675

7776
<subpackage name="log.remote">

examples/README

-12
This file was deleted.

examples/README.md

+9
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,9 @@
1+
# Kafka client examples
2+
3+
This module contains some Kafka client examples.
4+
5+
1. Start a Kafka 2.5+ local cluster with a plain listener configured on port 9092.
6+
2. Run `examples/bin/java-producer-consumer-demo.sh 10000` to asynchronously send 10k records to topic1 and consume them.
7+
3. Run `examples/bin/java-producer-consumer-demo.sh 10000 sync` to synchronous send 10k records to topic1 and consume them.
8+
4. Run `examples/bin/exactly-once-demo.sh 6 3 10000` to create input-topic and output-topic with 6 partitions each,
9+
start 3 transactional application instances and process 10k records.

examples/src/main/java/kafka/examples/KafkaConsumerProducerDemo.java

+48-18
Original file line numberDiff line numberDiff line change
@@ -16,29 +16,59 @@
1616
*/
1717
package kafka.examples;
1818

19-
import org.apache.kafka.common.errors.TimeoutException;
20-
2119
import java.util.Optional;
2220
import java.util.concurrent.CountDownLatch;
2321
import java.util.concurrent.TimeUnit;
2422

23+
/**
24+
* This example can be decomposed into the following stages:
25+
*
26+
* 1. Clean any topics left from previous runs.
27+
* 2. Create a producer thread to send a set of records to topic1.
28+
* 3. Create a consumer thread to fetch all previously sent records from topic1.
29+
*
30+
* If you are using IntelliJ IDEA, the above arguments should be put in `Modify Run Configuration - Program Arguments`.
31+
* You can also set an output log file in `Modify Run Configuration - Modify options - Save console output to file` to
32+
* record all the log output together.
33+
*/
2534
public class KafkaConsumerProducerDemo {
26-
public static void main(String[] args) throws InterruptedException {
27-
boolean isAsync = args.length == 0 || !args[0].trim().equalsIgnoreCase("sync");
28-
CountDownLatch latch = new CountDownLatch(2);
29-
Producer producerThread = new Producer(
30-
"producer", KafkaProperties.KAFKA_SERVER_URL + ":" + KafkaProperties.KAFKA_SERVER_PORT, KafkaProperties.TOPIC, isAsync, null, false, 10000, -1, latch);
31-
producerThread.start();
32-
33-
Consumer consumerThread = new Consumer(
34-
"consumer", KafkaProperties.KAFKA_SERVER_URL + ":" + KafkaProperties.KAFKA_SERVER_PORT, KafkaProperties.TOPIC, "DemoConsumer", Optional.empty(), false, 10000, latch);
35-
consumerThread.start();
36-
37-
if (!latch.await(5, TimeUnit.MINUTES)) {
38-
throw new TimeoutException("Timeout after 5 minutes waiting for demo producer and consumer to finish");
39-
}
35+
public static final String BOOTSTRAP_SERVERS = "localhost:9092";
36+
public static final String TOPIC_NAME = "my-topic";
37+
public static final String GROUP_NAME = "my-group";
38+
39+
public static void main(String[] args) {
40+
try {
41+
if (args.length == 0) {
42+
Utils.printHelp("This example takes 2 parameters (i.e. 10000 sync):%n" +
43+
"- records: total number of records to send (required)%n" +
44+
"- mode: pass 'sync' to send records synchronously (optional)");
45+
return;
46+
}
4047

41-
consumerThread.shutdown();
42-
System.out.println("All finished!");
48+
int numRecords = Integer.parseInt(args[0]);
49+
boolean isAsync = args.length == 1 || !args[1].trim().equalsIgnoreCase("sync");
50+
51+
// stage 1: clean any topics left from previous runs
52+
Utils.recreateTopics(BOOTSTRAP_SERVERS, -1, TOPIC_NAME);
53+
CountDownLatch latch = new CountDownLatch(2);
54+
55+
// stage 2: produce records to topic1
56+
Producer producerThread = new Producer(
57+
"producer", BOOTSTRAP_SERVERS, TOPIC_NAME, isAsync, null, false, numRecords, -1, latch);
58+
producerThread.start();
59+
60+
// stage 3: consume records from topic1
61+
Consumer consumerThread = new Consumer(
62+
"consumer", BOOTSTRAP_SERVERS, TOPIC_NAME, GROUP_NAME, Optional.empty(), false, numRecords, latch);
63+
consumerThread.start();
64+
65+
if (!latch.await(5, TimeUnit.MINUTES)) {
66+
Utils.printErr("Timeout after 5 minutes waiting for termination");
67+
producerThread.shutdown();
68+
consumerThread.shutdown();
69+
}
70+
} catch (Throwable e) {
71+
e.printStackTrace();
72+
}
4373
}
4474
}

examples/src/main/java/kafka/examples/KafkaExactlyOnceDemo.java

+64-156
Original file line numberDiff line numberDiff line change
@@ -16,185 +16,93 @@
1616
*/
1717
package kafka.examples;
1818

19-
import org.apache.kafka.clients.admin.Admin;
20-
import org.apache.kafka.clients.admin.NewTopic;
21-
import org.apache.kafka.clients.consumer.ConsumerConfig;
22-
import org.apache.kafka.common.errors.TimeoutException;
23-
import org.apache.kafka.common.errors.TopicExistsException;
24-
import org.apache.kafka.common.errors.UnknownTopicOrPartitionException;
25-
26-
import java.util.Arrays;
2719
import java.util.List;
2820
import java.util.Optional;
29-
import java.util.Properties;
30-
import java.util.Set;
3121
import java.util.concurrent.CountDownLatch;
32-
import java.util.concurrent.ExecutionException;
3322
import java.util.concurrent.TimeUnit;
3423
import java.util.stream.Collectors;
3524
import java.util.stream.IntStream;
3625

3726
/**
38-
* This exactly once demo driver takes 3 arguments:
39-
* - partition: number of partitions for input/output topic
40-
* - instances: number of instances
41-
* - records: number of records
42-
* An example argument list would be `6 3 50000`.
43-
*
44-
* If you are using IntelliJ IDEA, the above arguments should be put in the configuration's `Program Arguments`.
45-
* Also recommended to set an output log file by `Edit Configuration -> Logs -> Save console
46-
* output to file` to record all the log output together.
47-
*
48-
* The driver could be decomposed as following stages:
49-
*
50-
* 1. Cleanup any topic whose name conflicts with input and output topic, so that we have a clean-start.
51-
*
52-
* 2. Set up a producer in a separate thread to pre-populate a set of records with even number keys into
53-
* the input topic. The driver will block for the record generation to finish, so the producer
54-
* must be in synchronous sending mode.
27+
* This example can be decomposed into the following stages:
5528
*
56-
* 3. Set up transactional instances in separate threads which does a consume-process-produce loop,
57-
* tailing data from input topic (See {@link ExactlyOnceMessageProcessor}). Each EOS instance will
58-
* drain all the records from either given partitions or auto assigned partitions by actively
59-
* comparing log end offset with committed offset. Each record will be processed exactly once
60-
* as dividing the key by 2, and extend the value message. The driver will block for all the record
61-
* processing to finish. The transformed record shall be written to the output topic, with
62-
* transactional guarantee.
29+
* 1. Clean any topics left from previous runs.
30+
* 2. Set up a producer thread to pre-populate a set of records with even number keys into the input topic.
31+
* The demo will block for the record generation to finish, so the producer is synchronous.
32+
* 3. Set up the transactional instances in separate threads, each one executing a read-process-write loop
33+
* (See {@link ExactlyOnceMessageProcessor}). Each EOS instance will drain all records from either given
34+
* partitions or auto assigned partitions by actively comparing log end offset with committed offset.
35+
* Each record will be processed exactly-once with strong partition level ordering guarantee.
36+
* The demo will block until all records are processed and written to the output topic.
37+
* 4. Create a read_committed consumer thread to verify we have all records in the output topic,
38+
* and record ordering at the partition level is maintained.
39+
* The demo will block for the consumption of all committed records, with transactional guarantee.
6340
*
64-
* 4. Set up a read committed consumer in a separate thread to verify we have all records within
65-
* the output topic, while the message ordering on partition level is maintained.
66-
* The driver will block for the consumption of all committed records.
67-
*
68-
* From this demo, you could see that all the records from pre-population are processed exactly once,
69-
* with strong partition level ordering guarantee.
70-
*
71-
* Note: please start the kafka broker and zookeeper in local first. The broker version must be >= 2.5
72-
* in order to run, otherwise the app could throw
41+
* Broker version must be >= 2.5.0 in order to run, otherwise the example will throw
7342
* {@link org.apache.kafka.common.errors.UnsupportedVersionException}.
43+
*
44+
* If you are using IntelliJ IDEA, the above arguments should be put in `Modify Run Configuration - Program Arguments`.
45+
* You can also set an output log file in `Modify Run Configuration - Modify options - Save console output to file` to
46+
* record all the log output together.
7447
*/
7548
public class KafkaExactlyOnceDemo {
7649
public static final String BOOTSTRAP_SERVERS = "localhost:9092";
7750
private static final String INPUT_TOPIC = "input-topic";
7851
private static final String OUTPUT_TOPIC = "output-topic";
52+
public static final String GROUP_NAME = "check-group";
7953

80-
public static void main(String[] args) throws InterruptedException, ExecutionException {
81-
if (args.length != 3) {
82-
throw new IllegalArgumentException("Should accept 3 parameters: " +
83-
"[number of partitions], [number of instances], [number of records]");
84-
}
85-
86-
int numPartitions = Integer.parseInt(args[0]);
87-
int numInstances = Integer.parseInt(args[1]);
88-
int numRecords = Integer.parseInt(args[2]);
89-
90-
/* Stage 1: topic cleanup and recreation */
91-
recreateTopics(numPartitions);
92-
93-
CountDownLatch prePopulateLatch = new CountDownLatch(1);
94-
95-
/* Stage 2: pre-populate records */
96-
Producer producerThread = new Producer(
97-
"producer", KafkaProperties.KAFKA_SERVER_URL + ":" + KafkaProperties.KAFKA_SERVER_PORT, INPUT_TOPIC, false, null, true, numRecords, -1, prePopulateLatch);
98-
producerThread.start();
99-
100-
if (!prePopulateLatch.await(5, TimeUnit.MINUTES)) {
101-
throw new TimeoutException("Timeout after 5 minutes waiting for data pre-population");
102-
}
103-
104-
CountDownLatch transactionalCopyLatch = new CountDownLatch(numInstances);
105-
106-
/* Stage 3: transactionally process all messages */
107-
CountDownLatch processorsLatch = new CountDownLatch(numInstances);
108-
List<ExactlyOnceMessageProcessor> processors = IntStream.range(0, numInstances)
109-
.mapToObj(id -> new ExactlyOnceMessageProcessor(
110-
"processor-" + id, BOOTSTRAP_SERVERS, INPUT_TOPIC, OUTPUT_TOPIC, processorsLatch))
111-
.collect(Collectors.toList());
112-
processors.forEach(ExactlyOnceMessageProcessor::start);
113-
114-
if (!transactionalCopyLatch.await(5, TimeUnit.MINUTES)) {
115-
throw new TimeoutException("Timeout after 5 minutes waiting for transactionally message copy");
116-
}
117-
118-
CountDownLatch consumeLatch = new CountDownLatch(1);
119-
120-
/* Stage 4: consume all processed messages to verify exactly once */
121-
Consumer consumerThread = new Consumer(
122-
"consumer", "DemoConsumer", OUTPUT_TOPIC, "Verify-consumer", Optional.empty(), true, numRecords, consumeLatch);
123-
consumerThread.start();
124-
125-
if (!consumeLatch.await(5, TimeUnit.MINUTES)) {
126-
throw new TimeoutException("Timeout after 5 minutes waiting for output data consumption");
127-
}
128-
129-
consumerThread.shutdown();
130-
System.out.println("All finished!");
131-
}
132-
133-
private static void recreateTopics(final int numPartitions)
134-
throws ExecutionException, InterruptedException {
135-
Properties props = new Properties();
136-
props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG,
137-
KafkaProperties.KAFKA_SERVER_URL + ":" + KafkaProperties.KAFKA_SERVER_PORT);
138-
139-
Admin adminClient = Admin.create(props);
140-
141-
List<String> topicsToDelete = Arrays.asList(INPUT_TOPIC, OUTPUT_TOPIC);
142-
143-
deleteTopic(adminClient, topicsToDelete);
144-
145-
// Check topic existence in a retry loop
146-
while (true) {
147-
System.out.println("Making sure the topics are deleted successfully: " + topicsToDelete);
148-
149-
Set<String> listedTopics = adminClient.listTopics().names().get();
150-
System.out.println("Current list of topics: " + listedTopics);
151-
152-
boolean hasTopicInfo = false;
153-
for (String listedTopic : listedTopics) {
154-
if (topicsToDelete.contains(listedTopic)) {
155-
hasTopicInfo = true;
156-
break;
157-
}
158-
}
159-
if (!hasTopicInfo) {
160-
break;
54+
public static void main(String[] args) {
55+
try {
56+
if (args.length != 3) {
57+
Utils.printHelp("This example takes 3 parameters (i.e. 6 3 10000):%n" +
58+
"- partition: number of partitions for input and output topics (required)%n" +
59+
"- instances: number of application instances (required)%n" +
60+
"- records: total number of records (required)");
61+
return;
16162
}
162-
Thread.sleep(1000);
163-
}
16463

165-
// Create topics in a retry loop
166-
while (true) {
167-
final short replicationFactor = 1;
168-
final List<NewTopic> newTopics = Arrays.asList(
169-
new NewTopic(INPUT_TOPIC, numPartitions, replicationFactor),
170-
new NewTopic(OUTPUT_TOPIC, numPartitions, replicationFactor));
171-
try {
172-
adminClient.createTopics(newTopics).all().get();
173-
System.out.println("Created new topics: " + newTopics);
174-
break;
175-
} catch (ExecutionException e) {
176-
if (!(e.getCause() instanceof TopicExistsException)) {
177-
throw e;
178-
}
179-
System.out.println("Metadata of the old topics are not cleared yet...");
180-
181-
deleteTopic(adminClient, topicsToDelete);
64+
int numPartitions = Integer.parseInt(args[0]);
65+
int numInstances = Integer.parseInt(args[1]);
66+
int numRecords = Integer.parseInt(args[2]);
67+
68+
// stage 1: clean any topics left from previous runs
69+
Utils.recreateTopics(BOOTSTRAP_SERVERS, numPartitions, INPUT_TOPIC, OUTPUT_TOPIC);
70+
71+
// stage 2: send demo records to the input-topic
72+
CountDownLatch producerLatch = new CountDownLatch(1);
73+
Producer producerThread = new Producer(
74+
"producer", BOOTSTRAP_SERVERS, INPUT_TOPIC, false, null, true, numRecords, -1, producerLatch);
75+
producerThread.start();
76+
if (!producerLatch.await(2, TimeUnit.MINUTES)) {
77+
Utils.printErr("Timeout after 2 minutes waiting for data load");
78+
producerThread.shutdown();
79+
return;
80+
}
18281

183-
Thread.sleep(1000);
82+
// stage 3: read from input-topic, process once and write to the output-topic
83+
CountDownLatch processorsLatch = new CountDownLatch(numInstances);
84+
List<ExactlyOnceMessageProcessor> processors = IntStream.range(0, numInstances)
85+
.mapToObj(id -> new ExactlyOnceMessageProcessor(
86+
"processor-" + id, BOOTSTRAP_SERVERS, INPUT_TOPIC, OUTPUT_TOPIC, processorsLatch))
87+
.collect(Collectors.toList());
88+
processors.forEach(ExactlyOnceMessageProcessor::start);
89+
if (!processorsLatch.await(2, TimeUnit.MINUTES)) {
90+
Utils.printErr("Timeout after 2 minutes waiting for record copy");
91+
processors.forEach(ExactlyOnceMessageProcessor::shutdown);
92+
return;
18493
}
185-
}
186-
}
18794

188-
private static void deleteTopic(final Admin adminClient, final List<String> topicsToDelete)
189-
throws InterruptedException, ExecutionException {
190-
try {
191-
adminClient.deleteTopics(topicsToDelete).all().get();
192-
} catch (ExecutionException e) {
193-
if (!(e.getCause() instanceof UnknownTopicOrPartitionException)) {
194-
throw e;
95+
// stage 4: check consuming records from the output-topic
96+
CountDownLatch consumerLatch = new CountDownLatch(1);
97+
Consumer consumerThread = new Consumer(
98+
"consumer", BOOTSTRAP_SERVERS, OUTPUT_TOPIC, GROUP_NAME, Optional.empty(), true, numRecords, consumerLatch);
99+
consumerThread.start();
100+
if (!consumerLatch.await(2, TimeUnit.MINUTES)) {
101+
Utils.printErr("Timeout after 2 minutes waiting for output read");
102+
consumerThread.shutdown();
195103
}
196-
System.out.println("Encountered exception during topic deletion: " + e.getCause());
104+
} catch (Throwable e) {
105+
e.printStackTrace();
197106
}
198-
System.out.println("Deleted old topics: " + topicsToDelete);
199107
}
200108
}

0 commit comments

Comments
 (0)