|
16 | 16 | */
|
17 | 17 | package kafka.examples;
|
18 | 18 |
|
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; |
27 | 19 | import java.util.List;
|
28 | 20 | import java.util.Optional;
|
29 |
| -import java.util.Properties; |
30 |
| -import java.util.Set; |
31 | 21 | import java.util.concurrent.CountDownLatch;
|
32 |
| -import java.util.concurrent.ExecutionException; |
33 | 22 | import java.util.concurrent.TimeUnit;
|
34 | 23 | import java.util.stream.Collectors;
|
35 | 24 | import java.util.stream.IntStream;
|
36 | 25 |
|
37 | 26 | /**
|
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: |
55 | 28 | *
|
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. |
63 | 40 | *
|
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 |
73 | 42 | * {@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. |
74 | 47 | */
|
75 | 48 | public class KafkaExactlyOnceDemo {
|
76 | 49 | public static final String BOOTSTRAP_SERVERS = "localhost:9092";
|
77 | 50 | private static final String INPUT_TOPIC = "input-topic";
|
78 | 51 | private static final String OUTPUT_TOPIC = "output-topic";
|
| 52 | + public static final String GROUP_NAME = "check-group"; |
79 | 53 |
|
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; |
161 | 62 | }
|
162 |
| - Thread.sleep(1000); |
163 |
| - } |
164 | 63 |
|
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 | + } |
182 | 81 |
|
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; |
184 | 93 | }
|
185 |
| - } |
186 |
| - } |
187 | 94 |
|
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(); |
195 | 103 | }
|
196 |
| - System.out.println("Encountered exception during topic deletion: " + e.getCause()); |
| 104 | + } catch (Throwable e) { |
| 105 | + e.printStackTrace(); |
197 | 106 | }
|
198 |
| - System.out.println("Deleted old topics: " + topicsToDelete); |
199 | 107 | }
|
200 | 108 | }
|
0 commit comments