forked from confluentinc/librdkafka
-
Notifications
You must be signed in to change notification settings - Fork 5
/
Copy pathrdkafka_admin.c
2737 lines (2195 loc) · 98.4 KB
/
rdkafka_admin.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
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
/*
* librdkafka - Apache Kafka C library
*
* Copyright (c) 2018 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 "rdkafka_int.h"
#include "rdkafka_admin.h"
#include "rdkafka_request.h"
#include "rdkafka_aux.h"
#include <stdarg.h>
/** @brief Descriptive strings for rko_u.admin_request.state */
static const char *rd_kafka_admin_state_desc[] = {
"initializing",
"waiting for broker",
"waiting for controller",
"constructing request",
"waiting for response from broker",
};
/**
* @brief Admin API implementation.
*
* The public Admin API in librdkafka exposes a completely asynchronous
* interface where the initial request API (e.g., ..CreateTopics())
* is non-blocking and returns immediately, and the application polls
* a ..queue_t for the result.
*
* The underlying handling of the request is also completely asynchronous
* inside librdkafka, for two reasons:
* - everything is async in librdkafka so adding something new that isn't
* would mean that existing functionality will need to be changed if
* it should be able to work simultaneously (such as statistics, timers,
* etc). There is no functional value to making the admin API
* synchronous internally, even if it would simplify its implementation.
* So making it async allows the Admin API to be used with existing
* client types in existing applications without breakage.
* - the async approach allows multiple outstanding Admin API requests
* simultaneously.
*
* The internal async implementation relies on the following concepts:
* - it uses a single rko (rd_kafka_op_t) to maintain state.
* - the rko has a callback attached - called the worker callback.
* - the worker callback is a small state machine that triggers
* async operations (be it controller lookups, timeout timers,
* protocol transmits, etc).
* - the worker callback is only called on the rdkafka main thread.
* - the callback is triggered by different events and sources by enqueuing
* the rko on the rdkafka main ops queue.
*
*
* Let's illustrate this with a DeleteTopics example. This might look
* daunting, but it boils down to an asynchronous state machine being
* triggered by enqueuing the rko op.
*
* 1. [app thread] The user constructs the input arguments,
* including a response rkqu queue and then calls DeleteTopics().
*
* 2. [app thread] DeleteTopics() creates a new internal op (rko) of type
* RD_KAFKA_OP_DELETETOPICS, makes a **copy** on the rko of all the
* input arguments (which allows the caller to free the originals
* whenever she likes). The rko op worker callback is set to the
* generic admin worker callback rd_kafka_admin_worker()
*
* 3. [app thread] DeleteTopics() enqueues the rko on librdkafka's main ops
* queue that is served by the rdkafka main thread in rd_kafka_thread_main()
*
* 4. [rdkafka main thread] The rko is dequeued by rd_kafka_q_serve and
* the rd_kafka_poll_cb() is called.
*
* 5. [rdkafka main thread] The rko_type switch case identifies the rko
* as an RD_KAFKA_OP_DELETETOPICS which is served by the op callback
* set in step 2.
*
* 6. [rdkafka main thread] The worker callback is called.
* After some initial checking of err==ERR__DESTROY events
* (which is used to clean up outstanding ops (etc) on termination),
* the code hits a state machine using rko_u.admin.request_state.
*
* 7. [rdkafka main thread] The initial state is RD_KAFKA_ADMIN_STATE_INIT
* where the worker validates the user input.
* An enqueue once (eonce) object is created - the use of this object
* allows having multiple outstanding async functions referencing the
* same underlying rko object, but only allowing the first one
* to trigger an event.
* A timeout timer is set up to trigger the eonce object when the
* full options.request_timeout has elapsed.
*
* 8. [rdkafka main thread] After initialization the state is updated
* to WAIT_BROKER or WAIT_CONTROLLER and the code falls through to
* looking up a specific broker or the controller broker and waiting for
* an active connection.
* Both the lookup and the waiting for an active connection are
* fully asynchronous, and the same eonce used for the timer is passed
* to the rd_kafka_broker_controller_async() or broker_async() functions
* which will trigger the eonce when a broker state change occurs.
* If the controller is already known (from metadata) and the connection
* is up a rkb broker object is returned and the eonce is not used,
* skip to step 11.
*
* 9. [rdkafka main thread] Upon metadata retrieval (which is triggered
* automatically by other parts of the code) the controller_id may be
* updated in which case the eonce is triggered.
* The eonce triggering enqueues the original rko on the rdkafka main
* ops queue again and we go to step 8 which will check if the controller
* connection is up.
*
* 10. [broker thread] If the controller_id is now known we wait for
* the corresponding broker's connection to come up. This signaling
* is performed from the broker thread upon broker state changes
* and uses the same eonce. The eonce triggering enqueues the original
* rko on the rdkafka main ops queue again we go to back to step 8
* to check if broker is now available.
*
* 11. [rdkafka main thread] Back in the worker callback we now have an
* rkb broker pointer (with reference count increased) for the controller
* with the connection up (it might go down while we're referencing it,
* but that does not stop us from enqueuing a protocol request).
*
* 12. [rdkafka main thread] A DeleteTopics protocol request buffer is
* constructed using the input parameters saved on the rko and the
* buffer is enqueued on the broker's transmit queue.
* The buffer is set up to provide the reply buffer on the rdkafka main
* ops queue (the same queue we are operating from) with a handler
* callback of rd_kafka_admin_handle_response().
* The state is updated to the RD_KAFKA_ADMIN_STATE_WAIT_RESPONSE.
*
* 13. [broker thread] If the request times out, a response with error code
* (ERR__TIMED_OUT) is enqueued. Go to 16.
*
* 14. [broker thread] If a response is received, the response buffer
* is enqueued. Go to 16.
*
* 15. [rdkafka main thread] The buffer callback (..handle_response())
* is called, which attempts to extra the original rko from the eonce,
* but if the eonce has already been triggered by some other source
* (the timeout timer) the buffer callback simply returns and does nothing
* since the admin request is over and a result (probably a timeout)
* has been enqueued for the application.
* If the rko was still intact we temporarily set the reply buffer
* in the rko struct and call the worker callback. Go to 17.
*
* 16. [rdkafka main thread] The worker callback is called in state
* RD_KAFKA_ADMIN_STATE_WAIT_RESPONSE without a response but with an error.
* An error result op is created and enqueued on the application's
* provided response rkqu queue.
*
* 17. [rdkafka main thread] The worker callback is called in state
* RD_KAFKA_ADMIN_STATE_WAIT_RESPONSE with a response buffer with no
* error set.
* The worker calls the response `parse()` callback to parse the response
* buffer and populates a result op (rko_result) with the response
* information (such as per-topic error codes, etc).
* The result op is returned to the worker.
*
* 18. [rdkafka main thread] The worker enqueues the result up (rko_result)
* on the application's provided response rkqu queue.
*
* 19. [app thread] The application calls rd_kafka_queue_poll() to
* receive the result of the operation. The result may have been
* enqueued in step 18 thanks to succesful completion, or in any
* of the earlier stages when an error was encountered.
*
* 20. [app thread] The application uses rd_kafka_event_DeleteTopics_result()
* to retrieve the request-specific result type.
*
* 21. Done.
*
*/
/**
* @brief Admin op callback types
*/
typedef rd_kafka_resp_err_t (rd_kafka_admin_Request_cb_t) (
rd_kafka_broker_t *rkb,
const rd_list_t *configs /*(ConfigResource_t*)*/,
rd_kafka_AdminOptions_t *options,
char *errstr, size_t errstr_size,
rd_kafka_replyq_t replyq,
rd_kafka_resp_cb_t *resp_cb,
void *opaque)
RD_WARN_UNUSED_RESULT;
typedef rd_kafka_resp_err_t (rd_kafka_admin_Response_parse_cb_t) (
rd_kafka_op_t *rko_req,
rd_kafka_op_t **rko_resultp,
rd_kafka_buf_t *reply,
char *errstr, size_t errstr_size)
RD_WARN_UNUSED_RESULT;
/**
* @struct Request-specific worker callbacks.
*/
struct rd_kafka_admin_worker_cbs {
/**< Protocol request callback which is called
* to construct and send the request. */
rd_kafka_admin_Request_cb_t *request;
/**< Protocol response parser callback which is called
* to translate the response to a rko_result op. */
rd_kafka_admin_Response_parse_cb_t *parse;
};
/* Forward declarations */
static void rd_kafka_AdminOptions_init (rd_kafka_t *rk,
rd_kafka_AdminOptions_t *options);
static rd_kafka_op_res_t
rd_kafka_admin_worker (rd_kafka_t *rk, rd_kafka_q_t *rkq, rd_kafka_op_t *rko);
static rd_kafka_ConfigEntry_t *
rd_kafka_ConfigEntry_copy (const rd_kafka_ConfigEntry_t *src);
static void rd_kafka_ConfigEntry_free (void *ptr);
static void *rd_kafka_ConfigEntry_list_copy (const void *src, void *opaque);
/**
* @name Common admin request code
* @{
*
*
*/
/**
* @brief Create a new admin_result op based on the request op \p rko_req
*/
static rd_kafka_op_t *rd_kafka_admin_result_new (const rd_kafka_op_t *rko_req) {
rd_kafka_op_t *rko_result;
rko_result = rd_kafka_op_new(RD_KAFKA_OP_ADMIN_RESULT);
rko_result->rko_rk = rko_req->rko_rk;
rko_result->rko_u.admin_result.opaque =
rd_kafka_confval_get_ptr(&rko_req->rko_u.admin_request.
options.opaque);
rko_result->rko_u.admin_result.reqtype = rko_req->rko_type;
rko_result->rko_evtype = rko_req->rko_u.admin_request.reply_event_type;
return rko_result;
}
/**
* @brief Set error code and error string on admin_result op \p rko.
*/
static void rd_kafka_admin_result_set_err0 (rd_kafka_op_t *rko,
rd_kafka_resp_err_t err,
const char *fmt, va_list ap) {
char buf[512];
rd_vsnprintf(buf, sizeof(buf), fmt, ap);
rko->rko_err = err;
if (rko->rko_u.admin_result.errstr)
rd_free(rko->rko_u.admin_result.errstr);
rko->rko_u.admin_result.errstr = rd_strdup(buf);
rd_kafka_dbg(rko->rko_rk, ADMIN, "ADMINFAIL",
"Admin %s result error: %s",
rd_kafka_op2str(rko->rko_u.admin_result.reqtype),
rko->rko_u.admin_result.errstr);
}
/**
* @sa rd_kafka_admin_result_set_err0
*/
static RD_UNUSED void rd_kafka_admin_result_set_err (rd_kafka_op_t *rko,
rd_kafka_resp_err_t err,
const char *fmt, ...) {
va_list ap;
va_start(ap, fmt);
rd_kafka_admin_result_set_err0(rko, err, fmt, ap);
va_end(ap);
}
/**
* @brief Enqueue admin_result on application's queue.
*/
static RD_INLINE
void rd_kafka_admin_result_enq (rd_kafka_op_t *rko_req,
rd_kafka_op_t *rko_result) {
rd_kafka_replyq_enq(&rko_req->rko_u.admin_request.replyq, rko_result,
rko_req->rko_u.admin_request.replyq.version);
}
/**
* @brief Set request-level error code and string in reply op.
*/
static void rd_kafka_admin_result_fail (rd_kafka_op_t *rko_req,
rd_kafka_resp_err_t err,
const char *fmt, ...) {
va_list ap;
rd_kafka_op_t *rko_result;
rko_result = rd_kafka_admin_result_new(rko_req);
va_start(ap, fmt);
rd_kafka_admin_result_set_err0(rko_result, err, fmt, ap);
va_end(ap);
rd_kafka_admin_result_enq(rko_req, rko_result);
}
/**
* @brief Return the topics list from a topic-related result object.
*/
static const rd_kafka_topic_result_t **
rd_kafka_admin_result_ret_topics (const rd_kafka_op_t *rko,
size_t *cntp) {
rd_kafka_op_type_t reqtype =
rko->rko_u.admin_result.reqtype & ~RD_KAFKA_OP_FLAGMASK;
rd_assert(reqtype == RD_KAFKA_OP_CREATETOPICS ||
reqtype == RD_KAFKA_OP_DELETETOPICS ||
reqtype == RD_KAFKA_OP_CREATEPARTITIONS);
*cntp = rd_list_cnt(&rko->rko_u.admin_result.results);
return (const rd_kafka_topic_result_t **)rko->rko_u.admin_result.
results.rl_elems;
}
/**
* @brief Return the ConfigResource list from a config-related result object.
*/
static const rd_kafka_ConfigResource_t **
rd_kafka_admin_result_ret_resources (const rd_kafka_op_t *rko,
size_t *cntp) {
rd_kafka_op_type_t reqtype =
rko->rko_u.admin_result.reqtype & ~RD_KAFKA_OP_FLAGMASK;
rd_assert(reqtype == RD_KAFKA_OP_ALTERCONFIGS ||
reqtype == RD_KAFKA_OP_DESCRIBECONFIGS);
*cntp = rd_list_cnt(&rko->rko_u.admin_result.results);
return (const rd_kafka_ConfigResource_t **)rko->rko_u.admin_result.
results.rl_elems;
}
/**
* @brief Create a new admin_request op of type \p optype and sets up the
* generic (type independent files).
*
* The caller shall then populate the admin_request.args list
* and enqueue the op on rk_ops for further processing work.
*
* @param cbs Callbacks, must reside in .data segment.
* @param options Optional options, may be NULL to use defaults.
*
* @locks none
* @locality application thread
*/
static rd_kafka_op_t *
rd_kafka_admin_request_op_new (rd_kafka_t *rk,
rd_kafka_op_type_t optype,
rd_kafka_event_type_t reply_event_type,
const struct rd_kafka_admin_worker_cbs *cbs,
const rd_kafka_AdminOptions_t *options,
rd_kafka_queue_t *rkqu) {
rd_kafka_op_t *rko;
rd_assert(rk);
rd_assert(rkqu);
rd_assert(cbs);
rko = rd_kafka_op_new_cb(rk, optype, rd_kafka_admin_worker);
rko->rko_u.admin_request.reply_event_type = reply_event_type;
rko->rko_u.admin_request.cbs = (struct rd_kafka_admin_worker_cbs *)cbs;
/* Make a copy of the options */
if (options)
rko->rko_u.admin_request.options = *options;
else
rd_kafka_AdminOptions_init(rk,
&rko->rko_u.admin_request.options);
/* Default to controller */
rko->rko_u.admin_request.broker_id = -1;
/* Calculate absolute timeout */
rko->rko_u.admin_request.abs_timeout =
rd_timeout_init(
rd_kafka_confval_get_int(&rko->rko_u.admin_request.
options.request_timeout));
/* Setup enq-op-once, which is triggered by either timer code
* or future wait-controller code. */
rko->rko_u.admin_request.eonce =
rd_kafka_enq_once_new(rko, RD_KAFKA_REPLYQ(rk->rk_ops, 0));
/* The timer itself must be started from the rdkafka main thread,
* not here. */
/* Set up replyq */
rd_kafka_set_replyq(&rko->rko_u.admin_request.replyq,
rkqu->rkqu_q, 0);
rko->rko_u.admin_request.state = RD_KAFKA_ADMIN_STATE_INIT;
return rko;
}
/**
* @brief Timer timeout callback for the admin rko's eonce object.
*/
static void rd_kafka_admin_eonce_timeout_cb (rd_kafka_timers_t *rkts,
void *arg) {
rd_kafka_enq_once_t *eonce = arg;
rd_kafka_enq_once_trigger(eonce, RD_KAFKA_RESP_ERR__TIMED_OUT,
"timer timeout");
}
/**
* @brief Common worker destroy to be called in destroy: label
* in worker.
*/
static void rd_kafka_admin_common_worker_destroy (rd_kafka_t *rk,
rd_kafka_op_t *rko) {
int timer_was_stopped;
/* Free resources for this op. */
timer_was_stopped =
rd_kafka_timer_stop(&rk->rk_timers,
&rko->rko_u.admin_request.tmr, rd_true);
if (rko->rko_u.admin_request.eonce) {
/* Remove the stopped timer's eonce reference since its
* callback will not have fired if we stopped the timer. */
if (timer_was_stopped)
rd_kafka_enq_once_del_source(rko->rko_u.admin_request.
eonce, "timeout timer");
/* This is thread-safe to do even if there are outstanding
* timers or wait-controller references to the eonce
* since they only hold direct reference to the eonce,
* not the rko (the eonce holds a reference to the rko but
* it is cleared here). */
rd_kafka_enq_once_destroy(rko->rko_u.admin_request.eonce);
rko->rko_u.admin_request.eonce = NULL;
}
}
/**
* @brief Asynchronously look up a broker.
* To be called repeatedly from each invocation of the worker
* when in state RD_KAFKA_ADMIN_STATE_WAIT_BROKER until
* a valid rkb is returned.
*
* @returns the broker rkb with refcount increased, or NULL if not yet
* available.
*/
static rd_kafka_broker_t *
rd_kafka_admin_common_get_broker (rd_kafka_t *rk,
rd_kafka_op_t *rko,
int32_t broker_id) {
rd_kafka_broker_t *rkb;
rd_kafka_dbg(rk, ADMIN, "ADMIN", "%s: looking up broker %"PRId32,
rd_kafka_op2str(rko->rko_type), broker_id);
/* Since we're iterating over this broker_async() call
* (asynchronously) until a broker is availabe (or timeout)
* we need to re-enable the eonce to be triggered again (which
* is not necessary the first time we get here, but there
* is no harm doing it then either). */
rd_kafka_enq_once_reenable(rko->rko_u.admin_request.eonce,
rko, RD_KAFKA_REPLYQ(rk->rk_ops, 0));
/* Look up the broker asynchronously, if the broker
* is not available the eonce is registered for broker
* state changes which will cause our function to be called
* again as soon as (any) broker state changes.
* When we are called again we perform the broker lookup
* again and hopefully get an rkb back, otherwise defer a new
* async wait. Repeat until success or timeout. */
if (!(rkb = rd_kafka_broker_get_async(
rk, broker_id, RD_KAFKA_BROKER_STATE_UP,
rko->rko_u.admin_request.eonce))) {
/* Broker not available, wait asynchronously
* for broker metadata code to trigger eonce. */
return NULL;
}
rd_kafka_dbg(rk, ADMIN, "ADMIN", "%s: broker %"PRId32" is %s",
rd_kafka_op2str(rko->rko_type), broker_id, rkb->rkb_name);
return rkb;
}
/**
* @brief Asynchronously look up the controller.
* To be called repeatedly from each invocation of the worker
* when in state RD_KAFKA_ADMIN_STATE_WAIT_CONTROLLER until
* a valid rkb is returned.
*
* @returns the controller rkb with refcount increased, or NULL if not yet
* available.
*/
static rd_kafka_broker_t *
rd_kafka_admin_common_get_controller (rd_kafka_t *rk,
rd_kafka_op_t *rko) {
rd_kafka_broker_t *rkb;
rd_kafka_dbg(rk, ADMIN, "ADMIN", "%s: looking up controller",
rd_kafka_op2str(rko->rko_type));
/* Since we're iterating over this controller_async() call
* (asynchronously) until a controller is availabe (or timeout)
* we need to re-enable the eonce to be triggered again (which
* is not necessary the first time we get here, but there
* is no harm doing it then either). */
rd_kafka_enq_once_reenable(rko->rko_u.admin_request.eonce,
rko, RD_KAFKA_REPLYQ(rk->rk_ops, 0));
/* Look up the controller asynchronously, if the controller
* is not available the eonce is registered for broker
* state changes which will cause our function to be called
* again as soon as (any) broker state changes.
* When we are called again we perform the controller lookup
* again and hopefully get an rkb back, otherwise defer a new
* async wait. Repeat until success or timeout. */
if (!(rkb = rd_kafka_broker_controller_async(
rk, RD_KAFKA_BROKER_STATE_UP,
rko->rko_u.admin_request.eonce))) {
/* Controller not available, wait asynchronously
* for controller code to trigger eonce. */
return NULL;
}
rd_kafka_dbg(rk, ADMIN, "ADMIN", "%s: controller %s",
rd_kafka_op2str(rko->rko_type), rkb->rkb_name);
return rkb;
}
/**
* @brief Handle response from broker by triggering worker callback.
*
* @param opaque is the eonce from the worker protocol request call.
*/
static void rd_kafka_admin_handle_response (rd_kafka_t *rk,
rd_kafka_broker_t *rkb,
rd_kafka_resp_err_t err,
rd_kafka_buf_t *reply,
rd_kafka_buf_t *request,
void *opaque) {
rd_kafka_enq_once_t *eonce = opaque;
rd_kafka_op_t *rko;
/* From ...add_source("send") */
rko = rd_kafka_enq_once_disable(eonce);
if (!rko) {
/* The operation timed out and the worker was
* dismantled while we were waiting for broker response,
* do nothing - everything has been cleaned up. */
rd_kafka_dbg(rk, ADMIN, "ADMIN",
"Dropping outdated %sResponse with return code %s",
request ?
rd_kafka_ApiKey2str(request->rkbuf_reqhdr.ApiKey):
"???",
rd_kafka_err2str(err));
return;
}
/* Attach reply buffer to rko for parsing in the worker. */
rd_assert(!rko->rko_u.admin_request.reply_buf);
rko->rko_u.admin_request.reply_buf = reply;
rko->rko_err = err;
if (rko->rko_op_cb(rk, NULL, rko) == RD_KAFKA_OP_RES_HANDLED)
rd_kafka_op_destroy(rko);
}
/**
* @brief Common worker state machine handling regardless of request type.
*
* Tasks:
* - Sets up timeout on first call.
* - Checks for timeout.
* - Checks for and fails on errors.
* - Async Controller and broker lookups
* - Calls the Request callback
* - Calls the parse callback
* - Result reply
* - Destruction of rko
*
* rko->rko_err may be one of:
* RD_KAFKA_RESP_ERR_NO_ERROR, or
* RD_KAFKA_RESP_ERR__DESTROY for queue destruction cleanup, or
* RD_KAFKA_RESP_ERR__TIMED_OUT if request has timed out,
* or any other error code triggered by other parts of the code.
*
* @returns a hint to the op code whether the rko should be destroyed or not.
*/
static rd_kafka_op_res_t
rd_kafka_admin_worker (rd_kafka_t *rk, rd_kafka_q_t *rkq, rd_kafka_op_t *rko) {
const char *name = rd_kafka_op2str(rko->rko_type);
rd_ts_t timeout_in;
rd_kafka_broker_t *rkb = NULL;
rd_kafka_resp_err_t err;
char errstr[512];
if (rd_kafka_terminating(rk)) {
rd_kafka_dbg(rk, ADMIN, name,
"%s worker called in state %s: "
"handle is terminating: %s",
name,
rd_kafka_admin_state_desc[rko->rko_u.
admin_request.state],
rd_kafka_err2str(rko->rko_err));
goto destroy;
}
if (rko->rko_err == RD_KAFKA_RESP_ERR__DESTROY)
goto destroy; /* rko being destroyed (silent) */
rd_kafka_dbg(rk, ADMIN, name,
"%s worker called in state %s: %s",
name,
rd_kafka_admin_state_desc[rko->rko_u.admin_request.state],
rd_kafka_err2str(rko->rko_err));
rd_assert(thrd_is_current(rko->rko_rk->rk_thread));
/* Check for errors raised asynchronously (e.g., by timer) */
if (rko->rko_err) {
rd_kafka_admin_result_fail(
rko, rko->rko_err,
"Failed while %s: %s",
rd_kafka_admin_state_desc[rko->rko_u.
admin_request.state],
rd_kafka_err2str(rko->rko_err));
goto destroy;
}
/* Check for timeout */
timeout_in = rd_timeout_remains_us(rko->rko_u.admin_request.
abs_timeout);
if (timeout_in <= 0) {
rd_kafka_admin_result_fail(
rko, RD_KAFKA_RESP_ERR__TIMED_OUT,
"Timed out %s",
rd_kafka_admin_state_desc[rko->rko_u.
admin_request.state]);
goto destroy;
}
redo:
switch (rko->rko_u.admin_request.state)
{
case RD_KAFKA_ADMIN_STATE_INIT:
{
int32_t broker_id;
/* First call. */
/* Set up timeout timer. */
rd_kafka_enq_once_add_source(rko->rko_u.admin_request.eonce,
"timeout timer");
rd_kafka_timer_start_oneshot(&rk->rk_timers,
&rko->rko_u.admin_request.tmr,
timeout_in,
rd_kafka_admin_eonce_timeout_cb,
rko->rko_u.admin_request.eonce);
/* Use explicitly specified broker_id, if available. */
broker_id = (int32_t)rd_kafka_confval_get_int(
&rko->rko_u.admin_request.options.broker);
if (broker_id != -1) {
rd_kafka_dbg(rk, ADMIN, name,
"%s using explicitly "
"set broker id %"PRId32
" rather than %"PRId32,
name, broker_id,
rko->rko_u.admin_request.broker_id);
rko->rko_u.admin_request.broker_id = broker_id;
}
/* Look up controller or specific broker. */
if (rko->rko_u.admin_request.broker_id != -1) {
/* Specific broker */
rko->rko_u.admin_request.state =
RD_KAFKA_ADMIN_STATE_WAIT_BROKER;
} else {
/* Controller */
rko->rko_u.admin_request.state =
RD_KAFKA_ADMIN_STATE_WAIT_CONTROLLER;
}
goto redo; /* Trigger next state immediately */
}
case RD_KAFKA_ADMIN_STATE_WAIT_BROKER:
/* Broker lookup */
if (!(rkb = rd_kafka_admin_common_get_broker(
rk, rko, rko->rko_u.admin_request.broker_id))) {
/* Still waiting for broker to become available */
return RD_KAFKA_OP_RES_KEEP;
}
rko->rko_u.admin_request.state =
RD_KAFKA_ADMIN_STATE_CONSTRUCT_REQUEST;
goto redo;
case RD_KAFKA_ADMIN_STATE_WAIT_CONTROLLER:
if (!(rkb = rd_kafka_admin_common_get_controller(rk, rko))) {
/* Still waiting for controller to become available. */
return RD_KAFKA_OP_RES_KEEP;
}
rko->rko_u.admin_request.state =
RD_KAFKA_ADMIN_STATE_CONSTRUCT_REQUEST;
goto redo;
case RD_KAFKA_ADMIN_STATE_CONSTRUCT_REQUEST:
/* Got broker, send protocol request. */
/* Make sure we're called from a 'goto redo' where
* the rkb was set. */
rd_assert(rkb);
/* Still need to use the eonce since this worker may
* time out while waiting for response from broker, in which
* case the broker response will hit an empty eonce (ok). */
rd_kafka_enq_once_add_source(rko->rko_u.admin_request.eonce,
"send");
/* Send request (async) */
err = rko->rko_u.admin_request.cbs->request(
rkb,
&rko->rko_u.admin_request.args,
&rko->rko_u.admin_request.options,
errstr, sizeof(errstr),
RD_KAFKA_REPLYQ(rk->rk_ops, 0),
rd_kafka_admin_handle_response,
rko->rko_u.admin_request.eonce);
/* Loose broker refcount from get_broker(), get_controller() */
rd_kafka_broker_destroy(rkb);
if (err) {
rd_kafka_enq_once_del_source(
rko->rko_u.admin_request.eonce, "send");
rd_kafka_admin_result_fail(rko, err, "%s", errstr);
goto destroy;
}
rko->rko_u.admin_request.state =
RD_KAFKA_ADMIN_STATE_WAIT_RESPONSE;
/* Wait asynchronously for broker response, which will
* trigger the eonce and worker to be called again. */
return RD_KAFKA_OP_RES_KEEP;
case RD_KAFKA_ADMIN_STATE_WAIT_RESPONSE:
{
rd_kafka_op_t *rko_result;
/* Response received.
* Parse response and populate result to application */
err = rko->rko_u.admin_request.cbs->parse(
rko, &rko_result,
rko->rko_u.admin_request.reply_buf,
errstr, sizeof(errstr));
if (err) {
rd_kafka_admin_result_fail(
rko, err,
"%s worker failed to parse response: %s",
name, errstr);
goto destroy;
}
/* Enqueue result on application queue, we're done. */
rd_kafka_admin_result_enq(rko, rko_result);
goto destroy;
}
}
return RD_KAFKA_OP_RES_KEEP;
destroy:
rd_kafka_admin_common_worker_destroy(rk, rko);
return RD_KAFKA_OP_RES_HANDLED; /* trigger's op_destroy() */
}
/**@}*/
/**
* @name Generic AdminOptions
* @{
*
*
*/
rd_kafka_resp_err_t
rd_kafka_AdminOptions_set_request_timeout (rd_kafka_AdminOptions_t *options,
int timeout_ms,
char *errstr, size_t errstr_size) {
return rd_kafka_confval_set_type(&options->request_timeout,
RD_KAFKA_CONFVAL_INT, &timeout_ms,
errstr, errstr_size);
}
rd_kafka_resp_err_t
rd_kafka_AdminOptions_set_operation_timeout (rd_kafka_AdminOptions_t *options,
int timeout_ms,
char *errstr, size_t errstr_size) {
return rd_kafka_confval_set_type(&options->operation_timeout,
RD_KAFKA_CONFVAL_INT, &timeout_ms,
errstr, errstr_size);
}
rd_kafka_resp_err_t
rd_kafka_AdminOptions_set_validate_only (rd_kafka_AdminOptions_t *options,
int true_or_false,
char *errstr, size_t errstr_size) {
return rd_kafka_confval_set_type(&options->validate_only,
RD_KAFKA_CONFVAL_INT, &true_or_false,
errstr, errstr_size);
}
rd_kafka_resp_err_t
rd_kafka_AdminOptions_set_incremental (rd_kafka_AdminOptions_t *options,
int true_or_false,
char *errstr, size_t errstr_size) {
rd_snprintf(errstr, errstr_size,
"Incremental updates currently not supported, see KIP-248");
return RD_KAFKA_RESP_ERR__NOT_IMPLEMENTED;
return rd_kafka_confval_set_type(&options->incremental,
RD_KAFKA_CONFVAL_INT, &true_or_false,
errstr, errstr_size);
}
rd_kafka_resp_err_t
rd_kafka_AdminOptions_set_broker (rd_kafka_AdminOptions_t *options,
int32_t broker_id,
char *errstr, size_t errstr_size) {
int ibroker_id = (int)broker_id;
return rd_kafka_confval_set_type(&options->broker,
RD_KAFKA_CONFVAL_INT,
&ibroker_id,
errstr, errstr_size);
}
void
rd_kafka_AdminOptions_set_opaque (rd_kafka_AdminOptions_t *options,
void *opaque) {
rd_kafka_confval_set_type(&options->opaque,
RD_KAFKA_CONFVAL_PTR, opaque, NULL, 0);
}
/**
* @brief Initialize and set up defaults for AdminOptions
*/
static void rd_kafka_AdminOptions_init (rd_kafka_t *rk,
rd_kafka_AdminOptions_t *options) {
rd_kafka_confval_init_int(&options->request_timeout, "request_timeout",
0, 3600*1000,
rk->rk_conf.admin.request_timeout_ms);
if (options->for_api == RD_KAFKA_ADMIN_OP_ANY ||
options->for_api == RD_KAFKA_ADMIN_OP_CREATETOPICS ||
options->for_api == RD_KAFKA_ADMIN_OP_DELETETOPICS ||
options->for_api == RD_KAFKA_ADMIN_OP_CREATEPARTITIONS)
rd_kafka_confval_init_int(&options->operation_timeout,
"operation_timeout",
-1, 3600*1000, 0);
else
rd_kafka_confval_disable(&options->operation_timeout,
"operation_timeout");
if (options->for_api == RD_KAFKA_ADMIN_OP_ANY ||
options->for_api == RD_KAFKA_ADMIN_OP_CREATETOPICS ||
options->for_api == RD_KAFKA_ADMIN_OP_CREATEPARTITIONS ||
options->for_api == RD_KAFKA_ADMIN_OP_ALTERCONFIGS)
rd_kafka_confval_init_int(&options->validate_only,
"validate_only",
0, 1, 0);
else
rd_kafka_confval_disable(&options->validate_only,
"validate_only");
if (options->for_api == RD_KAFKA_ADMIN_OP_ANY ||
options->for_api == RD_KAFKA_ADMIN_OP_ALTERCONFIGS)
rd_kafka_confval_init_int(&options->incremental,
"incremental",
0, 1, 0);
else
rd_kafka_confval_disable(&options->incremental,
"incremental");
rd_kafka_confval_init_int(&options->broker, "broker",
0, INT32_MAX, -1);
rd_kafka_confval_init_ptr(&options->opaque, "opaque");
}
rd_kafka_AdminOptions_t *
rd_kafka_AdminOptions_new (rd_kafka_t *rk, rd_kafka_admin_op_t for_api) {
rd_kafka_AdminOptions_t *options;
if ((int)for_api < 0 || for_api >= RD_KAFKA_ADMIN_OP__CNT)
return NULL;
options = rd_calloc(1, sizeof(*options));
options->for_api = for_api;
rd_kafka_AdminOptions_init(rk, options);
return options;
}
void rd_kafka_AdminOptions_destroy (rd_kafka_AdminOptions_t *options) {
rd_free(options);
}
/**@}*/
/**
* @name CreateTopics
* @{
*
*
*
*/
rd_kafka_NewTopic_t *
rd_kafka_NewTopic_new (const char *topic,
int num_partitions,
int replication_factor,
char *errstr, size_t errstr_size) {