forked from confluentinc/librdkafka
-
Notifications
You must be signed in to change notification settings - Fork 0
/
0080-admin_ut.c
751 lines (610 loc) · 29.8 KB
/
0080-admin_ut.c
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
/*
* librdkafka - Apache Kafka C library
*
* Copyright (c) 2012-2015, Magnus Edenhill
* All rights reserved.
*
* Redistribution and use in source and binary forms, with or without
* modification, are permitted provided that the following conditions are met:
*
* 1. Redistributions of source code must retain the above copyright notice,
* this list of conditions and the following disclaimer.
* 2. Redistributions in binary form must reproduce the above copyright notice,
* this list of conditions and the following disclaimer in the documentation
* and/or other materials provided with the distribution.
*
* THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
* AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
* IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE
* ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE
* LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR
* CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF
* SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS
* INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN
* CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE)
* ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE
* POSSIBILITY OF SUCH DAMAGE.
*/
#include "test.h"
#include "rdkafka.h"
/**
* @brief Admin API local dry-run unit-tests.
*/
#define MY_SOCKET_TIMEOUT_MS 100
#define MY_SOCKET_TIMEOUT_MS_STR "100"
static mtx_t last_event_lock;
static cnd_t last_event_cnd;
static rd_kafka_event_t *last_event = NULL;
/**
* @brief The background event callback is called automatically
* by librdkafka from a background thread.
*/
static void background_event_cb (rd_kafka_t *rk, rd_kafka_event_t *rkev,
void *opaque) {
mtx_lock(&last_event_lock);
TEST_ASSERT(!last_event, "Multiple events seen in background_event_cb "
"(existing %s, new %s)",
rd_kafka_event_name(last_event), rd_kafka_event_name(rkev));
last_event = rkev;
mtx_unlock(&last_event_lock);
cnd_broadcast(&last_event_cnd);
rd_sleep(1);
}
static rd_kafka_event_t *wait_background_event_cb (void) {
rd_kafka_event_t *rkev;
mtx_lock(&last_event_lock);
while (!(rkev = last_event))
cnd_wait(&last_event_cnd, &last_event_lock);
last_event = NULL;
mtx_unlock(&last_event_lock);
return rkev;
}
/**
* @brief CreateTopics tests
*
*
*
*/
static void do_test_CreateTopics (const char *what,
rd_kafka_t *rk, rd_kafka_queue_t *useq,
int with_background_event_cb,
int with_options) {
rd_kafka_queue_t *q = useq ? useq : rd_kafka_queue_new(rk);
#define MY_NEW_TOPICS_CNT 6
rd_kafka_NewTopic_t *new_topics[MY_NEW_TOPICS_CNT];
rd_kafka_AdminOptions_t *options = NULL;
int exp_timeout = MY_SOCKET_TIMEOUT_MS;
int i;
char errstr[512];
const char *errstr2;
rd_kafka_resp_err_t err;
test_timing_t timing;
rd_kafka_event_t *rkev;
const rd_kafka_CreateTopics_result_t *res;
const rd_kafka_topic_result_t **restopics;
size_t restopic_cnt;
void *my_opaque = NULL, *opaque;
TEST_SAY(_C_MAG "[ %s CreateTopics with %s, timeout %dms ]\n",
rd_kafka_name(rk), what, exp_timeout);
/**
* Construct NewTopic array with different properties for
* different partitions.
*/
for (i = 0 ; i < MY_NEW_TOPICS_CNT ; i++) {
const char *topic = test_mk_topic_name(__FUNCTION__, 1);
int num_parts = i * 51 + 1;
int num_replicas = jitter(1, MY_NEW_TOPICS_CNT-1);
int set_config = (i & 2);
int set_replicas = !(i % 1);
new_topics[i] = rd_kafka_NewTopic_new(topic,
num_parts,
set_replicas ? -1 :
num_replicas,
NULL, 0);
if (set_config) {
/*
* Add various (unverified) configuration properties
*/
err = rd_kafka_NewTopic_set_config(new_topics[i],
"dummy.doesntexist",
"butThere'sNothing "
"to verify that");
TEST_ASSERT(!err, "%s", rd_kafka_err2str(err));
err = rd_kafka_NewTopic_set_config(new_topics[i],
"try.a.null.value",
NULL);
TEST_ASSERT(!err, "%s", rd_kafka_err2str(err));
err = rd_kafka_NewTopic_set_config(new_topics[i],
"or.empty", "");
TEST_ASSERT(!err, "%s", rd_kafka_err2str(err));
}
if (set_replicas) {
int32_t p;
int32_t replicas[MY_NEW_TOPICS_CNT];
int j;
for (j = 0 ; j < num_replicas ; j++)
replicas[j] = j;
/*
* Set valid replica assignments
*/
for (p = 0 ; p < num_parts ; p++) {
/* Try adding an existing out of order,
* should fail */
if (p == 1) {
err = rd_kafka_NewTopic_set_replica_assignment(
new_topics[i], p+1,
replicas, num_replicas,
errstr, sizeof(errstr));
TEST_ASSERT(err == RD_KAFKA_RESP_ERR__INVALID_ARG,
"%s", rd_kafka_err2str(err));
}
err = rd_kafka_NewTopic_set_replica_assignment(
new_topics[i], p,
replicas, num_replicas,
errstr, sizeof(errstr));
TEST_ASSERT(!err, "%s", errstr);
}
/* Try to add an existing partition, should fail */
err = rd_kafka_NewTopic_set_replica_assignment(
new_topics[i], 0,
replicas, num_replicas, NULL, 0);
TEST_ASSERT(err == RD_KAFKA_RESP_ERR__INVALID_ARG,
"%s", rd_kafka_err2str(err));
} else {
int32_t dummy_replicas[1] = {1};
/* Test invalid partition */
err = rd_kafka_NewTopic_set_replica_assignment(
new_topics[i], num_parts+1, dummy_replicas, 1,
errstr, sizeof(errstr));
TEST_ASSERT(err == RD_KAFKA_RESP_ERR__INVALID_ARG,
"%s: %s", rd_kafka_err2str(err),
err == RD_KAFKA_RESP_ERR_NO_ERROR ?
"" : errstr);
/* Setting replicas with with default replicas != -1
* is an error. */
err = rd_kafka_NewTopic_set_replica_assignment(
new_topics[i], 0, dummy_replicas, 1,
errstr, sizeof(errstr));
TEST_ASSERT(err == RD_KAFKA_RESP_ERR__INVALID_ARG,
"%s: %s", rd_kafka_err2str(err),
err == RD_KAFKA_RESP_ERR_NO_ERROR ?
"" : errstr);
}
}
if (with_options) {
options = rd_kafka_AdminOptions_new(rk, RD_KAFKA_ADMIN_OP_ANY);
exp_timeout = MY_SOCKET_TIMEOUT_MS * 2;
err = rd_kafka_AdminOptions_set_request_timeout(
options, exp_timeout, errstr, sizeof(errstr));
TEST_ASSERT(!err, "%s", rd_kafka_err2str(err));
my_opaque = (void *)123;
rd_kafka_AdminOptions_set_opaque(options, my_opaque);
}
TIMING_START(&timing, "CreateTopics");
TEST_SAY("Call CreateTopics, timeout is %dms\n", exp_timeout);
rd_kafka_CreateTopics(rk, new_topics, MY_NEW_TOPICS_CNT,
options, q);
TIMING_ASSERT_LATER(&timing, 0, 50);
if (with_background_event_cb) {
/* Result event will be triggered by callback from
* librdkafka background queue thread. */
TIMING_START(&timing, "CreateTopics.wait_background_event_cb");
rkev = wait_background_event_cb();
} else {
/* Poll result queue */
TIMING_START(&timing, "CreateTopics.queue_poll");
rkev = rd_kafka_queue_poll(q, exp_timeout + 1000);
}
TIMING_ASSERT_LATER(&timing, exp_timeout-100, exp_timeout+100);
TEST_ASSERT(rkev != NULL, "expected result in %dms",
exp_timeout);
TEST_SAY("CreateTopics: got %s in %.3fs\n",
rd_kafka_event_name(rkev),
TIMING_DURATION(&timing) / 1000.0f);
/* Convert event to proper result */
res = rd_kafka_event_CreateTopics_result(rkev);
TEST_ASSERT(res, "expected CreateTopics_result, not %s",
rd_kafka_event_name(rkev));
opaque = rd_kafka_event_opaque(rkev);
TEST_ASSERT(opaque == my_opaque, "expected opaque to be %p, not %p",
my_opaque, opaque);
/* Expecting error */
err = rd_kafka_event_error(rkev);
errstr2 = rd_kafka_event_error_string(rkev);
TEST_ASSERT(err == RD_KAFKA_RESP_ERR__TIMED_OUT,
"expected CreateTopics to return error %s, not %s (%s)",
rd_kafka_err2str(RD_KAFKA_RESP_ERR__TIMED_OUT),
rd_kafka_err2str(err),
err ? errstr2 : "n/a");
/* Attempt to extract topics anyway, should return NULL. */
restopics = rd_kafka_CreateTopics_result_topics(res, &restopic_cnt);
TEST_ASSERT(!restopics && restopic_cnt == 0,
"expected no result_topics, got %p cnt %"PRIusz,
restopics, restopic_cnt);
rd_kafka_event_destroy(rkev);
rd_kafka_NewTopic_destroy_array(new_topics, MY_NEW_TOPICS_CNT);
if (options)
rd_kafka_AdminOptions_destroy(options);
if (!useq)
rd_kafka_queue_destroy(q);
}
/**
* @brief DeleteTopics tests
*
*
*
*/
static void do_test_DeleteTopics (const char *what,
rd_kafka_t *rk, rd_kafka_queue_t *useq,
int with_options) {
rd_kafka_queue_t *q = useq ? useq : rd_kafka_queue_new(rk);
#define MY_DEL_TOPICS_CNT 4
rd_kafka_DeleteTopic_t *del_topics[MY_DEL_TOPICS_CNT];
rd_kafka_AdminOptions_t *options = NULL;
int exp_timeout = MY_SOCKET_TIMEOUT_MS;
int i;
char errstr[512];
const char *errstr2;
rd_kafka_resp_err_t err;
test_timing_t timing;
rd_kafka_event_t *rkev;
const rd_kafka_DeleteTopics_result_t *res;
const rd_kafka_topic_result_t **restopics;
size_t restopic_cnt;
void *my_opaque = NULL, *opaque;
TEST_SAY(_C_MAG "[ %s DeleteTopics with %s, timeout %dms ]\n",
rd_kafka_name(rk), what, exp_timeout);
for (i = 0 ; i < MY_DEL_TOPICS_CNT ; i++)
del_topics[i] = rd_kafka_DeleteTopic_new(test_mk_topic_name(__FUNCTION__, 1));
if (with_options) {
options = rd_kafka_AdminOptions_new(
rk, RD_KAFKA_ADMIN_OP_DELETETOPICS);
exp_timeout = MY_SOCKET_TIMEOUT_MS * 2;
err = rd_kafka_AdminOptions_set_request_timeout(
options, exp_timeout, errstr, sizeof(errstr));
TEST_ASSERT(!err, "%s", rd_kafka_err2str(err));
if (useq) {
my_opaque = (void *)456;
rd_kafka_AdminOptions_set_opaque(options, my_opaque);
}
}
TIMING_START(&timing, "DeleteTopics");
TEST_SAY("Call DeleteTopics, timeout is %dms\n", exp_timeout);
rd_kafka_DeleteTopics(rk, del_topics, MY_DEL_TOPICS_CNT,
options, q);
TIMING_ASSERT_LATER(&timing, 0, 50);
/* Poll result queue */
TIMING_START(&timing, "DeleteTopics.queue_poll");
rkev = rd_kafka_queue_poll(q, exp_timeout + 1000);
TIMING_ASSERT_LATER(&timing, exp_timeout-100, exp_timeout+100);
TEST_ASSERT(rkev != NULL, "expected result in %dms", exp_timeout);
TEST_SAY("DeleteTopics: got %s in %.3fs\n",
rd_kafka_event_name(rkev), TIMING_DURATION(&timing) / 1000.0f);
/* Convert event to proper result */
res = rd_kafka_event_DeleteTopics_result(rkev);
TEST_ASSERT(res, "expected DeleteTopics_result, not %s",
rd_kafka_event_name(rkev));
opaque = rd_kafka_event_opaque(rkev);
TEST_ASSERT(opaque == my_opaque, "expected opaque to be %p, not %p",
my_opaque, opaque);
/* Expecting error */
err = rd_kafka_event_error(rkev);
errstr2 = rd_kafka_event_error_string(rkev);
TEST_ASSERT(err == RD_KAFKA_RESP_ERR__TIMED_OUT,
"expected DeleteTopics to return error %s, not %s (%s)",
rd_kafka_err2str(RD_KAFKA_RESP_ERR__TIMED_OUT),
rd_kafka_err2str(err),
err ? errstr2 : "n/a");
/* Attempt to extract topics anyway, should return NULL. */
restopics = rd_kafka_DeleteTopics_result_topics(res, &restopic_cnt);
TEST_ASSERT(!restopics && restopic_cnt == 0,
"expected no result_topics, got %p cnt %"PRIusz,
restopics, restopic_cnt);
rd_kafka_event_destroy(rkev);
rd_kafka_DeleteTopic_destroy_array(del_topics, MY_DEL_TOPICS_CNT);
if (options)
rd_kafka_AdminOptions_destroy(options);
if (!useq)
rd_kafka_queue_destroy(q);
}
/**
* @brief Test a mix of APIs using the same replyq.
*
* - Create topics A,B
* - Delete topic B
* - Create topic C
* - Create extra partitions for topic D
*/
static void do_test_mix (rd_kafka_t *rk, rd_kafka_queue_t *rkqu) {
char *topics[] = { "topicA", "topicB", "topicC" };
int cnt = 0;
struct waiting {
rd_kafka_event_type_t evtype;
int seen;
};
struct waiting id1 = {RD_KAFKA_EVENT_CREATETOPICS_RESULT};
struct waiting id2 = {RD_KAFKA_EVENT_DELETETOPICS_RESULT};
struct waiting id3 = {RD_KAFKA_EVENT_CREATETOPICS_RESULT};
struct waiting id4 = {RD_KAFKA_EVENT_CREATEPARTITIONS_RESULT};
TEST_SAY(_C_MAG "[ Mixed mode test on %s]\n", rd_kafka_name(rk));
test_CreateTopics_simple(rk, rkqu, topics, 2, 1, &id1);
test_DeleteTopics_simple(rk, rkqu, &topics[1], 1, &id2);
test_CreateTopics_simple(rk, rkqu, &topics[2], 1, 1, &id3);
test_CreatePartitions_simple(rk, rkqu, "topicD", 15, &id4);
while (cnt < 4) {
rd_kafka_event_t *rkev;
struct waiting *w;
rkev = rd_kafka_queue_poll(rkqu, -1);
TEST_ASSERT(rkev);
TEST_SAY("Got event %s: %s\n",
rd_kafka_event_name(rkev),
rd_kafka_event_error_string(rkev));
w = rd_kafka_event_opaque(rkev);
TEST_ASSERT(w);
TEST_ASSERT(w->evtype == rd_kafka_event_type(rkev),
"Expected evtype %d, not %d (%s)",
w->evtype, rd_kafka_event_type(rkev),
rd_kafka_event_name(rkev));
TEST_ASSERT(w->seen == 0, "Duplicate results");
w->seen++;
cnt++;
rd_kafka_event_destroy(rkev);
}
}
/**
* @brief Test AlterConfigs and DescribeConfigs
*/
static void do_test_configs (rd_kafka_t *rk, rd_kafka_queue_t *rkqu) {
#define MY_CONFRES_CNT RD_KAFKA_RESOURCE__CNT + 2
rd_kafka_ConfigResource_t *configs[MY_CONFRES_CNT];
rd_kafka_AdminOptions_t *options;
rd_kafka_event_t *rkev;
rd_kafka_resp_err_t err;
const rd_kafka_AlterConfigs_result_t *res;
const rd_kafka_ConfigResource_t **rconfigs;
size_t rconfig_cnt;
char errstr[128];
int i;
/* Check invalids */
configs[0] = rd_kafka_ConfigResource_new(
(rd_kafka_ResourceType_t)-1, "something");
TEST_ASSERT(!configs[0]);
configs[0] = rd_kafka_ConfigResource_new(
(rd_kafka_ResourceType_t)0, NULL);
TEST_ASSERT(!configs[0]);
for (i = 0 ; i < MY_CONFRES_CNT ; i++) {
int set_config = !(i % 2);
/* librdkafka shall not limit the use of illogical
* or unknown settings, they are enforced by the broker. */
configs[i] = rd_kafka_ConfigResource_new(
(rd_kafka_ResourceType_t)i, "3");
TEST_ASSERT(configs[i] != NULL);
if (set_config) {
rd_kafka_ConfigResource_set_config(configs[i],
"some.conf",
"which remains "
"unchecked");
rd_kafka_ConfigResource_set_config(configs[i],
"some.conf.null",
NULL);
}
}
options = rd_kafka_AdminOptions_new(rk, RD_KAFKA_ADMIN_OP_ANY);
err = rd_kafka_AdminOptions_set_request_timeout(options, 1000, errstr,
sizeof(errstr));
TEST_ASSERT(!err, "%s", errstr);
/* AlterConfigs */
rd_kafka_AlterConfigs(rk, configs, MY_CONFRES_CNT,
options, rkqu);
rkev = test_wait_admin_result(rkqu, RD_KAFKA_EVENT_ALTERCONFIGS_RESULT,
2000);
TEST_ASSERT(rd_kafka_event_error(rkev) == RD_KAFKA_RESP_ERR__TIMED_OUT,
"Expected timeout, not %s",
rd_kafka_event_error_string(rkev));
res = rd_kafka_event_AlterConfigs_result(rkev);
TEST_ASSERT(res);
rconfigs = rd_kafka_AlterConfigs_result_resources(res, &rconfig_cnt);
TEST_ASSERT(!rconfigs && !rconfig_cnt,
"Expected no result resources, got %"PRIusz,
rconfig_cnt);
rd_kafka_event_destroy(rkev);
/* DescribeConfigs: reuse same configs and options */
rd_kafka_DescribeConfigs(rk, configs, MY_CONFRES_CNT,
options, rkqu);
rd_kafka_AdminOptions_destroy(options);
rd_kafka_ConfigResource_destroy_array(configs, MY_CONFRES_CNT);
rkev = test_wait_admin_result(rkqu,
RD_KAFKA_EVENT_DESCRIBECONFIGS_RESULT,
2000);
TEST_ASSERT(rd_kafka_event_error(rkev) == RD_KAFKA_RESP_ERR__TIMED_OUT,
"Expected timeout, not %s",
rd_kafka_event_error_string(rkev));
res = rd_kafka_event_DescribeConfigs_result(rkev);
TEST_ASSERT(res);
rconfigs = rd_kafka_DescribeConfigs_result_resources(res, &rconfig_cnt);
TEST_ASSERT(!rconfigs && !rconfig_cnt,
"Expected no result resources, got %"PRIusz,
rconfig_cnt);
rd_kafka_event_destroy(rkev);
}
/**
* @brief Verify that an unclean rd_kafka_destroy() does not hang.
*/
static void do_test_unclean_destroy (rd_kafka_type_t cltype, int with_mainq) {
rd_kafka_t *rk;
char errstr[512];
rd_kafka_conf_t *conf;
rd_kafka_queue_t *q;
rd_kafka_event_t *rkev;
rd_kafka_DeleteTopic_t *topic;
test_timing_t t_destroy;
test_conf_init(&conf, NULL, 0);
/* Remove brokers, if any, since this is a local test and we
* rely on the controller not being found. */
test_conf_set(conf, "bootstrap.servers", "");
test_conf_set(conf, "socket.timeout.ms", "60000");
rk = rd_kafka_new(cltype, conf, errstr, sizeof(errstr));
TEST_ASSERT(rk, "kafka_new(%d): %s", cltype, errstr);
TEST_SAY(_C_MAG "[ Test unclean destroy for %s using %s]\n", rd_kafka_name(rk),
with_mainq ? "mainq" : "tempq");
if (with_mainq)
q = rd_kafka_queue_get_main(rk);
else
q = rd_kafka_queue_new(rk);
topic = rd_kafka_DeleteTopic_new("test");
rd_kafka_DeleteTopics(rk, &topic, 1, NULL, q);
rd_kafka_DeleteTopic_destroy(topic);
/* We're not expecting a result yet since DeleteTopics will attempt
* to look up the controller for socket.timeout.ms (1 minute). */
rkev = rd_kafka_queue_poll(q, 100);
TEST_ASSERT(!rkev, "Did not expect result: %s", rd_kafka_event_name(rkev));
rd_kafka_queue_destroy(q);
TEST_SAY("Giving rd_kafka_destroy() 5s to finish, "
"despite Admin API request being processed\n");
test_timeout_set(5);
TIMING_START(&t_destroy, "rd_kafka_destroy()");
rd_kafka_destroy(rk);
TIMING_STOP(&t_destroy);
/* Restore timeout */
test_timeout_set(60);
}
/**
* @brief Test AdminOptions
*/
static void do_test_options (rd_kafka_t *rk) {
#define _all_apis { RD_KAFKA_ADMIN_OP_CREATETOPICS, \
RD_KAFKA_ADMIN_OP_DELETETOPICS, \
RD_KAFKA_ADMIN_OP_CREATEPARTITIONS, \
RD_KAFKA_ADMIN_OP_ALTERCONFIGS, \
RD_KAFKA_ADMIN_OP_DESCRIBECONFIGS, \
RD_KAFKA_ADMIN_OP_ANY /* Must be last */}
struct {
const char *setter;
const rd_kafka_admin_op_t valid_apis[8];
} matrix[] = {
{ "request_timeout", _all_apis },
{ "operation_timeout", { RD_KAFKA_ADMIN_OP_CREATETOPICS,
RD_KAFKA_ADMIN_OP_DELETETOPICS,
RD_KAFKA_ADMIN_OP_CREATEPARTITIONS } },
{ "validate_only", { RD_KAFKA_ADMIN_OP_CREATETOPICS,
RD_KAFKA_ADMIN_OP_CREATEPARTITIONS,
RD_KAFKA_ADMIN_OP_ALTERCONFIGS } },
{ "broker", _all_apis },
{ "opaque", _all_apis },
{ NULL },
};
int i;
rd_kafka_AdminOptions_t *options;
for (i = 0 ; matrix[i].setter ; i++) {
static const rd_kafka_admin_op_t all_apis[] = _all_apis;
const rd_kafka_admin_op_t *for_api;
for (for_api = all_apis ; ; for_api++) {
rd_kafka_resp_err_t err = RD_KAFKA_RESP_ERR_NO_ERROR;
rd_kafka_resp_err_t exp_err = RD_KAFKA_RESP_ERR_NO_ERROR;
char errstr[512];
int fi;
options = rd_kafka_AdminOptions_new(rk, *for_api);
TEST_ASSERT(options,
"AdminOptions_new(%d) failed", *for_api);
if (!strcmp(matrix[i].setter, "request_timeout"))
err = rd_kafka_AdminOptions_set_request_timeout(
options, 1234, errstr, sizeof(errstr));
else if (!strcmp(matrix[i].setter, "operation_timeout"))
err = rd_kafka_AdminOptions_set_operation_timeout(
options, 12345, errstr, sizeof(errstr));
else if (!strcmp(matrix[i].setter, "validate_only"))
err = rd_kafka_AdminOptions_set_validate_only(
options, 1, errstr, sizeof(errstr));
else if (!strcmp(matrix[i].setter, "broker"))
err = rd_kafka_AdminOptions_set_broker(
options, 5, errstr, sizeof(errstr));
else if (!strcmp(matrix[i].setter, "opaque")) {
rd_kafka_AdminOptions_set_opaque(
options, (void *)options);
err = RD_KAFKA_RESP_ERR_NO_ERROR;
} else
TEST_FAIL("Invalid setter: %s",
matrix[i].setter);
TEST_SAYL(3, "AdminOptions_set_%s on "
"RD_KAFKA_ADMIN_OP_%d options "
"returned %s: %s\n",
matrix[i].setter,
*for_api,
rd_kafka_err2name(err),
err ? errstr : "success");
/* Scan matrix valid_apis to see if this
* setter should be accepted or not. */
if (exp_err) {
/* An expected error is already set */
} else if (*for_api != RD_KAFKA_ADMIN_OP_ANY) {
exp_err = RD_KAFKA_RESP_ERR__INVALID_ARG;
for (fi = 0 ; matrix[i].valid_apis[fi] ; fi++) {
if (matrix[i].valid_apis[fi] ==
*for_api)
exp_err = RD_KAFKA_RESP_ERR_NO_ERROR;
}
} else {
exp_err = RD_KAFKA_RESP_ERR_NO_ERROR;
}
if (err != exp_err)
TEST_FAIL_LATER("Expected AdminOptions_set_%s "
"for RD_KAFKA_ADMIN_OP_%d "
"options to return %s, "
"not %s",
matrix[i].setter,
*for_api,
rd_kafka_err2name(exp_err),
rd_kafka_err2name(err));
rd_kafka_AdminOptions_destroy(options);
if (*for_api == RD_KAFKA_ADMIN_OP_ANY)
break; /* This was the last one */
}
}
/* Try an invalid for_api */
options = rd_kafka_AdminOptions_new(rk, (rd_kafka_admin_op_t)1234);
TEST_ASSERT(!options, "Expectred AdminOptions_new() to fail "
"with an invalid for_api, didn't.");
TEST_LATER_CHECK();
}
static void do_test_apis (rd_kafka_type_t cltype) {
rd_kafka_t *rk;
char errstr[512];
rd_kafka_queue_t *mainq, *backgroundq;
rd_kafka_conf_t *conf;
mtx_init(&last_event_lock, mtx_plain);
cnd_init(&last_event_cnd);
do_test_unclean_destroy(cltype, 0/*tempq*/);
do_test_unclean_destroy(cltype, 1/*mainq*/);
test_conf_init(&conf, NULL, 0);
/* Remove brokers, if any, since this is a local test and we
* rely on the controller not being found. */
test_conf_set(conf, "bootstrap.servers", "");
test_conf_set(conf, "socket.timeout.ms", MY_SOCKET_TIMEOUT_MS_STR);
/* For use with the background queue */
rd_kafka_conf_set_background_event_cb(conf, background_event_cb);
rk = rd_kafka_new(cltype, conf, errstr, sizeof(errstr));
TEST_ASSERT(rk, "kafka_new(%d): %s", cltype, errstr);
mainq = rd_kafka_queue_get_main(rk);
backgroundq = rd_kafka_queue_get_background(rk);
do_test_options(rk);
do_test_CreateTopics("temp queue, no options", rk, NULL, 0, 0);
do_test_CreateTopics("temp queue, no options, background_event_cb",
rk, backgroundq, 1, 0);
do_test_CreateTopics("temp queue, options", rk, NULL, 0, 1);
do_test_CreateTopics("main queue, options", rk, mainq, 0, 1);
do_test_DeleteTopics("temp queue, no options", rk, NULL, 0);
do_test_DeleteTopics("temp queue, options", rk, NULL, 1);
do_test_DeleteTopics("main queue, options", rk, mainq, 1);
do_test_mix(rk, mainq);
do_test_configs(rk, mainq);
rd_kafka_queue_destroy(backgroundq);
rd_kafka_queue_destroy(mainq);
rd_kafka_destroy(rk);
mtx_destroy(&last_event_lock);
cnd_destroy(&last_event_cnd);
}
int main_0080_admin_ut (int argc, char **argv) {
do_test_apis(RD_KAFKA_PRODUCER);
do_test_apis(RD_KAFKA_CONSUMER);
return 0;
}