1 /*
2 * librdkafka - Apache Kafka C library
3 *
4 * Copyright (c) 2012-2013, Magnus Edenhill
5 * All rights reserved.
6 *
7 * Redistribution and use in source and binary forms, with or without
8 * modification, are permitted provided that the following conditions are met:
9 *
10 * 1. Redistributions of source code must retain the above copyright notice,
11 * this list of conditions and the following disclaimer.
12 * 2. Redistributions in binary form must reproduce the above copyright notice,
13 * this list of conditions and the following disclaimer in the documentation
14 * and/or other materials provided with the distribution.
15 *
16 * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
17 * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
18 * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
19 * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE
20 * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
21 * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
22 * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
23 * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
24 * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
25 * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE
26 * POSSIBILITY OF SUCH DAMAGE.
27 */
28
29 /**
30 * Produce NULL payload messages, then consume them.
31 */
32
33 #include "test.h"
34
35 /* Typical include path would be <librdkafka/rdkafka.h>, but this program
36 * is built from within the librdkafka source tree and thus differs. */
37 #include "rdkafka.h" /* for Kafka driver */
38
39
40 static int prod_msg_remains = 0;
41 static int fails = 0;
42
43 /**
44 * Delivery reported callback.
45 * Called for each message once to signal its delivery status.
46 */
dr_cb(rd_kafka_t * rk,void * payload,size_t len,rd_kafka_resp_err_t err,void * opaque,void * msg_opaque)47 static void dr_cb (rd_kafka_t *rk, void *payload, size_t len,
48 rd_kafka_resp_err_t err, void *opaque, void *msg_opaque) {
49
50 if (err != RD_KAFKA_RESP_ERR_NO_ERROR)
51 TEST_FAIL("Message delivery failed: %s\n",
52 rd_kafka_err2str(err));
53
54 if (prod_msg_remains == 0)
55 TEST_FAIL("Too many messages delivered (prod_msg_remains %i)",
56 prod_msg_remains);
57
58 prod_msg_remains--;
59 }
60
61
62 /**
63 * Produces 'msgcnt' messages split over 'partition_cnt' partitions.
64 */
produce_null_messages(uint64_t testid,const char * topic,int partition_cnt,int msgcnt)65 static void produce_null_messages (uint64_t testid, const char *topic,
66 int partition_cnt, int msgcnt) {
67 int r;
68 rd_kafka_t *rk;
69 rd_kafka_topic_t *rkt;
70 rd_kafka_conf_t *conf;
71 rd_kafka_topic_conf_t *topic_conf;
72 char errstr[512];
73 int i;
74 int32_t partition;
75 int msgid = 0;
76
77 test_conf_init(&conf, &topic_conf, 20);
78
79 rd_kafka_conf_set_dr_cb(conf, dr_cb);
80
81 /* Make sure all replicas are in-sync after producing
82 * so that consume test wont fail. */
83 rd_kafka_topic_conf_set(topic_conf, "request.required.acks", "-1",
84 errstr, sizeof(errstr));
85
86 /* Create kafka instance */
87 rk = test_create_handle(RD_KAFKA_PRODUCER, conf);
88
89 rkt = rd_kafka_topic_new(rk, topic, topic_conf);
90 if (!rkt)
91 TEST_FAIL("Failed to create topic: %s\n",
92 rd_kafka_err2str(rd_kafka_last_error()));
93
94 /* Produce messages */
95 prod_msg_remains = msgcnt;
96 for (partition = 0 ; partition < partition_cnt ; partition++) {
97 int batch_cnt = msgcnt / partition_cnt;
98
99 for (i = 0 ; i < batch_cnt ; i++) {
100 char key[128];
101 rd_snprintf(key, sizeof(key),
102 "testid=%"PRIu64", partition=%i, msg=%i",
103 testid, (int)partition, msgid);
104 r = rd_kafka_produce(rkt, partition, 0,
105 NULL, 0,
106 key, strlen(key),
107 NULL);
108 if (r == -1)
109 TEST_FAIL("Failed to produce message %i "
110 "to partition %i: %s",
111 msgid, (int)partition,
112 rd_kafka_err2str(rd_kafka_last_error()));
113 msgid++;
114 }
115 }
116
117
118 TEST_SAY("Produced %d messages to %d partition(s), "
119 "waiting for deliveries\n", msgcnt, partition_cnt);
120 /* Wait for messages to be delivered */
121 while (rd_kafka_outq_len(rk) > 0)
122 rd_kafka_poll(rk, 100);
123
124 if (fails)
125 TEST_FAIL("%i failures, see previous errors", fails);
126
127 if (prod_msg_remains != 0)
128 TEST_FAIL("Still waiting for %i messages to be produced",
129 prod_msg_remains);
130 else
131 TEST_SAY("All messages delivered\n");
132
133 /* Destroy topic */
134 rd_kafka_topic_destroy(rkt);
135
136 /* Destroy rdkafka instance */
137 TEST_SAY("Destroying kafka instance %s\n", rd_kafka_name(rk));
138 rd_kafka_destroy(rk);
139 }
140
141
142
143 static int *cons_msgs;
144 static int cons_msgs_size;
145 static int cons_msgs_cnt;
146
verify_consumed_msg_reset(int msgcnt)147 static void verify_consumed_msg_reset (int msgcnt) {
148 if (cons_msgs) {
149 free(cons_msgs);
150 cons_msgs = NULL;
151 }
152
153 if (msgcnt) {
154 int i;
155
156 cons_msgs = malloc(sizeof(*cons_msgs) * msgcnt);
157 for (i = 0 ; i < msgcnt ; i++)
158 cons_msgs[i] = -1;
159 }
160
161 cons_msgs_size = msgcnt;
162 cons_msgs_cnt = 0;
163 }
164
165
int_cmp(const void * _a,const void * _b)166 static int int_cmp (const void *_a, const void *_b) {
167 int a = *(int *)_a;
168 int b = *(int *)_b;
169 return RD_CMP(a, b);
170 }
171
verify_consumed_msg_check0(const char * func,int line)172 static void verify_consumed_msg_check0 (const char *func, int line) {
173 int i;
174 int fails = 0;
175
176 if (cons_msgs_cnt < cons_msgs_size) {
177 TEST_SAY("Missing %i messages in consumer\n",
178 cons_msgs_size - cons_msgs_cnt);
179 fails++;
180 }
181
182 qsort(cons_msgs, cons_msgs_size, sizeof(*cons_msgs), int_cmp);
183
184 for (i = 0 ; i < cons_msgs_size ; i++) {
185 if (cons_msgs[i] != i) {
186 TEST_SAY("Consumed message #%i is wrong, "
187 "expected #%i\n",
188 cons_msgs[i], i);
189 fails++;
190 }
191 }
192
193 if (fails)
194 TEST_FAIL("See above error(s)");
195
196 verify_consumed_msg_reset(0);
197 }
198
199
200 #define verify_consumed_msg_check() \
201 verify_consumed_msg_check0(__FUNCTION__,__LINE__)
202
203
204
verify_consumed_msg0(const char * func,int line,uint64_t testid,int32_t partition,int msgnum,rd_kafka_message_t * rkmessage)205 static void verify_consumed_msg0 (const char *func, int line,
206 uint64_t testid, int32_t partition,
207 int msgnum,
208 rd_kafka_message_t *rkmessage) {
209 uint64_t in_testid;
210 int in_part;
211 int in_msgnum;
212 char buf[128];
213
214 if (rkmessage->len != 0)
215 TEST_FAIL("Incoming message not NULL: %i bytes",
216 (int)rkmessage->len);
217
218 if (rkmessage->key_len +1 >= sizeof(buf))
219 TEST_FAIL("Incoming message key too large (%i): "
220 "not sourced by this test",
221 (int)rkmessage->key_len);
222
223 rd_snprintf(buf, sizeof(buf), "%.*s",
224 (int)rkmessage->key_len, (char *)rkmessage->key);
225
226 if (sscanf(buf, "testid=%"SCNu64", partition=%i, msg=%i",
227 &in_testid, &in_part, &in_msgnum) != 3)
228 TEST_FAIL("Incorrect key format: %s", buf);
229
230 if (testid != in_testid ||
231 (partition != -1 && partition != in_part) ||
232 (msgnum != -1 && msgnum != in_msgnum) ||
233 (in_msgnum < 0 || in_msgnum > cons_msgs_size))
234 goto fail_match;
235
236 if (test_level > 2) {
237 TEST_SAY("%s:%i: Our testid %"PRIu64", part %i (%i), "
238 "msg %i/%i did "
239 ", key's: \"%s\"\n",
240 func, line,
241 testid, (int)partition, (int)rkmessage->partition,
242 msgnum, cons_msgs_size, buf);
243 }
244
245 if (cons_msgs_cnt == cons_msgs_size) {
246 TEST_SAY("Too many messages in cons_msgs (%i) while reading "
247 "message key \"%s\"\n",
248 cons_msgs_cnt, buf);
249 verify_consumed_msg_check();
250 TEST_FAIL("See above error(s)");
251 }
252
253 cons_msgs[cons_msgs_cnt++] = in_msgnum;
254
255 return;
256
257 fail_match:
258 TEST_FAIL("%s:%i: Our testid %"PRIu64", part %i, msg %i/%i did "
259 "not match message's key: \"%s\"\n",
260 func, line,
261 testid, (int)partition, msgnum, cons_msgs_size, buf);
262 }
263
264 #define verify_consumed_msg(testid,part,msgnum,rkmessage) \
265 verify_consumed_msg0(__FUNCTION__,__LINE__,testid,part,msgnum,rkmessage)
266
267
consume_messages(uint64_t testid,const char * topic,int32_t partition,int msg_base,int batch_cnt,int msgcnt)268 static void consume_messages (uint64_t testid, const char *topic,
269 int32_t partition, int msg_base, int batch_cnt,
270 int msgcnt) {
271 rd_kafka_t *rk;
272 rd_kafka_topic_t *rkt;
273 rd_kafka_conf_t *conf;
274 rd_kafka_topic_conf_t *topic_conf;
275 int i;
276
277 test_conf_init(&conf, &topic_conf, 20);
278
279 /* Create kafka instance */
280 rk = test_create_handle(RD_KAFKA_CONSUMER, conf);
281
282 rkt = rd_kafka_topic_new(rk, topic, topic_conf);
283 if (!rkt)
284 TEST_FAIL("Failed to create topic: %s\n",
285 rd_kafka_err2str(rd_kafka_last_error()));
286
287 TEST_SAY("Consuming %i messages from partition %i\n",
288 batch_cnt, partition);
289
290 /* Consume messages */
291 if (rd_kafka_consume_start(rkt, partition,
292 RD_KAFKA_OFFSET_TAIL(batch_cnt)) == -1)
293 TEST_FAIL("consume_start(%i, -%i) failed: %s",
294 (int)partition, batch_cnt,
295 rd_kafka_err2str(rd_kafka_last_error()));
296
297 for (i = 0 ; i < batch_cnt ; i++) {
298 rd_kafka_message_t *rkmessage;
299
300 rkmessage = rd_kafka_consume(rkt, partition, tmout_multip(5000));
301 if (!rkmessage)
302 TEST_FAIL("Failed to consume message %i/%i from "
303 "partition %i: %s",
304 i, batch_cnt, (int)partition,
305 rd_kafka_err2str(rd_kafka_last_error()));
306 if (rkmessage->err)
307 TEST_FAIL("Consume message %i/%i from partition %i "
308 "has error: %s",
309 i, batch_cnt, (int)partition,
310 rd_kafka_err2str(rkmessage->err));
311
312 verify_consumed_msg(testid, partition, msg_base+i, rkmessage);
313
314 rd_kafka_message_destroy(rkmessage);
315 }
316
317 rd_kafka_consume_stop(rkt, partition);
318
319 /* Destroy topic */
320 rd_kafka_topic_destroy(rkt);
321
322 /* Destroy rdkafka instance */
323 TEST_SAY("Destroying kafka instance %s\n", rd_kafka_name(rk));
324 rd_kafka_destroy(rk);
325 }
326
327
consume_messages_with_queues(uint64_t testid,const char * topic,int partition_cnt,int msgcnt)328 static void consume_messages_with_queues (uint64_t testid, const char *topic,
329 int partition_cnt, int msgcnt) {
330 rd_kafka_t *rk;
331 rd_kafka_topic_t *rkt;
332 rd_kafka_conf_t *conf;
333 rd_kafka_topic_conf_t *topic_conf;
334 rd_kafka_queue_t *rkqu;
335 int i;
336 int32_t partition;
337 int batch_cnt = msgcnt / partition_cnt;
338
339 test_conf_init(&conf, &topic_conf, 20);
340
341 /* Create kafka instance */
342 rk = test_create_handle(RD_KAFKA_CONSUMER, conf);
343
344 /* Create queue */
345 rkqu = rd_kafka_queue_new(rk);
346
347
348 rkt = rd_kafka_topic_new(rk, topic, topic_conf);
349 if (!rkt)
350 TEST_FAIL("Failed to create topic: %s\n",
351 rd_kafka_err2str(rd_kafka_last_error()));
352
353 TEST_SAY("Consuming %i messages from one queue serving %i partitions\n",
354 msgcnt, partition_cnt);
355
356 /* Start consuming each partition */
357 for (partition = 0 ; partition < partition_cnt ; partition++) {
358 /* Consume messages */
359 TEST_SAY("Start consuming partition %i at tail offset -%i\n",
360 partition, batch_cnt);
361 if (rd_kafka_consume_start_queue(rkt, partition,
362 RD_KAFKA_OFFSET_TAIL(batch_cnt),
363 rkqu) == -1)
364 TEST_FAIL("consume_start_queue(%i) failed: %s",
365 (int)partition,
366 rd_kafka_err2str(rd_kafka_last_error()));
367 }
368
369
370 /* Consume messages from queue */
371 for (i = 0 ; i < msgcnt ; i++) {
372 rd_kafka_message_t *rkmessage;
373
374 rkmessage = rd_kafka_consume_queue(rkqu, tmout_multip(5000));
375 if (!rkmessage)
376 TEST_FAIL("Failed to consume message %i/%i from "
377 "queue: %s",
378 i, msgcnt,
379 rd_kafka_err2str(rd_kafka_last_error()));
380 if (rkmessage->err)
381 TEST_FAIL("Consume message %i/%i from queue "
382 "has error (partition %"PRId32"): %s",
383 i, msgcnt,
384 rkmessage->partition,
385 rd_kafka_err2str(rkmessage->err));
386
387 verify_consumed_msg(testid, -1, -1, rkmessage);
388
389 rd_kafka_message_destroy(rkmessage);
390 }
391
392 /* Stop consuming each partition */
393 for (partition = 0 ; partition < partition_cnt ; partition++)
394 rd_kafka_consume_stop(rkt, partition);
395
396 /* Destroy queue */
397 rd_kafka_queue_destroy(rkqu);
398
399 /* Destroy topic */
400 rd_kafka_topic_destroy(rkt);
401
402 /* Destroy rdkafka instance */
403 TEST_SAY("Destroying kafka instance %s\n", rd_kafka_name(rk));
404 rd_kafka_destroy(rk);
405 }
406
407
test_produce_consume(void)408 static void test_produce_consume (void) {
409 int msgcnt = test_quick ? 100 : 1000;
410 int partition_cnt = 1;
411 int i;
412 uint64_t testid;
413 int msg_base = 0;
414 const char *topic;
415
416 /* Generate a testid so we can differentiate messages
417 * from other tests */
418 testid = test_id_generate();
419
420 /* Read test.conf to configure topic name */
421 test_conf_init(NULL, NULL, 20);
422 topic = test_mk_topic_name("0013", 0);
423
424 TEST_SAY("Topic %s, testid %"PRIu64"\n", topic, testid);
425
426 /* Produce messages */
427 produce_null_messages(testid, topic, partition_cnt, msgcnt);
428
429
430 /* Consume messages with standard interface */
431 verify_consumed_msg_reset(msgcnt);
432 for (i = 0 ; i < partition_cnt ; i++) {
433 consume_messages(testid, topic, i,
434 msg_base, msgcnt / partition_cnt, msgcnt);
435 msg_base += msgcnt / partition_cnt;
436 }
437 verify_consumed_msg_check();
438
439 /* Consume messages with queue interface */
440 verify_consumed_msg_reset(msgcnt);
441 consume_messages_with_queues(testid, topic, partition_cnt, msgcnt);
442 verify_consumed_msg_check();
443
444 return;
445 }
446
447
448
449
main_0013_null_msgs(int argc,char ** argv)450 int main_0013_null_msgs (int argc, char **argv) {
451 test_produce_consume();
452 return 0;
453 }
454