-
Notifications
You must be signed in to change notification settings - Fork 3.2k
/
Copy pathrdkafka_conf.c
4527 lines (3964 loc) · 190 KB
/
rdkafka_conf.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) 2012-2022, Magnus Edenhill
* 2023 Confluent Inc.
* 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 "rd.h"
#include "rdfloat.h"
#include <stdlib.h>
#include <ctype.h>
#include <stddef.h>
#include "rdkafka_int.h"
#include "rdkafka_feature.h"
#include "rdkafka_interceptor.h"
#include "rdkafka_idempotence.h"
#include "rdkafka_assignor.h"
#include "rdkafka_sasl_oauthbearer.h"
#if WITH_PLUGINS
#include "rdkafka_plugin.h"
#endif
#include "rdunittest.h"
#ifndef _WIN32
#include <netinet/tcp.h>
#else
#ifndef WIN32_MEAN_AND_LEAN
#define WIN32_MEAN_AND_LEAN
#endif
#include <windows.h>
#endif
struct rd_kafka_property {
rd_kafka_conf_scope_t scope;
const char *name;
enum {
_RK_C_STR,
_RK_C_INT,
_RK_C_DBL, /* Double */
_RK_C_S2I, /* String to Integer mapping.
* Supports limited canonical str->int mappings
* using s2i[] */
_RK_C_S2F, /* CSV String to Integer flag mapping (OR:ed) */
_RK_C_BOOL,
_RK_C_PTR, /* Only settable through special set functions */
_RK_C_PATLIST, /* Pattern list */
_RK_C_KSTR, /* Kafka string */
_RK_C_ALIAS, /* Alias: points to other property through .sdef */
_RK_C_INTERNAL, /* Internal, don't expose to application */
_RK_C_INVALID, /* Invalid property, used to catch known
* but unsupported Java properties. */
} type;
int offset;
const char *desc;
int vmin;
int vmax;
int vdef; /* Default value (int) */
const char *sdef; /* Default value (string) */
void *pdef; /* Default value (pointer) */
double ddef; /* Default value (double) */
double dmin;
double dmax;
struct {
int val;
const char *str;
const char *unsupported; /**< Reason for value not being
* supported in this build. */
} s2i[21]; /* _RK_C_S2I and _RK_C_S2F */
const char *unsupported; /**< Reason for propery not being supported
* in this build.
* Will be included in the conf_set()
* error string. */
/* Value validator (STR) */
int (*validate)(const struct rd_kafka_property *prop,
const char *val,
int ival);
/* Configuration object constructors and destructor for use when
* the property value itself is not used, or needs extra care. */
void (*ctor)(int scope, void *pconf);
void (*dtor)(int scope, void *pconf);
void (*copy)(int scope,
void *pdst,
const void *psrc,
void *dstptr,
const void *srcptr,
size_t filter_cnt,
const char **filter);
rd_kafka_conf_res_t (*set)(int scope,
void *pconf,
const char *name,
const char *value,
void *dstptr,
rd_kafka_conf_set_mode_t set_mode,
char *errstr,
size_t errstr_size);
};
#define _RK(field) offsetof(rd_kafka_conf_t, field)
#define _RKT(field) offsetof(rd_kafka_topic_conf_t, field)
#if WITH_SSL
#define _UNSUPPORTED_SSL .unsupported = NULL
#else
#define _UNSUPPORTED_SSL .unsupported = "OpenSSL not available at build time"
#endif
#if OPENSSL_VERSION_NUMBER >= 0x1000200fL && defined(WITH_SSL) && \
!defined(LIBRESSL_VERSION_NUMBER)
#define _UNSUPPORTED_OPENSSL_1_0_2 .unsupported = NULL
#else
#define _UNSUPPORTED_OPENSSL_1_0_2 \
.unsupported = "OpenSSL >= 1.0.2 not available at build time"
#endif
#if OPENSSL_VERSION_NUMBER >= 0x10100000 && defined(WITH_SSL) && \
!defined(LIBRESSL_VERSION_NUMBER)
#define _UNSUPPORTED_OPENSSL_1_1_0 .unsupported = NULL
#else
#define _UNSUPPORTED_OPENSSL_1_1_0 \
.unsupported = "OpenSSL >= 1.1.0 not available at build time"
#endif
#if WITH_SSL_ENGINE
#define _UNSUPPORTED_SSL_ENGINE .unsupported = NULL
#else
#define _UNSUPPORTED_SSL_ENGINE \
.unsupported = "OpenSSL >= 1.1.x not available at build time"
#endif
#if OPENSSL_VERSION_NUMBER >= 0x30000000 && defined(WITH_SSL)
#define _UNSUPPORTED_SSL_3 .unsupported = NULL
#else
#define _UNSUPPORTED_SSL_3 \
.unsupported = "OpenSSL >= 3.0.0 not available at build time"
#endif
#if WITH_ZLIB
#define _UNSUPPORTED_ZLIB .unsupported = NULL
#else
#define _UNSUPPORTED_ZLIB .unsupported = "zlib not available at build time"
#endif
#if WITH_SNAPPY
#define _UNSUPPORTED_SNAPPY .unsupported = NULL
#else
#define _UNSUPPORTED_SNAPPY .unsupported = "snappy not enabled at build time"
#endif
#if WITH_ZSTD
#define _UNSUPPORTED_ZSTD .unsupported = NULL
#else
#define _UNSUPPORTED_ZSTD .unsupported = "libzstd not available at build time"
#endif
#if WITH_CURL
#define _UNSUPPORTED_HTTP .unsupported = NULL
#else
#define _UNSUPPORTED_HTTP .unsupported = "libcurl not available at build time"
#endif
#if WITH_OAUTHBEARER_OIDC
#define _UNSUPPORTED_OIDC .unsupported = NULL
#else
#define _UNSUPPORTED_OIDC \
.unsupported = \
"OAuth/OIDC depends on libcurl and OpenSSL which were not " \
"available at build time"
#endif
#ifdef _WIN32
#define _UNSUPPORTED_WIN32_GSSAPI \
.unsupported = \
"Kerberos keytabs are not supported on Windows, " \
"instead the logged on " \
"user's credentials are used through native SSPI"
#else
#define _UNSUPPORTED_WIN32_GSSAPI .unsupported = NULL
#endif
#if defined(_WIN32) || defined(WITH_SASL_CYRUS)
#define _UNSUPPORTED_GSSAPI .unsupported = NULL
#else
#define _UNSUPPORTED_GSSAPI \
.unsupported = "cyrus-sasl/libsasl2 not available at build time"
#endif
#define _UNSUPPORTED_OAUTHBEARER _UNSUPPORTED_SSL
static rd_kafka_conf_res_t
rd_kafka_anyconf_get0(const void *conf,
const struct rd_kafka_property *prop,
char *dest,
size_t *dest_size);
/**
* @returns a unique index for property \p prop, using the byte position
* of the field.
*/
static RD_INLINE int rd_kafka_prop2idx(const struct rd_kafka_property *prop) {
return prop->offset;
}
/**
* @brief Set the property as modified.
*
* We do this by mapping the property's conf struct field byte offset
* to a bit in a bit vector.
* If the bit is set the property has been modified, otherwise it is
* at its default unmodified value.
*
* \p is_modified 1: set as modified, 0: clear modified
*/
static void rd_kafka_anyconf_set_modified(void *conf,
const struct rd_kafka_property *prop,
int is_modified) {
int idx = rd_kafka_prop2idx(prop);
int bkt = idx / 64;
uint64_t bit = (uint64_t)1 << (idx % 64);
struct rd_kafka_anyconf_hdr *confhdr = conf;
rd_assert(idx < RD_KAFKA_CONF_PROPS_IDX_MAX &&
*"Increase RD_KAFKA_CONF_PROPS_IDX_MAX");
if (is_modified)
confhdr->modified[bkt] |= bit;
else
confhdr->modified[bkt] &= ~bit;
}
/**
* @brief Clear is_modified for all properties.
* @warning Does NOT clear/reset the value.
*/
static void rd_kafka_anyconf_clear_all_is_modified(void *conf) {
struct rd_kafka_anyconf_hdr *confhdr = conf;
memset(confhdr, 0, sizeof(*confhdr));
}
/**
* @returns true of the property has been set/modified, else false.
*/
static rd_bool_t
rd_kafka_anyconf_is_modified(const void *conf,
const struct rd_kafka_property *prop) {
int idx = rd_kafka_prop2idx(prop);
int bkt = idx / 64;
uint64_t bit = (uint64_t)1 << (idx % 64);
const struct rd_kafka_anyconf_hdr *confhdr = conf;
return !!(confhdr->modified[bkt] & bit);
}
/**
* @returns true if any property in \p conf has been set/modified.
*/
static rd_bool_t rd_kafka_anyconf_is_any_modified(const void *conf) {
const struct rd_kafka_anyconf_hdr *confhdr = conf;
int i;
for (i = 0; i < (int)RD_ARRAYSIZE(confhdr->modified); i++)
if (confhdr->modified[i])
return rd_true;
return rd_false;
}
/**
* @brief Validate \p broker.version.fallback property.
*/
static int
rd_kafka_conf_validate_broker_version(const struct rd_kafka_property *prop,
const char *val,
int ival) {
struct rd_kafka_ApiVersion *apis;
size_t api_cnt;
return rd_kafka_get_legacy_ApiVersions(val, &apis, &api_cnt, NULL);
}
/**
* @brief Validate that string is a single item, without delimters (, space).
*/
static RD_UNUSED int
rd_kafka_conf_validate_single(const struct rd_kafka_property *prop,
const char *val,
int ival) {
return !strchr(val, ',') && !strchr(val, ' ');
}
/**
* @brief Validate builtin partitioner string
*/
static RD_UNUSED int
rd_kafka_conf_validate_partitioner(const struct rd_kafka_property *prop,
const char *val,
int ival) {
return !strcmp(val, "random") || !strcmp(val, "consistent") ||
!strcmp(val, "consistent_random") || !strcmp(val, "murmur2") ||
!strcmp(val, "murmur2_random") || !strcmp(val, "fnv1a") ||
!strcmp(val, "fnv1a_random");
}
/**
* librdkafka configuration property definitions.
*/
static const struct rd_kafka_property rd_kafka_properties[] = {
/* Global properties */
{_RK_GLOBAL, "builtin.features", _RK_C_S2F, _RK(builtin_features),
"Indicates the builtin features for this build of librdkafka. "
"An application can either query this value or attempt to set it "
"with its list of required features to check for library support.",
0, 0x7fffffff, 0xffff,
.s2i = {{0x1, "gzip", _UNSUPPORTED_ZLIB},
{0x2, "snappy", _UNSUPPORTED_SNAPPY},
{0x4, "ssl", _UNSUPPORTED_SSL},
{0x8, "sasl"},
{0x10, "regex"},
{0x20, "lz4"},
{0x40, "sasl_gssapi", _UNSUPPORTED_GSSAPI},
{0x80, "sasl_plain"},
{0x100, "sasl_scram", _UNSUPPORTED_SSL},
{0x200, "plugins"
#if !WITH_PLUGINS
,
.unsupported = "libdl/dlopen(3) not available at "
"build time"
#endif
},
{0x400, "zstd", _UNSUPPORTED_ZSTD},
{0x800, "sasl_oauthbearer", _UNSUPPORTED_SSL},
{0x1000, "http", _UNSUPPORTED_HTTP},
{0x2000, "oidc", _UNSUPPORTED_OIDC},
{0, NULL}}},
{_RK_GLOBAL, "client.id", _RK_C_STR, _RK(client_id_str),
"Client identifier.", .sdef = "rdkafka"},
{_RK_GLOBAL | _RK_HIDDEN, "client.software.name", _RK_C_STR, _RK(sw_name),
"Client software name as reported to broker version >= v2.4.0. "
"Broker-side character restrictions apply, as of broker version "
"v2.4.0 the allowed characters are `a-zA-Z0-9.-`. The local client "
"will replace any other character with `-` and strip leading and "
"trailing non-alphanumeric characters before tranmission to "
"the broker. "
"This property should only be set by high-level language "
"librdkafka client bindings.",
.sdef = "librdkafka"},
{
_RK_GLOBAL | _RK_HIDDEN,
"client.software.version",
_RK_C_STR,
_RK(sw_version),
"Client software version as reported to broker version >= v2.4.0. "
"Broker-side character restrictions apply, as of broker version "
"v2.4.0 the allowed characters are `a-zA-Z0-9.-`. The local client "
"will replace any other character with `-` and strip leading and "
"trailing non-alphanumeric characters before tranmission to "
"the broker. "
"This property should only be set by high-level language "
"librdkafka client bindings."
"If changing this property it is highly recommended to append the "
"librdkafka version.",
},
{_RK_GLOBAL | _RK_HIGH, "metadata.broker.list", _RK_C_STR, _RK(brokerlist),
"Initial list of brokers as a CSV list of broker host or host:port. "
"The application may also use `rd_kafka_brokers_add()` to add "
"brokers during runtime."},
{_RK_GLOBAL | _RK_HIGH, "bootstrap.servers", _RK_C_ALIAS, 0,
"See metadata.broker.list", .sdef = "metadata.broker.list"},
{_RK_GLOBAL | _RK_MED, "message.max.bytes", _RK_C_INT, _RK(max_msg_size),
"Maximum Kafka protocol request message size. "
"Due to differing framing overhead between protocol versions the "
"producer is unable to reliably enforce a strict max message limit "
"at produce time and may exceed the maximum size by one message in "
"protocol ProduceRequests, the broker will enforce the the topic's "
"`max.message.bytes` limit (see Apache Kafka documentation).",
1000, 1000000000, 1000000},
{_RK_GLOBAL, "message.copy.max.bytes", _RK_C_INT, _RK(msg_copy_max_size),
"Maximum size for message to be copied to buffer. "
"Messages larger than this will be passed by reference (zero-copy) "
"at the expense of larger iovecs.",
0, 1000000000, 0xffff},
{_RK_GLOBAL | _RK_MED, "receive.message.max.bytes", _RK_C_INT,
_RK(recv_max_msg_size),
"Maximum Kafka protocol response message size. "
"This serves as a safety precaution to avoid memory exhaustion in "
"case of protocol hickups. "
"This value must be at least `fetch.max.bytes` + 512 to allow "
"for protocol overhead; the value is adjusted automatically "
"unless the configuration property is explicitly set.",
1000, INT_MAX, 100000000},
{_RK_GLOBAL, "max.in.flight.requests.per.connection", _RK_C_INT,
_RK(max_inflight),
"Maximum number of in-flight requests per broker connection. "
"This is a generic property applied to all broker communication, "
"however it is primarily relevant to produce requests. "
"In particular, note that other mechanisms limit the number "
"of outstanding consumer fetch request per broker to one.",
1, 1000000, 1000000},
{_RK_GLOBAL, "max.in.flight", _RK_C_ALIAS,
.sdef = "max.in.flight.requests.per.connection"},
{_RK_GLOBAL, "metadata.recovery.strategy", _RK_C_S2I,
_RK(metadata_recovery_strategy),
"Controls how the client recovers when none of the brokers known to it "
"is available. If set to `none`, the client fails with a fatal error. "
"If set to `rebootstrap`, the client repeats the bootstrap process "
"using `bootstrap.servers` and brokers added through "
"`rd_kafka_brokers_add()`. Rebootstrapping is useful when a client "
"communicates with brokers so infrequently that the set of brokers "
"may change entirely before the client refreshes metadata. "
"Metadata recovery is triggered when all last-known brokers appear "
"unavailable simultaneously.",
.vdef = RD_KAFKA_METADATA_RECOVERY_STRATEGY_REBOOTSTRAP,
.s2i = {{RD_KAFKA_METADATA_RECOVERY_STRATEGY_NONE, "none"},
{RD_KAFKA_METADATA_RECOVERY_STRATEGY_REBOOTSTRAP, "rebootstrap"},
{0, NULL}}},
{_RK_GLOBAL | _RK_DEPRECATED | _RK_HIDDEN, "metadata.request.timeout.ms",
_RK_C_INT, _RK(metadata_request_timeout_ms), "Not used.", 10, 900 * 1000,
10},
{_RK_GLOBAL, "topic.metadata.refresh.interval.ms", _RK_C_INT,
_RK(metadata_refresh_interval_ms),
"Period of time in milliseconds at which topic and broker "
"metadata is refreshed in order to proactively discover any new "
"brokers, topics, partitions or partition leader changes. "
"Use -1 to disable the intervalled refresh (not recommended). "
"If there are no locally referenced topics "
"(no topic objects created, no messages produced, "
"no subscription or no assignment) then only the broker list will "
"be refreshed every interval but no more often than every 10s.",
-1, 3600 * 1000, 5 * 60 * 1000},
{_RK_GLOBAL, "metadata.max.age.ms", _RK_C_INT, _RK(metadata_max_age_ms),
"Metadata cache max age. "
"Defaults to topic.metadata.refresh.interval.ms * 3",
1, 24 * 3600 * 1000, 5 * 60 * 1000 * 3},
{_RK_GLOBAL, "topic.metadata.refresh.fast.interval.ms", _RK_C_INT,
_RK(metadata_refresh_fast_interval_ms),
"When a topic loses its leader a new metadata request will be "
"enqueued immediately and then with this initial interval, exponentially "
"increasing upto `retry.backoff.max.ms`, "
"until the topic metadata has been refreshed. "
"If not set explicitly, it will be defaulted to `retry.backoff.ms`. "
"This is used to recover quickly from transitioning leader brokers.",
1, 60 * 1000, 100},
{_RK_GLOBAL | _RK_DEPRECATED, "topic.metadata.refresh.fast.cnt", _RK_C_INT,
_RK(metadata_refresh_fast_cnt), "No longer used.", 0, 1000, 10},
{_RK_GLOBAL, "topic.metadata.refresh.sparse", _RK_C_BOOL,
_RK(metadata_refresh_sparse),
"Sparse metadata requests (consumes less network bandwidth)", 0, 1, 1},
{_RK_GLOBAL, "topic.metadata.propagation.max.ms", _RK_C_INT,
_RK(metadata_propagation_max_ms),
"Apache Kafka topic creation is asynchronous and it takes some "
"time for a new topic to propagate throughout the cluster to all "
"brokers. "
"If a client requests topic metadata after manual topic creation but "
"before the topic has been fully propagated to the broker the "
"client is requesting metadata from, the topic will seem to be "
"non-existent and the client will mark the topic as such, "
"failing queued produced messages with `ERR__UNKNOWN_TOPIC`. "
"This setting delays marking a topic as non-existent until the "
"configured propagation max time has passed. "
"The maximum propagation time is calculated from the time the "
"topic is first referenced in the client, e.g., on produce().",
0, 60 * 60 * 1000, 30 * 1000},
{_RK_GLOBAL, "topic.blacklist", _RK_C_PATLIST, _RK(topic_blacklist),
"Topic blacklist, a comma-separated list of regular expressions "
"for matching topic names that should be ignored in "
"broker metadata information as if the topics did not exist."},
{_RK_GLOBAL | _RK_MED, "debug", _RK_C_S2F, _RK(debug),
"A comma-separated list of debug contexts to enable. "
"Detailed Producer debugging: broker,topic,msg. "
"Consumer: consumer,cgrp,topic,fetch",
.s2i = {{RD_KAFKA_DBG_GENERIC, "generic"},
{RD_KAFKA_DBG_BROKER, "broker"},
{RD_KAFKA_DBG_TOPIC, "topic"},
{RD_KAFKA_DBG_METADATA, "metadata"},
{RD_KAFKA_DBG_FEATURE, "feature"},
{RD_KAFKA_DBG_QUEUE, "queue"},
{RD_KAFKA_DBG_MSG, "msg"},
{RD_KAFKA_DBG_PROTOCOL, "protocol"},
{RD_KAFKA_DBG_CGRP, "cgrp"},
{RD_KAFKA_DBG_SECURITY, "security"},
{RD_KAFKA_DBG_FETCH, "fetch"},
{RD_KAFKA_DBG_INTERCEPTOR, "interceptor"},
{RD_KAFKA_DBG_PLUGIN, "plugin"},
{RD_KAFKA_DBG_CONSUMER, "consumer"},
{RD_KAFKA_DBG_ADMIN, "admin"},
{RD_KAFKA_DBG_EOS, "eos"},
{RD_KAFKA_DBG_MOCK, "mock"},
{RD_KAFKA_DBG_ASSIGNOR, "assignor"},
{RD_KAFKA_DBG_CONF, "conf"},
{RD_KAFKA_DBG_TELEMETRY, "telemetry"},
{RD_KAFKA_DBG_ALL, "all"}}},
{_RK_GLOBAL, "socket.timeout.ms", _RK_C_INT, _RK(socket_timeout_ms),
"Default timeout for network requests. "
"Producer: ProduceRequests will use the lesser value of "
"`socket.timeout.ms` and remaining `message.timeout.ms` for the "
"first message in the batch. "
"Consumer: FetchRequests will use "
"`fetch.wait.max.ms` + `socket.timeout.ms`. "
"Admin: Admin requests will use `socket.timeout.ms` or explicitly "
"set `rd_kafka_AdminOptions_set_operation_timeout()` value.",
10, 300 * 1000, 60 * 1000},
{_RK_GLOBAL | _RK_DEPRECATED, "socket.blocking.max.ms", _RK_C_INT,
_RK(socket_blocking_max_ms), "No longer used.", 1, 60 * 1000, 1000},
{_RK_GLOBAL, "socket.send.buffer.bytes", _RK_C_INT, _RK(socket_sndbuf_size),
"Broker socket send buffer size. System default is used if 0.", 0,
100000000, 0},
{_RK_GLOBAL, "socket.receive.buffer.bytes", _RK_C_INT,
_RK(socket_rcvbuf_size),
"Broker socket receive buffer size. System default is used if 0.", 0,
100000000, 0},
{_RK_GLOBAL, "socket.keepalive.enable", _RK_C_BOOL, _RK(socket_keepalive),
"Enable TCP keep-alives (SO_KEEPALIVE) on broker sockets", 0, 1, 0
#ifndef SO_KEEPALIVE
,
.unsupported = "SO_KEEPALIVE not available at build time"
#endif
},
{_RK_GLOBAL, "socket.nagle.disable", _RK_C_BOOL, _RK(socket_nagle_disable),
"Disable the Nagle algorithm (TCP_NODELAY) on broker sockets.", 0, 1, 1
#ifndef TCP_NODELAY
,
.unsupported = "TCP_NODELAY not available at build time"
#endif
},
{_RK_GLOBAL, "socket.max.fails", _RK_C_INT, _RK(socket_max_fails),
"Disconnect from broker when this number of send failures "
"(e.g., timed out requests) is reached. Disable with 0. "
"WARNING: It is highly recommended to leave this setting at "
"its default value of 1 to avoid the client and broker to "
"become desynchronized in case of request timeouts. "
"NOTE: The connection is automatically re-established.",
0, 1000000, 1},
{_RK_GLOBAL, "broker.address.ttl", _RK_C_INT, _RK(broker_addr_ttl),
"How long to cache the broker address resolving "
"results (milliseconds).",
0, 86400 * 1000, 1 * 1000},
{_RK_GLOBAL, "broker.address.family", _RK_C_S2I, _RK(broker_addr_family),
"Allowed broker IP address families: any, v4, v6", .vdef = AF_UNSPEC,
.s2i =
{
{AF_UNSPEC, "any"},
{AF_INET, "v4"},
{AF_INET6, "v6"},
}},
{_RK_GLOBAL | _RK_MED, "socket.connection.setup.timeout.ms", _RK_C_INT,
_RK(socket_connection_setup_timeout_ms),
"Maximum time allowed for broker connection setup "
"(TCP connection setup as well SSL and SASL handshake). "
"If the connection to the broker is not fully functional after this "
"the connection will be closed and retried.",
1000, INT_MAX, 30 * 1000 /* 30s */},
{_RK_GLOBAL | _RK_MED, "connections.max.idle.ms", _RK_C_INT,
_RK(connections_max_idle_ms),
"Close broker connections after the specified time of "
"inactivity. "
"Disable with 0. "
"If this property is left at its default value some heuristics are "
"performed to determine a suitable default value, this is currently "
"limited to identifying brokers on Azure "
"(see librdkafka issue #3109 for more info).",
0, INT_MAX, 0},
{_RK_GLOBAL | _RK_MED | _RK_HIDDEN, "enable.sparse.connections", _RK_C_BOOL,
_RK(sparse_connections),
"When enabled the client will only connect to brokers "
"it needs to communicate with. When disabled the client "
"will maintain connections to all brokers in the cluster.",
0, 1, 1},
{_RK_GLOBAL | _RK_DEPRECATED, "reconnect.backoff.jitter.ms", _RK_C_INT,
_RK(reconnect_jitter_ms),
"No longer used. See `reconnect.backoff.ms` and "
"`reconnect.backoff.max.ms`.",
0, 60 * 60 * 1000, 0},
{_RK_GLOBAL | _RK_MED, "reconnect.backoff.ms", _RK_C_INT,
_RK(reconnect_backoff_ms),
"The initial time to wait before reconnecting to a broker "
"after the connection has been closed. "
"The time is increased exponentially until "
"`reconnect.backoff.max.ms` is reached. "
"-25% to +50% jitter is applied to each reconnect backoff. "
"A value of 0 disables the backoff and reconnects immediately.",
0, 60 * 60 * 1000, 100},
{_RK_GLOBAL | _RK_MED, "reconnect.backoff.max.ms", _RK_C_INT,
_RK(reconnect_backoff_max_ms),
"The maximum time to wait before reconnecting to a broker "
"after the connection has been closed.",
0, 60 * 60 * 1000, 10 * 1000},
{_RK_GLOBAL | _RK_HIGH, "statistics.interval.ms", _RK_C_INT,
_RK(stats_interval_ms),
"librdkafka statistics emit interval. The application also needs to "
"register a stats callback using `rd_kafka_conf_set_stats_cb()`. "
"The granularity is 1000ms. A value of 0 disables statistics.",
0, 86400 * 1000, 0},
{_RK_GLOBAL, "enabled_events", _RK_C_INT, _RK(enabled_events),
"See `rd_kafka_conf_set_events()`", 0, 0x7fffffff, 0},
{_RK_GLOBAL, "error_cb", _RK_C_PTR, _RK(error_cb),
"Error callback (set with rd_kafka_conf_set_error_cb())"},
{_RK_GLOBAL, "throttle_cb", _RK_C_PTR, _RK(throttle_cb),
"Throttle callback (set with rd_kafka_conf_set_throttle_cb())"},
{_RK_GLOBAL, "stats_cb", _RK_C_PTR, _RK(stats_cb),
"Statistics callback (set with rd_kafka_conf_set_stats_cb())"},
{_RK_GLOBAL, "log_cb", _RK_C_PTR, _RK(log_cb),
"Log callback (set with rd_kafka_conf_set_log_cb())",
.pdef = rd_kafka_log_print},
{_RK_GLOBAL, "log_level", _RK_C_INT, _RK(log_level),
"Logging level (syslog(3) levels)", 0, 7, 6},
{_RK_GLOBAL, "log.queue", _RK_C_BOOL, _RK(log_queue),
"Disable spontaneous log_cb from internal librdkafka "
"threads, instead enqueue log messages on queue set with "
"`rd_kafka_set_log_queue()` and serve log callbacks or "
"events through the standard poll APIs. "
"**NOTE**: Log messages will linger in a temporary queue "
"until the log queue has been set.",
0, 1, 0},
{_RK_GLOBAL, "log.thread.name", _RK_C_BOOL, _RK(log_thread_name),
"Print internal thread name in log messages "
"(useful for debugging librdkafka internals)",
0, 1, 1},
{_RK_GLOBAL, "enable.random.seed", _RK_C_BOOL, _RK(enable_random_seed),
"If enabled librdkafka will initialize the PRNG "
"with srand(current_time.milliseconds) on the first invocation of "
"rd_kafka_new() (required only if rand_r() is not available on your "
"platform). "
"If disabled the application must call srand() prior to calling "
"rd_kafka_new().",
0, 1, 1},
{_RK_GLOBAL, "log.connection.close", _RK_C_BOOL, _RK(log_connection_close),
"Log broker disconnects. "
"It might be useful to turn this off when interacting with "
"0.9 brokers with an aggressive `connections.max.idle.ms` value.",
0, 1, 1},
{_RK_GLOBAL, "background_event_cb", _RK_C_PTR, _RK(background_event_cb),
"Background queue event callback "
"(set with rd_kafka_conf_set_background_event_cb())"},
{_RK_GLOBAL, "socket_cb", _RK_C_PTR, _RK(socket_cb),
"Socket creation callback to provide race-free CLOEXEC",
.pdef =
#ifdef __linux__
rd_kafka_socket_cb_linux
#else
rd_kafka_socket_cb_generic
#endif
},
{
_RK_GLOBAL,
"connect_cb",
_RK_C_PTR,
_RK(connect_cb),
"Socket connect callback",
},
{
_RK_GLOBAL,
"closesocket_cb",
_RK_C_PTR,
_RK(closesocket_cb),
"Socket close callback",
},
{_RK_GLOBAL, "open_cb", _RK_C_PTR, _RK(open_cb),
"File open callback to provide race-free CLOEXEC",
.pdef =
#ifdef __linux__
rd_kafka_open_cb_linux
#else
rd_kafka_open_cb_generic
#endif
},
{_RK_GLOBAL, "resolve_cb", _RK_C_PTR, _RK(resolve_cb),
"Address resolution callback (set with rd_kafka_conf_set_resolve_cb())."},
{_RK_GLOBAL, "opaque", _RK_C_PTR, _RK(opaque),
"Application opaque (set with rd_kafka_conf_set_opaque())"},
{_RK_GLOBAL, "default_topic_conf", _RK_C_PTR, _RK(topic_conf),
"Default topic configuration for automatically subscribed topics"},
{_RK_GLOBAL, "internal.termination.signal", _RK_C_INT, _RK(term_sig),
"Signal that librdkafka will use to quickly terminate on "
"rd_kafka_destroy(). If this signal is not set then there will be a "
"delay before rd_kafka_wait_destroyed() returns true "
"as internal threads are timing out their system calls. "
"If this signal is set however the delay will be minimal. "
"The application should mask this signal as an internal "
"signal handler is installed.",
0, 128, 0},
{_RK_GLOBAL | _RK_HIGH | _RK_DEPRECATED, "api.version.request", _RK_C_BOOL,
_RK(api_version_request),
"**Post-deprecation actions: remove this configuration property, "
"brokers < 0.10.0 won't be supported anymore in librdkafka 3.x.** "
"Request broker's supported API versions to adjust functionality to "
"available protocol features. If set to false, or the "
"ApiVersionRequest fails, the fallback version "
"`broker.version.fallback` will be used. "
"**NOTE**: Depends on broker version >=0.10.0. If the request is not "
"supported by (an older) broker the `broker.version.fallback` fallback is "
"used.",
0, 1, 1},
{_RK_GLOBAL, "api.version.request.timeout.ms", _RK_C_INT,
_RK(api_version_request_timeout_ms),
"Timeout for broker API version requests.", 1, 5 * 60 * 1000, 10 * 1000},
{_RK_GLOBAL | _RK_MED | _RK_DEPRECATED, "api.version.fallback.ms",
_RK_C_INT, _RK(api_version_fallback_ms),
"**Post-deprecation actions: remove this configuration property, "
"brokers < 0.10.0 won't be supported anymore in librdkafka 3.x.** "
"Dictates how long the `broker.version.fallback` fallback is used "
"in the case the ApiVersionRequest fails. "
"**NOTE**: The ApiVersionRequest is only issued when a new connection "
"to the broker is made (such as after an upgrade).",
0, 86400 * 7 * 1000, 0},
{_RK_GLOBAL | _RK_MED | _RK_DEPRECATED, "broker.version.fallback",
_RK_C_STR, _RK(broker_version_fallback),
"**Post-deprecation actions: remove this configuration property, "
"brokers < 0.10.0 won't be supported anymore in librdkafka 3.x.** "
"Older broker versions (before 0.10.0) provide no way for a client to "
"query "
"for supported protocol features "
"(ApiVersionRequest, see `api.version.request`) making it impossible "
"for the client to know what features it may use. "
"As a workaround a user may set this property to the expected broker "
"version and the client will automatically adjust its feature set "
"accordingly if the ApiVersionRequest fails (or is disabled). "
"The fallback broker version will be used for `api.version.fallback.ms`. "
"Valid values are: 0.9.0, 0.8.2, 0.8.1, 0.8.0. "
"Any other value >= 0.10, such as 0.10.2.1, "
"enables ApiVersionRequests.",
.sdef = "0.10.0", .validate = rd_kafka_conf_validate_broker_version},
{_RK_GLOBAL, "allow.auto.create.topics", _RK_C_BOOL,
_RK(allow_auto_create_topics),
"Allow automatic topic creation on the broker when subscribing to "
"or assigning non-existent topics. "
"The broker must also be configured with "
"`auto.create.topics.enable=true` for this configuration to "
"take effect. "
"Note: the default value (true) for the producer is "
"different from the default value (false) for the consumer. "
"Further, the consumer default value is different from the Java "
"consumer (true), and this property is not supported by the Java "
"producer. Requires broker version >= 0.11.0.0, for older broker "
"versions only the broker configuration applies.",
0, 1, 0},
/* Security related global properties */
{_RK_GLOBAL | _RK_HIGH, "security.protocol", _RK_C_S2I,
_RK(security_protocol), "Protocol used to communicate with brokers.",
.vdef = RD_KAFKA_PROTO_PLAINTEXT,
.s2i = {{RD_KAFKA_PROTO_PLAINTEXT, "plaintext"},
{RD_KAFKA_PROTO_SSL, "ssl", _UNSUPPORTED_SSL},
{RD_KAFKA_PROTO_SASL_PLAINTEXT, "sasl_plaintext"},
{RD_KAFKA_PROTO_SASL_SSL, "sasl_ssl", _UNSUPPORTED_SSL},
{0, NULL}}},
{_RK_GLOBAL, "ssl.cipher.suites", _RK_C_STR, _RK(ssl.cipher_suites),
"A cipher suite is a named combination of authentication, "
"encryption, MAC and key exchange algorithm used to negotiate the "
"security settings for a network connection using TLS or SSL network "
"protocol. See manual page for `ciphers(1)` and "
"`SSL_CTX_set_cipher_list(3).",
_UNSUPPORTED_SSL},
{_RK_GLOBAL, "ssl.curves.list", _RK_C_STR, _RK(ssl.curves_list),
"The supported-curves extension in the TLS ClientHello message specifies "
"the curves (standard/named, or 'explicit' GF(2^k) or GF(p)) the client "
"is willing to have the server use. See manual page for "
"`SSL_CTX_set1_curves_list(3)`. OpenSSL >= 1.0.2 required.",
_UNSUPPORTED_OPENSSL_1_0_2},
{_RK_GLOBAL, "ssl.sigalgs.list", _RK_C_STR, _RK(ssl.sigalgs_list),
"The client uses the TLS ClientHello signature_algorithms extension "
"to indicate to the server which signature/hash algorithm pairs "
"may be used in digital signatures. See manual page for "
"`SSL_CTX_set1_sigalgs_list(3)`. OpenSSL >= 1.0.2 required.",
_UNSUPPORTED_OPENSSL_1_0_2},
{_RK_GLOBAL | _RK_SENSITIVE, "ssl.key.location", _RK_C_STR,
_RK(ssl.key_location),
"Path to client's private key (PEM) used for authentication.",
_UNSUPPORTED_SSL},
{_RK_GLOBAL | _RK_SENSITIVE, "ssl.key.password", _RK_C_STR,
_RK(ssl.key_password),
"Private key passphrase (for use with `ssl.key.location` "
"and `set_ssl_cert()`)",
_UNSUPPORTED_SSL},
{_RK_GLOBAL | _RK_SENSITIVE, "ssl.key.pem", _RK_C_STR, _RK(ssl.key_pem),
"Client's private key string (PEM format) used for authentication.",
_UNSUPPORTED_SSL},
{_RK_GLOBAL | _RK_SENSITIVE, "ssl_key", _RK_C_INTERNAL, _RK(ssl.key),
"Client's private key as set by rd_kafka_conf_set_ssl_cert()",
.dtor = rd_kafka_conf_cert_dtor, .copy = rd_kafka_conf_cert_copy,
_UNSUPPORTED_SSL},
{_RK_GLOBAL, "ssl.certificate.location", _RK_C_STR, _RK(ssl.cert_location),
"Path to client's public key (PEM) used for authentication.",
_UNSUPPORTED_SSL},
{_RK_GLOBAL, "ssl.certificate.pem", _RK_C_STR, _RK(ssl.cert_pem),
"Client's public key string (PEM format) used for authentication.",
_UNSUPPORTED_SSL},
{_RK_GLOBAL, "ssl_certificate", _RK_C_INTERNAL, _RK(ssl.key),
"Client's public key as set by rd_kafka_conf_set_ssl_cert()",
.dtor = rd_kafka_conf_cert_dtor, .copy = rd_kafka_conf_cert_copy,
_UNSUPPORTED_SSL},
{_RK_GLOBAL, "ssl.ca.location", _RK_C_STR, _RK(ssl.ca_location),
"File or directory path to CA certificate(s) for verifying "
"the broker's key. "
"Defaults: "
"On Windows the system's CA certificates are automatically looked "
"up in the Windows Root certificate store. "
"On Mac OSX this configuration defaults to `probe`. "
"It is recommended to install openssl using Homebrew, "
"to provide CA certificates. "
"On Linux install the distribution's ca-certificates package. "
"If OpenSSL is statically linked or `ssl.ca.location` is set to "
"`probe` a list of standard paths will be probed and the first one "
"found will be used as the default CA certificate location path. "
"If OpenSSL is dynamically linked the OpenSSL library's default "
"path will be used (see `OPENSSLDIR` in `openssl version -a`).",
_UNSUPPORTED_SSL},
{_RK_GLOBAL | _RK_SENSITIVE, "ssl.ca.pem", _RK_C_STR, _RK(ssl.ca_pem),
"CA certificate string (PEM format) for verifying the broker's key.",
_UNSUPPORTED_SSL},
{_RK_GLOBAL, "ssl_ca", _RK_C_INTERNAL, _RK(ssl.ca),
"CA certificate as set by rd_kafka_conf_set_ssl_cert()",
.dtor = rd_kafka_conf_cert_dtor, .copy = rd_kafka_conf_cert_copy,
_UNSUPPORTED_SSL},
{_RK_GLOBAL, "ssl.ca.certificate.stores", _RK_C_STR,
_RK(ssl.ca_cert_stores),
"Comma-separated list of Windows Certificate stores to load "
"CA certificates from. Certificates will be loaded in the same "
"order as stores are specified. If no certificates can be loaded "
"from any of the specified stores an error is logged and the "
"OpenSSL library's default CA location is used instead. "
"Store names are typically one or more of: MY, Root, Trust, CA.",
.sdef = "Root",
#if !defined(_WIN32)
.unsupported = "configuration only valid on Windows"
#endif
},
{_RK_GLOBAL, "ssl.crl.location", _RK_C_STR, _RK(ssl.crl_location),
"Path to CRL for verifying broker's certificate validity.",
_UNSUPPORTED_SSL},
{_RK_GLOBAL, "ssl.keystore.location", _RK_C_STR, _RK(ssl.keystore_location),
"Path to client's keystore (PKCS#12) used for authentication.",
_UNSUPPORTED_SSL},
{_RK_GLOBAL | _RK_SENSITIVE, "ssl.keystore.password", _RK_C_STR,
_RK(ssl.keystore_password), "Client's keystore (PKCS#12) password.",
_UNSUPPORTED_SSL},
{_RK_GLOBAL, "ssl.providers", _RK_C_STR, _RK(ssl.providers),
"Comma-separated list of OpenSSL 3.0.x implementation providers. "
"E.g., \"default,legacy\".",
_UNSUPPORTED_SSL_3},
{_RK_GLOBAL | _RK_DEPRECATED, "ssl.engine.location", _RK_C_STR,
_RK(ssl.engine_location),
"Path to OpenSSL engine library. OpenSSL >= 1.1.x required. "
"DEPRECATED: OpenSSL engine support is deprecated and should be "
"replaced by OpenSSL 3 providers.",
_UNSUPPORTED_SSL_ENGINE},
{_RK_GLOBAL, "ssl.engine.id", _RK_C_STR, _RK(ssl.engine_id),
"OpenSSL engine id is the name used for loading engine.",
.sdef = "dynamic", _UNSUPPORTED_SSL_ENGINE},
{_RK_GLOBAL, "ssl_engine_callback_data", _RK_C_PTR,
_RK(ssl.engine_callback_data),
"OpenSSL engine callback data (set "
"with rd_kafka_conf_set_engine_callback_data()).",
_UNSUPPORTED_SSL_ENGINE},
{_RK_GLOBAL, "enable.ssl.certificate.verification", _RK_C_BOOL,
_RK(ssl.enable_verify),
"Enable OpenSSL's builtin broker (server) certificate verification. "
"This verification can be extended by the application by "
"implementing a certificate_verify_cb.",
0, 1, 1, _UNSUPPORTED_SSL},
{_RK_GLOBAL, "ssl.endpoint.identification.algorithm", _RK_C_S2I,
_RK(ssl.endpoint_identification),
"Endpoint identification algorithm to validate broker "
"hostname using broker certificate. "
"https - Server (broker) hostname verification as "
"specified in RFC2818. "
"none - No endpoint verification. "
"OpenSSL >= 1.0.2 required.",
.vdef = RD_KAFKA_SSL_ENDPOINT_ID_HTTPS,
.s2i = {{RD_KAFKA_SSL_ENDPOINT_ID_NONE, "none"},
{RD_KAFKA_SSL_ENDPOINT_ID_HTTPS, "https"}},
_UNSUPPORTED_OPENSSL_1_0_2},
{_RK_GLOBAL, "ssl.certificate.verify_cb", _RK_C_PTR,
_RK(ssl.cert_verify_cb),
"Callback to verify the broker certificate chain.", _UNSUPPORTED_SSL},
/* Point user in the right direction if they try to apply
* Java client SSL / JAAS properties. */
{_RK_GLOBAL, "ssl.truststore.location", _RK_C_INVALID, _RK(dummy),
"Java TrustStores are not supported, use `ssl.ca.location` "
"and a certificate file instead. "
"See "
"https://github.com/confluentinc/librdkafka/"
"wiki/Using-SSL-with-librdkafka "
"for more information."},
{_RK_GLOBAL, "sasl.jaas.config", _RK_C_INVALID, _RK(dummy),
"Java JAAS configuration is not supported, see "
"https://github.com/confluentinc/librdkafka/"
"wiki/Using-SASL-with-librdkafka "
"for more information."},
{_RK_GLOBAL | _RK_HIGH, "sasl.mechanisms", _RK_C_STR, _RK(sasl.mechanisms),
"SASL mechanism to use for authentication. "
"Supported: GSSAPI, PLAIN, SCRAM-SHA-256, SCRAM-SHA-512, OAUTHBEARER. "
"**NOTE**: Despite the name only one mechanism must be configured.",
.sdef = "GSSAPI", .validate = rd_kafka_conf_validate_single},
{_RK_GLOBAL | _RK_HIGH, "sasl.mechanism", _RK_C_ALIAS,
.sdef = "sasl.mechanisms"},
{_RK_GLOBAL, "sasl.kerberos.service.name", _RK_C_STR,
_RK(sasl.service_name),
"Kerberos principal name that Kafka runs as, "
"not including /hostname@REALM",
.sdef = "kafka"},
{_RK_GLOBAL, "sasl.kerberos.principal", _RK_C_STR, _RK(sasl.principal),
"This client's Kerberos principal name. "
"(Not supported on Windows, will use the logon user's principal).",
.sdef = "kafkaclient"},
{_RK_GLOBAL, "sasl.kerberos.kinit.cmd", _RK_C_STR, _RK(sasl.kinit_cmd),
"Shell command to refresh or acquire the client's Kerberos ticket. "
"This command is executed on client creation and every "
"sasl.kerberos.min.time.before.relogin (0=disable). "
"%{config.prop.name} is replaced by corresponding config "
"object value.",
.sdef =
/* First attempt to refresh, else acquire. */
"kinit -R -t \"%{sasl.kerberos.keytab}\" "
"-k %{sasl.kerberos.principal} || "
"kinit -t \"%{sasl.kerberos.keytab}\" -k %{sasl.kerberos.principal}",
_UNSUPPORTED_WIN32_GSSAPI},
{_RK_GLOBAL, "sasl.kerberos.keytab", _RK_C_STR, _RK(sasl.keytab),
"Path to Kerberos keytab file. "
"This configuration property is only used as a variable in "
"`sasl.kerberos.kinit.cmd` as "
"` ... -t \"%{sasl.kerberos.keytab}\"`.",
_UNSUPPORTED_WIN32_GSSAPI},
{_RK_GLOBAL, "sasl.kerberos.min.time.before.relogin", _RK_C_INT,
_RK(sasl.relogin_min_time),
"Minimum time in milliseconds between key refresh attempts. "
"Disable automatic key refresh by setting this property to 0.",
0, 86400 * 1000, 60 * 1000, _UNSUPPORTED_WIN32_GSSAPI},
{_RK_GLOBAL | _RK_HIGH | _RK_SENSITIVE, "sasl.username", _RK_C_STR,
_RK(sasl.username),
"SASL username for use with the PLAIN and SASL-SCRAM-.. mechanisms"},
{_RK_GLOBAL | _RK_HIGH | _RK_SENSITIVE, "sasl.password", _RK_C_STR,
_RK(sasl.password),
"SASL password for use with the PLAIN and SASL-SCRAM-.. mechanism"},
{_RK_GLOBAL | _RK_SENSITIVE, "sasl.oauthbearer.config", _RK_C_STR,
_RK(sasl.oauthbearer_config),
"SASL/OAUTHBEARER configuration. The format is "
"implementation-dependent and must be parsed accordingly. The "
"default unsecured token implementation (see "
"https://tools.ietf.org/html/rfc7515#appendix-A.5) recognizes "
"space-separated name=value pairs with valid names including "
"principalClaimName, principal, scopeClaimName, scope, and "
"lifeSeconds. The default value for principalClaimName is \"sub\", "
"the default value for scopeClaimName is \"scope\", and the default "
"value for lifeSeconds is 3600. The scope value is CSV format with "
"the default value being no/empty scope. For example: "
"`principalClaimName=azp principal=admin scopeClaimName=roles "
"scope=role1,role2 lifeSeconds=600`. In addition, SASL extensions "
"can be communicated to the broker via "
"`extension_NAME=value`. For example: "
"`principal=admin extension_traceId=123`",
_UNSUPPORTED_OAUTHBEARER},
{_RK_GLOBAL, "enable.sasl.oauthbearer.unsecure.jwt", _RK_C_BOOL,
_RK(sasl.enable_oauthbearer_unsecure_jwt),
"Enable the builtin unsecure JWT OAUTHBEARER token handler "