forked from twmb/franz-go
/
config.go
1742 lines (1566 loc) · 73.7 KB
/
config.go
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
package kgo
import (
"context"
"crypto/tls"
"errors"
"fmt"
"math"
"math/rand"
"net"
"regexp"
"runtime/debug"
"sync"
"time"
"github.com/twmb/franz-go/pkg/kmsg"
"github.com/twmb/franz-go/pkg/kversion"
"github.com/twmb/franz-go/pkg/sasl"
)
// Opt is an option to configure a client.
type Opt interface {
apply(*cfg)
}
// ProducerOpt is a producer specific option to configure a client.
// This is simply a namespaced Opt.
type ProducerOpt interface {
Opt
producerOpt()
}
// ConsumerOpt is a consumer specific option to configure a client.
// This is simply a namespaced Opt.
type ConsumerOpt interface {
Opt
consumerOpt()
}
// GroupOpt is a consumer group specific option to configure a client.
// This is simply a namespaced Opt.
type GroupOpt interface {
Opt
groupOpt()
}
type (
clientOpt struct{ fn func(*cfg) }
producerOpt struct{ fn func(*cfg) }
consumerOpt struct{ fn func(*cfg) }
groupOpt struct{ fn func(*cfg) }
)
func (opt clientOpt) apply(cfg *cfg) { opt.fn(cfg) }
func (opt producerOpt) apply(cfg *cfg) { opt.fn(cfg) }
func (opt consumerOpt) apply(cfg *cfg) { opt.fn(cfg) }
func (opt groupOpt) apply(cfg *cfg) { opt.fn(cfg) }
func (producerOpt) producerOpt() {}
func (consumerOpt) consumerOpt() {}
func (groupOpt) groupOpt() {}
// A cfg can be written to while initializing a client, and after that it is
// (mostly) only ever read from. Some areas can continue to be modified --
// particularly reconfiguring what to consume from -- but most areas are
// static.
type cfg struct {
/////////////////////
// GENERAL SECTION //
/////////////////////
id *string // client ID
dialFn func(context.Context, string, string) (net.Conn, error)
dialTimeout time.Duration
dialTLS *tls.Config
requestTimeoutOverhead time.Duration
connIdleTimeout time.Duration
softwareName string // KIP-511
softwareVersion string // KIP-511
logger Logger
seedBrokers []string
maxVersions *kversion.Versions
minVersions *kversion.Versions
retryBackoff func(int) time.Duration
retries int64
retryTimeout func(int16) time.Duration
maxBrokerWriteBytes int32
maxBrokerReadBytes int32
allowAutoTopicCreation bool
metadataMaxAge time.Duration
metadataMinAge time.Duration
sasls []sasl.Mechanism
hooks hooks
//////////////////////
// PRODUCER SECTION //
//////////////////////
txnID *string
txnTimeout time.Duration
acks Acks
disableIdempotency bool
maxProduceInflight int // if idempotency is disabled, we allow a configurable max inflight
compression []CompressionCodec // order of preference
defaultProduceTopic string
maxRecordBatchBytes int32
maxBufferedRecords int64
maxBufferedBytes int64
produceTimeout time.Duration
recordRetries int64
maxUnknownFailures int64
linger time.Duration
recordTimeout time.Duration
manualFlushing bool
txnBackoff time.Duration
missingTopicDelete time.Duration
partitioner Partitioner
stopOnDataLoss bool
onDataLoss func(string, int32)
//////////////////////
// CONSUMER SECTION //
//////////////////////
maxWait int32
minBytes int32
maxBytes lazyI32
maxPartBytes lazyI32
resetOffset Offset
isolationLevel int8
keepControl bool
rack string
preferLagFn PreferLagFn
maxConcurrentFetches int
disableFetchSessions bool
keepRetryableFetchErrors bool
topics map[string]*regexp.Regexp // topics to consume; if regex is true, values are compiled regular expressions
partitions map[string]map[int32]Offset // partitions to directly consume from
regex bool
////////////////////////////
// CONSUMER GROUP SECTION //
////////////////////////////
group string // group we are in
instanceID *string // optional group instance ID
balancers []GroupBalancer // balancers we can use
protocol string // "consumer" by default, expected to never be overridden
sessionTimeout time.Duration
rebalanceTimeout time.Duration
heartbeatInterval time.Duration
requireStable bool
onAssigned func(context.Context, *Client, map[string][]int32)
onRevoked func(context.Context, *Client, map[string][]int32)
onLost func(context.Context, *Client, map[string][]int32)
onFetched func(context.Context, *Client, *kmsg.OffsetFetchResponse) error
adjustOffsetsBeforeAssign func(ctx context.Context, offsets map[string]map[int32]Offset) (map[string]map[int32]Offset, error)
blockRebalanceOnPoll bool
setAssigned bool
setRevoked bool
setLost bool
setCommitCallback bool
autocommitDisable bool // true if autocommit was disabled or we are transactional
autocommitGreedy bool
autocommitMarks bool
autocommitInterval time.Duration
commitCallback func(*Client, *kmsg.OffsetCommitRequest, *kmsg.OffsetCommitResponse, error)
}
func (cfg *cfg) validate() error {
if len(cfg.seedBrokers) == 0 {
return errors.New("config erroneously has no seed brokers")
}
// We clamp maxPartBytes to maxBytes because some fake Kafka endpoints
// (Oracle) cannot handle the mismatch correctly.
if cfg.maxPartBytes > cfg.maxBytes {
cfg.maxPartBytes = cfg.maxBytes
}
if cfg.disableIdempotency {
if cfg.txnID != nil {
return errors.New("cannot both disable idempotent writes and use transactional IDs")
}
if cfg.maxProduceInflight <= 0 {
return fmt.Errorf("invalid max produce inflight %d with idempotency disabled", cfg.maxProduceInflight)
}
} else {
if cfg.acks.val != -1 {
return errors.New("idempotency requires acks=all")
}
if cfg.maxProduceInflight != 1 {
return fmt.Errorf("invalid usage of MaxProduceRequestsInflightPerBroker with idempotency enabled")
}
}
for _, limit := range []struct {
name string
sp **string // if field is a *string, we take addr to it
s string
allowed int
}{
// A 256 byte ID / software name & version is good enough and
// fits with our max broker write byte min of 1K.
{name: "client id", sp: &cfg.id, allowed: 256},
{name: "software name", s: cfg.softwareName, allowed: 256},
{name: "software version", s: cfg.softwareVersion, allowed: 256},
// The following is the limit transitioning from two byte
// prefix for flexible stuff to three bytes; as with above, it
// is more than reasonable.
{name: "transactional id", sp: &cfg.txnID, allowed: 16382},
{name: "rack", s: cfg.rack, allowed: 512},
} {
s := limit.s
if limit.sp != nil && *limit.sp != nil {
s = **limit.sp
}
if len(s) > limit.allowed {
return fmt.Errorf("%s length %d is larger than max allowed %d", limit.name, len(s), limit.allowed)
}
}
i64lt := func(l, r int64) (bool, string) { return l < r, "less" }
i64gt := func(l, r int64) (bool, string) { return l > r, "larger" }
for _, limit := range []struct {
name string
v int64
allowed int64
badcmp func(int64, int64) (bool, string)
fmt string
durs bool
}{
// Min write of 1K and max of 1G is reasonable.
{name: "max broker write bytes", v: int64(cfg.maxBrokerWriteBytes), allowed: 1 << 10, badcmp: i64lt},
{name: "max broker write bytes", v: int64(cfg.maxBrokerWriteBytes), allowed: 1 << 30, badcmp: i64gt},
// Same for read bytes.
{name: "max broker read bytes", v: int64(cfg.maxBrokerReadBytes), allowed: 1 << 10, badcmp: i64lt},
{name: "max broker read bytes", v: int64(cfg.maxBrokerReadBytes), allowed: 1 << 30, badcmp: i64gt},
// For batches, we want at least 512 (reasonable), and the
// upper limit is the max num when a uvarint transitions from 4
// to 5 bytes. The upper limit is also more than reasonable
// (256MiB).
{name: "max record batch bytes", v: int64(cfg.maxRecordBatchBytes), allowed: 512, badcmp: i64lt},
{name: "max record batch bytes", v: int64(cfg.maxRecordBatchBytes), allowed: 256 << 20, badcmp: i64gt},
// We do not want the broker write bytes to be less than the
// record batch bytes, nor the read bytes to be less than what
// we indicate to fetch.
//
// We cannot enforce if a single batch is larger than the max
// fetch bytes limit, but hopefully we do not run into that.
{v: int64(cfg.maxBrokerWriteBytes), allowed: int64(cfg.maxRecordBatchBytes), badcmp: i64lt, fmt: "max broker write bytes %v is erroneously less than max record batch bytes %v"},
{v: int64(cfg.maxBrokerReadBytes), allowed: int64(cfg.maxBytes), badcmp: i64lt, fmt: "max broker read bytes %v is erroneously less than max fetch bytes %v"},
// 0 <= allowed concurrency
{name: "max concurrent fetches", v: int64(cfg.maxConcurrentFetches), allowed: 0, badcmp: i64lt},
// 1s <= request timeout overhead <= 15m
{name: "request timeout max overhead", v: int64(cfg.requestTimeoutOverhead), allowed: int64(15 * time.Minute), badcmp: i64gt, durs: true},
{name: "request timeout min overhead", v: int64(cfg.requestTimeoutOverhead), allowed: int64(time.Second), badcmp: i64lt, durs: true},
// 1s <= conn idle <= 15m
{name: "conn min idle timeout", v: int64(cfg.connIdleTimeout), allowed: int64(time.Second), badcmp: i64lt, durs: true},
{name: "conn max idle timeout", v: int64(cfg.connIdleTimeout), allowed: int64(15 * time.Minute), badcmp: i64gt, durs: true},
// 10ms <= metadata <= 1hr
{name: "metadata max age", v: int64(cfg.metadataMaxAge), allowed: int64(time.Hour), badcmp: i64gt, durs: true},
{name: "metadata min age", v: int64(cfg.metadataMinAge), allowed: int64(10 * time.Millisecond), badcmp: i64lt, durs: true},
{v: int64(cfg.metadataMaxAge), allowed: int64(cfg.metadataMinAge), badcmp: i64lt, fmt: "metadata max age %v is erroneously less than metadata min age %v", durs: true},
// Some random producer settings.
{name: "max buffered records", v: cfg.maxBufferedRecords, allowed: 1, badcmp: i64lt},
{name: "max buffered bytes", v: cfg.maxBufferedBytes, allowed: 0, badcmp: i64lt},
{name: "linger", v: int64(cfg.linger), allowed: int64(time.Minute), badcmp: i64gt, durs: true},
{name: "produce timeout", v: int64(cfg.produceTimeout), allowed: int64(100 * time.Millisecond), badcmp: i64lt, durs: true},
{name: "record timeout", v: int64(cfg.recordTimeout), allowed: int64(time.Second), badcmp: func(l, r int64) (bool, string) {
if l == 0 {
return false, "" // we print nothing when things are good
}
return l < r, "less"
}, durs: true},
// Consumer settings. maxWait is stored as int32 milliseconds,
// but we want the error message to be in the nice
// time.Duration string format.
{name: "max fetch wait", v: int64(cfg.maxWait) * int64(time.Millisecond), allowed: int64(10 * time.Millisecond), badcmp: i64lt, durs: true},
// Group settings.
{name: "number of balancers", v: int64(len(cfg.balancers)), allowed: 1, badcmp: i64lt},
{name: "consumer protocol length", v: int64(len(cfg.protocol)), allowed: 1, badcmp: i64lt},
{name: "session timeout", v: int64(cfg.sessionTimeout), allowed: int64(100 * time.Millisecond), badcmp: i64lt, durs: true},
{name: "rebalance timeout", v: int64(cfg.rebalanceTimeout), allowed: int64(100 * time.Millisecond), badcmp: i64lt, durs: true},
{name: "autocommit interval", v: int64(cfg.autocommitInterval), allowed: int64(100 * time.Millisecond), badcmp: i64lt, durs: true},
{v: int64(cfg.heartbeatInterval), allowed: int64(cfg.rebalanceTimeout) * int64(time.Millisecond), badcmp: i64gt, durs: true, fmt: "heartbeat interval %v is erroneously larger than the session timeout %v"},
} {
bad, cmp := limit.badcmp(limit.v, limit.allowed)
if bad {
if limit.fmt != "" {
if limit.durs {
return fmt.Errorf(limit.fmt, time.Duration(limit.v), time.Duration(limit.allowed))
}
return fmt.Errorf(limit.fmt, limit.v, limit.allowed)
}
if limit.durs {
return fmt.Errorf("%s %v is %s than allowed %v", limit.name, time.Duration(limit.v), cmp, time.Duration(limit.allowed))
}
return fmt.Errorf("%s %v is %s than allowed %v", limit.name, limit.v, cmp, limit.allowed)
}
}
if cfg.dialFn != nil {
if cfg.dialTLS != nil {
return errors.New("cannot set both Dialer and DialTLSConfig")
}
}
if len(cfg.group) > 0 {
if len(cfg.partitions) != 0 {
return errors.New("invalid direct-partition consuming option when consuming as a group")
}
}
if cfg.regex {
if len(cfg.partitions) != 0 {
return errors.New("invalid direct-partition consuming option when consuming as regex")
}
for re := range cfg.topics {
compiled, err := regexp.Compile(re)
if err != nil {
return fmt.Errorf("invalid regular expression %q", re)
}
cfg.topics[re] = compiled
}
}
if cfg.topics != nil && cfg.partitions != nil {
for topic := range cfg.partitions {
if _, exists := cfg.topics[topic]; exists {
return fmt.Errorf("topic %q seen in both ConsumePartitions and ConsumeTopics; these options are a union, it is invalid to specify specific partitions for a topic while also consuming the entire topic", topic)
}
}
}
if cfg.autocommitDisable && cfg.autocommitGreedy {
return errors.New("cannot both disable autocommitting and enable greedy autocommitting")
}
if cfg.autocommitDisable && cfg.autocommitMarks {
return errors.New("cannot both disable autocommitting and enable marked autocommitting")
}
if cfg.autocommitGreedy && cfg.autocommitMarks {
return errors.New("cannot enable both greedy autocommitting and marked autocommitting")
}
if (cfg.autocommitGreedy || cfg.autocommitDisable || cfg.autocommitMarks || cfg.setCommitCallback) && len(cfg.group) == 0 {
return errors.New("invalid autocommit options specified when a group was not specified")
}
if (cfg.setLost || cfg.setRevoked || cfg.setAssigned) && len(cfg.group) == 0 {
return errors.New("invalid group partition assigned/revoked/lost functions set when a group was not specified")
}
processedHooks, err := processHooks(cfg.hooks)
if err != nil {
return err
}
cfg.hooks = processedHooks
return nil
}
// processHooks will inspect and recursively unpack slices of hooks stopping
// if the instance implements any hook interface. It will return an error on
// the first instance that implements no hook interface
func processHooks(hooks []Hook) ([]Hook, error) {
var processedHooks []Hook
for _, hook := range hooks {
if implementsAnyHook(hook) {
processedHooks = append(processedHooks, hook)
} else if moreHooks, ok := hook.([]Hook); ok {
more, err := processHooks(moreHooks)
if err != nil {
return nil, err
}
processedHooks = append(processedHooks, more...)
} else {
return nil, errors.New("found an argument that implements no hook interfaces")
}
}
return processedHooks, nil
}
var reVersion = regexp.MustCompile(`^[a-zA-Z0-9](?:[a-zA-Z0-9.-]*[a-zA-Z0-9])?$`)
func softwareVersion() string {
info, ok := debug.ReadBuildInfo()
if ok {
for _, dep := range info.Deps {
if dep.Path == "github.com/twmb/franz-go" {
if reVersion.MatchString(dep.Version) {
return dep.Version
}
}
}
}
return "unknown"
}
func defaultCfg() cfg {
defaultID := "kgo"
return cfg{
/////////////
// general //
/////////////
id: &defaultID,
dialTimeout: 10 * time.Second,
requestTimeoutOverhead: 10 * time.Second,
connIdleTimeout: 20 * time.Second,
softwareName: "kgo",
softwareVersion: softwareVersion(),
logger: new(nopLogger),
seedBrokers: []string{"127.0.0.1"},
maxVersions: kversion.Stable(),
retryBackoff: func() func(int) time.Duration {
var rngMu sync.Mutex
rng := rand.New(rand.NewSource(time.Now().UnixNano()))
return func(fails int) time.Duration {
const (
min = 250 * time.Millisecond
max = 5 * time.Second / 2
)
if fails <= 0 {
return min
}
if fails > 10 {
return max
}
backoff := min * time.Duration(1<<(fails-1))
rngMu.Lock()
jitter := 0.8 + 0.4*rng.Float64()
rngMu.Unlock()
backoff = time.Duration(float64(backoff) * jitter)
if backoff > max {
return max
}
return backoff
}
}(),
retries: 20,
maxBrokerWriteBytes: 100 << 20, // Kafka socket.request.max.bytes default is 100<<20
maxBrokerReadBytes: 100 << 20,
metadataMaxAge: 5 * time.Minute,
metadataMinAge: 5 * time.Second,
missingTopicDelete: 15 * time.Second,
//////////////
// producer //
//////////////
txnTimeout: 40 * time.Second,
acks: AllISRAcks(),
maxProduceInflight: 1,
compression: []CompressionCodec{SnappyCompression(), NoCompression()},
maxRecordBatchBytes: 1000012, // Kafka max.message.bytes default is 1000012
maxBufferedRecords: 10000,
produceTimeout: 10 * time.Second,
recordRetries: math.MaxInt64, // effectively unbounded
maxUnknownFailures: 4,
partitioner: UniformBytesPartitioner(64<<10, true, true, nil),
txnBackoff: 20 * time.Millisecond,
//////////////
// consumer //
//////////////
maxWait: 5000,
minBytes: 1,
maxBytes: 50 << 20,
maxPartBytes: 1 << 20,
resetOffset: NewOffset().AtStart(),
isolationLevel: 0,
maxConcurrentFetches: 0, // unbounded default
///////////
// group //
///////////
balancers: []GroupBalancer{
CooperativeStickyBalancer(),
},
protocol: "consumer",
sessionTimeout: 45000 * time.Millisecond,
rebalanceTimeout: 60000 * time.Millisecond,
heartbeatInterval: 3000 * time.Millisecond,
autocommitInterval: 5 * time.Second,
}
}
//////////////////////////
// CLIENT CONFIGURATION //
//////////////////////////
// ClientID uses id for all requests sent to Kafka brokers, overriding the
// default "kgo".
func ClientID(id string) Opt {
return clientOpt{func(cfg *cfg) { cfg.id = &id }}
}
// SoftwareNameAndVersion sets the client software name and version that will
// be sent to Kafka as part of the ApiVersions request as of Kafka 2.4,
// overriding the default "kgo" and internal version number.
//
// Kafka exposes this through metrics to help operators understand the impact
// of clients.
//
// It is generally not recommended to set this. As well, if you do, the name
// and version must match the following regular expression:
//
// [a-zA-Z0-9](?:[a-zA-Z0-9\.-]*[a-zA-Z0-9])?
//
// Note this means neither the name nor version can be empty.
func SoftwareNameAndVersion(name, version string) Opt {
return clientOpt{func(cfg *cfg) { cfg.softwareName = name; cfg.softwareVersion = version }}
}
// WithLogger sets the client to use the given logger, overriding the default
// to not use a logger.
//
// It is invalid to use a nil logger; doing so will cause panics.
func WithLogger(l Logger) Opt {
return clientOpt{func(cfg *cfg) { cfg.logger = &wrappedLogger{l} }}
}
// RequestTimeoutOverhead uses the given time as overhead while deadlining
// requests, overriding the default overhead of 10s.
//
// For most requests, the overhead will simply be this timeout. However, for
// any request with a TimeoutMillis field, the overhead is added on top of the
// request's TimeoutMillis. This ensures that we give Kafka enough time to
// actually process the request given the timeout, while still having a
// deadline on the connection as a whole to ensure it does not hang.
//
// For writes, the timeout is always the overhead. We buffer writes in our
// client before one quick flush, so we always expect the write to be fast.
//
// Note that hitting the timeout kills a connection, which will fail any other
// active writes or reads on the connection.
//
// This option is roughly equivalent to request.timeout.ms, but grants
// additional time to requests that have timeout fields.
func RequestTimeoutOverhead(overhead time.Duration) Opt {
return clientOpt{func(cfg *cfg) { cfg.requestTimeoutOverhead = overhead }}
}
// ConnIdleTimeout is a rough amount of time to allow connections to idle
// before they are closed, overriding the default 20.
//
// In the worst case, a connection can be allowed to idle for up to 2x this
// time, while the average is expected to be 1.5x (essentially, a uniform
// distribution from this interval to 2x the interval).
//
// It is possible that a connection can be reaped just as it is about to be
// written to, but the client internally retries in these cases.
//
// Connections are not reaped if they are actively being written to or read
// from; thus, a request can take a really long time itself and not be reaped
// (however, this may lead to the RequestTimeoutOverhead).
func ConnIdleTimeout(timeout time.Duration) Opt {
return clientOpt{func(cfg *cfg) { cfg.connIdleTimeout = timeout }}
}
// Dialer uses fn to dial addresses, overriding the default dialer that uses a
// 10s dial timeout and no TLS.
//
// The context passed to the dial function is the context used in the request
// that caused the dial. If the request is a client-internal request, the
// context is the context on the client itself (which is canceled when the
// client is closed).
//
// This function has the same signature as net.Dialer's DialContext and
// tls.Dialer's DialContext, meaning you can use this function like so:
//
// kgo.Dialer((&net.Dialer{Timeout: 10*time.Second}).DialContext)
//
// or
//
// kgo.Dialer((&tls.Dialer{...}).DialContext)
func Dialer(fn func(ctx context.Context, network, host string) (net.Conn, error)) Opt {
return clientOpt{func(cfg *cfg) { cfg.dialFn = fn }}
}
// DialTimeout sets the dial timeout, overriding the default of 10s. This
// option is useful if you do not want to set a custom dialer, and is useful in
// tandem with DialTLSConfig.
func DialTimeout(timeout time.Duration) Opt {
return clientOpt{func(cfg *cfg) { cfg.dialTimeout = timeout }}
}
// DialTLSConfig opts into dialing brokers with the given TLS config with a
// 10s dial timeout. This is a shortcut for manually specifying a tls dialer
// using the Dialer option. You can also change the default 10s timeout with
// DialTimeout.
//
// Every dial, the input config is cloned. If the config's ServerName is not
// specified, this function uses net.SplitHostPort to extract the host from the
// broker being dialed and sets the ServerName. In short, it is not necessary
// to set the ServerName.
func DialTLSConfig(c *tls.Config) Opt {
return clientOpt{func(cfg *cfg) { cfg.dialTLS = c }}
}
// DialTLS opts into dialing brokers with TLS. This is a shortcut for
// DialTLSConfig with an empty config. See DialTLSConfig for more details.
func DialTLS() Opt {
return DialTLSConfig(new(tls.Config))
}
// SeedBrokers sets the seed brokers for the client to use, overriding the
// default 127.0.0.1:9092.
//
// Any seeds that are missing a port use the default Kafka port 9092.
func SeedBrokers(seeds ...string) Opt {
return clientOpt{func(cfg *cfg) { cfg.seedBrokers = append(cfg.seedBrokers[:0], seeds...) }}
}
// MaxVersions sets the maximum Kafka version to try, overriding the
// internal unbounded (latest stable) versions.
//
// Note that specific max version pinning is required if trying to interact
// with versions pre 0.10.0. Otherwise, unless using more complicated requests
// that this client itself does not natively use, it is generally safe to opt
// for the latest version. If using the kmsg package directly to issue
// requests, it is recommended to pin versions so that new fields on requests
// do not get invalid default zero values before you update your usage.
func MaxVersions(versions *kversion.Versions) Opt {
return clientOpt{func(cfg *cfg) { cfg.maxVersions = versions }}
}
// MinVersions sets the minimum Kafka version a request can be downgraded to,
// overriding the default of the lowest version.
//
// This option is useful if you are issuing requests that you absolutely do not
// want to be downgraded; that is, if you are relying on features in newer
// requests, and you are not sure if your brokers can handle those features.
// By setting a min version, if the client detects it needs to downgrade past
// the version, it will instead avoid issuing the request.
//
// Unlike MaxVersions, if a request is issued that is unknown to the min
// versions, the request is allowed. It is assumed that there is no lower bound
// for that request.
func MinVersions(versions *kversion.Versions) Opt {
return clientOpt{func(cfg *cfg) { cfg.minVersions = versions }}
}
// RetryBackoffFn sets the backoff strategy for how long to backoff for a given
// amount of retries, overriding the default jittery exponential backoff that
// ranges from 250ms min to 2.5s max.
//
// This (roughly) corresponds to Kafka's retry.backoff.ms setting and
// retry.backoff.max.ms (which is being introduced with KIP-500).
func RetryBackoffFn(backoff func(int) time.Duration) Opt {
return clientOpt{func(cfg *cfg) { cfg.retryBackoff = backoff }}
}
// RequestRetries sets the number of tries that retryable requests are allowed,
// overriding the default of 20.
//
// This option does not apply to produce requests; to limit produce request
// retries / record retries, see RecordRetries.
func RequestRetries(n int) Opt {
return clientOpt{func(cfg *cfg) { cfg.retries = int64(n) }}
}
// RetryTimeout sets the upper limit on how long we allow requests to retry,
// overriding the default of:
//
// JoinGroup: cfg.SessionTimeout (default 45s)
// SyncGroup: cfg.SessionTimeout (default 45s)
// Heartbeat: cfg.SessionTimeout (default 45s)
// others: 30s
//
// This timeout applies to any request issued through a client's Request
// function. It does not apply to fetches nor produces.
//
// A value of zero indicates no request timeout.
//
// The timeout is evaluated after a request is issued. If a retry backoff
// places the next request past the retry timeout deadline, the request will
// still be tried once more once the backoff expires.
func RetryTimeout(t time.Duration) Opt {
return RetryTimeoutFn(func(int16) time.Duration { return t })
}
// RetryTimeoutFn sets the per-request upper limit on how long we allow
// requests to retry, overriding the default of:
//
// JoinGroup: cfg.SessionTimeout (default 45s)
// SyncGroup: cfg.SessionTimeout (default 45s)
// Heartbeat: cfg.SessionTimeout (default 45s)
// others: 30s
//
// This timeout applies to any request issued through a client's Request
// function. It does not apply to fetches nor produces.
//
// The function is called with the request key that is being retried. While it
// is not expected that the request key will be used, including it gives users
// the opportinuty to have different retry timeouts for different keys.
//
// If the function returns zero, there is no retry timeout.
//
// The timeout is evaluated after a request is issued. If a retry backoff
// places the next request past the retry timeout deadline, the request will
// still be tried once more once the backoff expires.
func RetryTimeoutFn(t func(int16) time.Duration) Opt {
return clientOpt{func(cfg *cfg) { cfg.retryTimeout = t }}
}
// AllowAutoTopicCreation enables topics to be auto created if they do
// not exist when fetching their metadata.
func AllowAutoTopicCreation() Opt {
return clientOpt{func(cfg *cfg) { cfg.allowAutoTopicCreation = true }}
}
// BrokerMaxWriteBytes upper bounds the number of bytes written to a broker
// connection in a single write, overriding the default 100MiB.
//
// This number corresponds to the a broker's socket.request.max.bytes, which
// defaults to 100MiB.
//
// The only Kafka request that could come reasonable close to hitting this
// limit should be produce requests, and thus this limit is only enforced for
// produce requests.
func BrokerMaxWriteBytes(v int32) Opt {
return clientOpt{func(cfg *cfg) { cfg.maxBrokerWriteBytes = v }}
}
// BrokerMaxReadBytes sets the maximum response size that can be read from
// Kafka, overriding the default 100MiB.
//
// This is a safety measure to avoid OOMing on invalid responses. This is
// slightly double FetchMaxBytes; if bumping that, consider bump this. No other
// response should run the risk of hitting this limit.
func BrokerMaxReadBytes(v int32) Opt {
return clientOpt{func(cfg *cfg) { cfg.maxBrokerReadBytes = v }}
}
// MetadataMaxAge sets the maximum age for the client's cached metadata,
// overriding the default 5m, to allow detection of new topics, partitions,
// etc.
//
// This corresponds to Kafka's metadata.max.age.ms.
func MetadataMaxAge(age time.Duration) Opt {
return clientOpt{func(cfg *cfg) { cfg.metadataMaxAge = age }}
}
// MetadataMinAge sets the minimum time between metadata queries, overriding
// the default 5s. You may want to raise or lower this to reduce the number of
// metadata queries the client will make. Notably, if metadata detects an error
// in any topic or partition, it triggers itself to update as soon as allowed.
func MetadataMinAge(age time.Duration) Opt {
return clientOpt{func(cfg *cfg) { cfg.metadataMinAge = age }}
}
// SASL appends sasl authentication options to use for all connections.
//
// SASL is tried in order; if the broker supports the first mechanism, all
// connections will use that mechanism. If the first mechanism fails, the
// client will pick the first supported mechanism. If the broker does not
// support any client mechanisms, connections will fail.
func SASL(sasls ...sasl.Mechanism) Opt {
return clientOpt{func(cfg *cfg) { cfg.sasls = append(cfg.sasls, sasls...) }}
}
// WithHooks sets hooks to call whenever relevant.
//
// Hooks can be used to layer in metrics (such as Prometheus hooks) or anything
// else. The client will call all hooks in order. See the Hooks interface for
// more information, as well as any interface that contains "Hook" in the name
// to know the available hooks. A single hook can implement zero or all hook
// interfaces, and only the hooks that it implements will be called.
func WithHooks(hooks ...Hook) Opt {
return clientOpt{func(cfg *cfg) { cfg.hooks = append(cfg.hooks, hooks...) }}
}
// ConcurrentTransactionsBackoff sets the backoff interval to use during
// transactional requests in case we encounter CONCURRENT_TRANSACTIONS error,
// overriding the default 20ms.
//
// Sometimes, when a client begins a transaction quickly enough after finishing
// a previous one, Kafka will return a CONCURRENT_TRANSACTIONS error. Clients
// are expected to backoff slightly and retry the operation. Lower backoffs may
// increase load on the brokers, while higher backoffs may increase transaction
// latency in clients.
//
// Note that if brokers are hanging in this concurrent transactions state for
// too long, the client progressively increases the backoff.
func ConcurrentTransactionsBackoff(backoff time.Duration) Opt {
return clientOpt{func(cfg *cfg) { cfg.txnBackoff = backoff }}
}
// ConsiderMissingTopicDeletedAfter sets the amount of time a topic can be
// missing from metadata responses _after_ loading it at least once before it
// is considered deleted, overriding the default of 15s. Note that for newer
// versions of Kafka, it may take a bit of time (~15s) for the cluster to fully
// recognize a newly created topic. If this option is set too low, there is
// some risk that the client will internally purge and re-see a topic a few
// times until the cluster fully broadcasts the topic creation.
func ConsiderMissingTopicDeletedAfter(t time.Duration) Opt {
return clientOpt{func(cfg *cfg) { cfg.missingTopicDelete = t }}
}
////////////////////////////
// PRODUCER CONFIGURATION //
////////////////////////////
// DefaultProduceTopic sets the default topic to produce to if the topic field
// is empty in a Record.
//
// If this option is not used, if a record has an empty topic, the record
// cannot be produced and will be failed immediately.
func DefaultProduceTopic(t string) ProducerOpt {
return producerOpt{func(cfg *cfg) { cfg.defaultProduceTopic = t }}
}
// Acks represents the number of acks a broker leader must have before
// a produce request is considered complete.
//
// This controls the durability of written records and corresponds to "acks" in
// Kafka's Producer Configuration documentation.
//
// The default is LeaderAck.
type Acks struct {
val int16
}
// NoAck considers records sent as soon as they are written on the wire.
// The leader does not reply to records.
func NoAck() Acks { return Acks{0} }
// LeaderAck causes Kafka to reply that a record is written after only
// the leader has written a message. The leader does not wait for in-sync
// replica replies.
func LeaderAck() Acks { return Acks{1} }
// AllISRAcks ensures that all in-sync replicas have acknowledged they
// wrote a record before the leader replies success.
func AllISRAcks() Acks { return Acks{-1} }
// RequiredAcks sets the required acks for produced records,
// overriding the default RequireAllISRAcks.
func RequiredAcks(acks Acks) ProducerOpt {
return producerOpt{func(cfg *cfg) { cfg.acks = acks }}
}
// DisableIdempotentWrite disables idempotent produce requests, opting out of
// Kafka server-side deduplication in the face of reissued requests due to
// transient network problems. Disabling idempotent write by default
// upper-bounds the number of in-flight produce requests per broker to 1, vs.
// the default of 5 when using idempotency.
//
// Idempotent production is strictly a win, but does require the
// IDEMPOTENT_WRITE permission on CLUSTER (pre Kafka 3.0), and not all clients
// can have that permission.
//
// This option is incompatible with specifying a transactional id.
func DisableIdempotentWrite() ProducerOpt {
return producerOpt{func(cfg *cfg) { cfg.disableIdempotency = true }}
}
// MaxProduceRequestsInflightPerBroker changes the number of allowed produce
// requests in flight per broker if you disable idempotency, overriding the
// default of 1. If using idempotency, this option has no effect: the maximum
// in flight for Kafka v0.11 is 1, and from v1 onward is 5.
//
// Using more than 1 may result in out of order records and may result in
// duplicates if there are connection issues.
func MaxProduceRequestsInflightPerBroker(n int) ProducerOpt {
return producerOpt{func(cfg *cfg) { cfg.maxProduceInflight = n }}
}
// ProducerBatchCompression sets the compression codec to use for producing
// records.
//
// Compression is chosen in the order preferred based on broker support. For
// example, zstd compression was introduced in Kafka 2.1, so the preference
// can be first zstd, fallback snappy, fallback none.
//
// The default preference is [snappy, none], which should be fine for all old
// consumers since snappy compression has existed since Kafka 0.8.0. To use
// zstd, your brokers must be at least 2.1 and all consumers must be upgraded
// to support decoding zstd records.
func ProducerBatchCompression(preference ...CompressionCodec) ProducerOpt {
return producerOpt{func(cfg *cfg) { cfg.compression = preference }}
}
// ProducerBatchMaxBytes upper bounds the size of a record batch, overriding
// the default 1,000,012 bytes. This mirrors Kafka's max.message.bytes.
//
// Record batches are independent of a ProduceRequest: a record batch is
// specific to a topic and partition, whereas the produce request can contain
// many record batches for many topics.
//
// If a single record encodes larger than this number (before compression), it
// will will not be written and a callback will have the appropriate error.
//
// Note that this is the maximum size of a record batch before compression. If
// a batch compresses poorly and actually grows the batch, the uncompressed
// form will be used.
func ProducerBatchMaxBytes(v int32) ProducerOpt {
return producerOpt{func(cfg *cfg) { cfg.maxRecordBatchBytes = v }}
}
// MaxBufferedRecords sets the max amount of records the client will buffer,
// blocking produces until records are finished if this limit is reached.
// This overrides the default of 10,000.
func MaxBufferedRecords(n int) ProducerOpt {
return producerOpt{func(cfg *cfg) { cfg.maxBufferedRecords = int64(n) }}
}
// MaxBufferedBytes sets the max amount of bytes that the client will buffer
// while producing, blocking produces until records are finished if this limit
// is reached. This overrides the unlimited default.
//
// Note that this option does _not_ apply for consuming: the client cannot
// limit bytes buffered for consuming because of decompression. You can roughly
// control consuming memory by using [MaxConcurrentFetches], [FetchMaxBytes],
// and [FetchMaxPartitionBytes].
//
// If you produce a record that is larger than n, the record is immediately
// failed with kerr.MessageTooLarge.
//
// Note that this limit applies after [MaxBufferedRecords].
func MaxBufferedBytes(n int) ProducerOpt {
return producerOpt{func(cfg *cfg) { cfg.maxBufferedBytes = int64(n) }}
}
// RecordPartitioner uses the given partitioner to partition records, overriding
// the default UniformBytesPartitioner(64KiB, true, true, nil).
func RecordPartitioner(partitioner Partitioner) ProducerOpt {
return producerOpt{func(cfg *cfg) { cfg.partitioner = partitioner }}
}
// ProduceRequestTimeout sets how long Kafka broker's are allowed to respond to
// produce requests, overriding the default 10s. If a broker exceeds this
// duration, it will reply with a request timeout error.
//
// This somewhat corresponds to Kafka's request.timeout.ms setting, but only
// applies to produce requests. This settings sets the TimeoutMillis field in
// the produce request itself. The RequestTimeoutOverhead is applied as a write
// limit and read limit in addition to this.
func ProduceRequestTimeout(limit time.Duration) ProducerOpt {
return producerOpt{func(cfg *cfg) { cfg.produceTimeout = limit }}
}
// RecordRetries sets the number of tries for producing records, overriding the
// unlimited default.
//
// If idempotency is enabled (as it is by default), this option is only
// enforced if it is safe to do so without creating invalid sequence numbers.
// It is safe to enforce if a record was never issued in a request to Kafka, or
// if it was requested and received a response.
//
// If a record fails due to retries, all records buffered in the same partition