forked from segmentio/kafka-go
-
Notifications
You must be signed in to change notification settings - Fork 0
/
conn.go
1687 lines (1493 loc) · 46.9 KB
/
conn.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 kafka
import (
"bufio"
"errors"
"fmt"
"io"
"math"
"math/rand"
"net"
"os"
"path/filepath"
"sync"
"sync/atomic"
"time"
)
var (
errInvalidWriteTopic = errors.New("writes must NOT set Topic on kafka.Message")
errInvalidWritePartition = errors.New("writes must NOT set Partition on kafka.Message")
)
// Conn represents a connection to a kafka broker.
//
// Instances of Conn are safe to use concurrently from multiple goroutines.
type Conn struct {
// base network connection
conn net.Conn
// number of inflight requests on the connection.
inflight int32
// offset management (synchronized on the mutex field)
mutex sync.Mutex
offset int64
// read buffer (synchronized on rlock)
rlock sync.Mutex
rbuf bufio.Reader
// write buffer (synchronized on wlock)
wlock sync.Mutex
wbuf bufio.Writer
wb writeBuffer
// sasl session
saslSessionDeadline time.Time
saslAuth func() error
// deadline management
wdeadline connDeadline
rdeadline connDeadline
// immutable values of the connection object
clientID string
topic string
partition int32
fetchMaxBytes int32
fetchMinSize int32
broker int32
rack string
// correlation ID generator (synchronized on wlock)
correlationID int32
// number of replica acks required when publishing to a partition
requiredAcks int32
// lazily loaded API versions used by this connection
apiVersions atomic.Value // apiVersionMap
transactionalID *string
}
type apiVersionMap map[apiKey]ApiVersion
func (v apiVersionMap) negotiate(key apiKey, sortedSupportedVersions ...apiVersion) apiVersion {
x := v[key]
for i := len(sortedSupportedVersions) - 1; i >= 0; i-- {
s := sortedSupportedVersions[i]
if apiVersion(x.MaxVersion) >= s {
return s
}
}
return -1
}
// ConnConfig is a configuration object used to create new instances of Conn.
type ConnConfig struct {
ClientID string
Topic string
Partition int
Broker int
Rack string
// The transactional id to use for transactional delivery. Idempotent
// deliver should be enabled if transactional id is configured.
// For more details look at transactional.id description here: http://kafka.apache.org/documentation.html#producerconfigs
// Empty string means that this connection can't be transactional.
TransactionalID string
}
// ReadBatchConfig is a configuration object used for reading batches of messages.
type ReadBatchConfig struct {
// MinBytes indicates to the broker the minimum batch size that the consumer
// will accept. Setting a high minimum when consuming from a low-volume topic
// may result in delayed delivery when the broker does not have enough data to
// satisfy the defined minimum.
MinBytes int
// MaxBytes indicates to the broker the maximum batch size that the consumer
// will accept. The broker will truncate a message to satisfy this maximum, so
// choose a value that is high enough for your largest message size.
MaxBytes int
// IsolationLevel controls the visibility of transactional records.
// ReadUncommitted makes all records visible. With ReadCommitted only
// non-transactional and committed records are visible.
IsolationLevel IsolationLevel
// MaxWait is the amount of time for the broker while waiting to hit the
// min/max byte targets. This setting is independent of any network-level
// timeouts or deadlines.
//
// For backward compatibility, when this field is left zero, kafka-go will
// infer the max wait from the connection's read deadline.
MaxWait time.Duration
}
type IsolationLevel int8
const (
ReadUncommitted IsolationLevel = 0
ReadCommitted IsolationLevel = 1
)
var (
// DefaultClientID is the default value used as ClientID of kafka
// connections.
DefaultClientID string
)
func init() {
progname := filepath.Base(os.Args[0])
hostname, _ := os.Hostname()
DefaultClientID = fmt.Sprintf("%s@%s (github.com/segmentio/kafka-go)", progname, hostname)
}
// NewConn returns a new kafka connection for the given topic and partition.
func NewConn(conn net.Conn, topic string, partition int) *Conn {
return NewConnWith(conn, ConnConfig{
Topic: topic,
Partition: partition,
})
}
func emptyToNullable(transactionalID string) (result *string) {
if transactionalID != "" {
result = &transactionalID
}
return result
}
// NewConnWith returns a new kafka connection configured with config.
// The offset is initialized to FirstOffset.
func NewConnWith(conn net.Conn, config ConnConfig) *Conn {
if len(config.ClientID) == 0 {
config.ClientID = DefaultClientID
}
if config.Partition < 0 || config.Partition > math.MaxInt32 {
panic(fmt.Sprintf("invalid partition number: %d", config.Partition))
}
c := &Conn{
conn: conn,
rbuf: *bufio.NewReader(conn),
wbuf: *bufio.NewWriter(conn),
clientID: config.ClientID,
topic: config.Topic,
partition: int32(config.Partition),
broker: int32(config.Broker),
rack: config.Rack,
offset: FirstOffset,
requiredAcks: -1,
transactionalID: emptyToNullable(config.TransactionalID),
}
c.wb.w = &c.wbuf
// The fetch request needs to ask for a MaxBytes value that is at least
// enough to load the control data of the response. To avoid having to
// recompute it on every read, it is cached here in the Conn value.
c.fetchMinSize = (fetchResponseV2{
Topics: []fetchResponseTopicV2{{
TopicName: config.Topic,
Partitions: []fetchResponsePartitionV2{{
Partition: int32(config.Partition),
MessageSet: messageSet{{}},
}},
}},
}).size()
c.fetchMaxBytes = math.MaxInt32 - c.fetchMinSize
return c
}
func (c *Conn) negotiateVersion(key apiKey, sortedSupportedVersions ...apiVersion) (apiVersion, error) {
v, err := c.loadVersions()
if err != nil {
return -1, err
}
a := v.negotiate(key, sortedSupportedVersions...)
if a < 0 {
return -1, fmt.Errorf("no matching versions were found between the client and the broker for API key %d", key)
}
return a, nil
}
func (c *Conn) loadVersions() (apiVersionMap, error) {
v, _ := c.apiVersions.Load().(apiVersionMap)
if v != nil {
return v, nil
}
brokerVersions, err := c.ApiVersions()
if err != nil {
return nil, err
}
v = make(apiVersionMap, len(brokerVersions))
for _, a := range brokerVersions {
v[apiKey(a.ApiKey)] = a
}
c.apiVersions.Store(v)
return v, nil
}
// Broker returns a Broker value representing the kafka broker that this
// connection was established to.
func (c *Conn) Broker() Broker {
addr := c.conn.RemoteAddr()
host, port, _ := splitHostPortNumber(addr.String())
return Broker{
Host: host,
Port: port,
ID: int(c.broker),
Rack: c.rack,
}
}
// Controller requests kafka for the current controller and returns its URL.
func (c *Conn) Controller() (broker Broker, err error) {
err = c.readOperation(
func(deadline time.Time, id int32) error {
return c.writeRequest(metadata, v1, id, topicMetadataRequestV1([]string{}))
},
func(deadline time.Time, size int) error {
var res metadataResponseV1
if err := c.readResponse(size, &res); err != nil {
return err
}
for _, brokerMeta := range res.Brokers {
if brokerMeta.NodeID == res.ControllerID {
broker = Broker{ID: int(brokerMeta.NodeID),
Port: int(brokerMeta.Port),
Host: brokerMeta.Host,
Rack: brokerMeta.Rack}
break
}
}
return nil
},
)
return broker, err
}
// Brokers retrieve the broker list from the Kafka metadata.
func (c *Conn) Brokers() ([]Broker, error) {
var brokers []Broker
err := c.readOperation(
func(deadline time.Time, id int32) error {
return c.writeRequest(metadata, v1, id, topicMetadataRequestV1([]string{}))
},
func(deadline time.Time, size int) error {
var res metadataResponseV1
if err := c.readResponse(size, &res); err != nil {
return err
}
brokers = make([]Broker, len(res.Brokers))
for i, brokerMeta := range res.Brokers {
brokers[i] = Broker{
ID: int(brokerMeta.NodeID),
Port: int(brokerMeta.Port),
Host: brokerMeta.Host,
Rack: brokerMeta.Rack,
}
}
return nil
},
)
return brokers, err
}
// DeleteTopics deletes the specified topics.
func (c *Conn) DeleteTopics(topics ...string) error {
_, err := c.deleteTopics(deleteTopicsRequestV0{
Topics: topics,
})
return err
}
// findCoordinator finds the coordinator for the specified group or transaction
//
// See http://kafka.apache.org/protocol.html#The_Messages_FindCoordinator
func (c *Conn) findCoordinator(request findCoordinatorRequestV0) (findCoordinatorResponseV0, error) {
var response findCoordinatorResponseV0
err := c.readOperation(
func(deadline time.Time, id int32) error {
return c.writeRequest(findCoordinator, v0, id, request)
},
func(deadline time.Time, size int) error {
return expectZeroSize(func() (remain int, err error) {
return (&response).readFrom(&c.rbuf, size)
}())
},
)
if err != nil {
return findCoordinatorResponseV0{}, err
}
if response.ErrorCode != 0 {
return findCoordinatorResponseV0{}, Error(response.ErrorCode)
}
return response, nil
}
// heartbeat sends a heartbeat message required by consumer groups
//
// See http://kafka.apache.org/protocol.html#The_Messages_Heartbeat
func (c *Conn) heartbeat(request heartbeatRequestV0) (heartbeatResponseV0, error) {
var response heartbeatResponseV0
err := c.writeOperation(
func(deadline time.Time, id int32) error {
return c.writeRequest(heartbeat, v0, id, request)
},
func(deadline time.Time, size int) error {
return expectZeroSize(func() (remain int, err error) {
return (&response).readFrom(&c.rbuf, size)
}())
},
)
if err != nil {
return heartbeatResponseV0{}, err
}
if response.ErrorCode != 0 {
return heartbeatResponseV0{}, Error(response.ErrorCode)
}
return response, nil
}
// joinGroup attempts to join a consumer group
//
// See http://kafka.apache.org/protocol.html#The_Messages_JoinGroup
func (c *Conn) joinGroup(request joinGroupRequestV1) (joinGroupResponseV1, error) {
var response joinGroupResponseV1
err := c.writeOperation(
func(deadline time.Time, id int32) error {
return c.writeRequest(joinGroup, v1, id, request)
},
func(deadline time.Time, size int) error {
return expectZeroSize(func() (remain int, err error) {
return (&response).readFrom(&c.rbuf, size)
}())
},
)
if err != nil {
return joinGroupResponseV1{}, err
}
if response.ErrorCode != 0 {
return joinGroupResponseV1{}, Error(response.ErrorCode)
}
return response, nil
}
// leaveGroup leaves the consumer from the consumer group
//
// See http://kafka.apache.org/protocol.html#The_Messages_LeaveGroup
func (c *Conn) leaveGroup(request leaveGroupRequestV0) (leaveGroupResponseV0, error) {
var response leaveGroupResponseV0
err := c.writeOperation(
func(deadline time.Time, id int32) error {
return c.writeRequest(leaveGroup, v0, id, request)
},
func(deadline time.Time, size int) error {
return expectZeroSize(func() (remain int, err error) {
return (&response).readFrom(&c.rbuf, size)
}())
},
)
if err != nil {
return leaveGroupResponseV0{}, err
}
if response.ErrorCode != 0 {
return leaveGroupResponseV0{}, Error(response.ErrorCode)
}
return response, nil
}
// listGroups lists all the consumer groups
//
// See http://kafka.apache.org/protocol.html#The_Messages_ListGroups
func (c *Conn) listGroups(request listGroupsRequestV1) (listGroupsResponseV1, error) {
var response listGroupsResponseV1
err := c.readOperation(
func(deadline time.Time, id int32) error {
return c.writeRequest(listGroups, v1, id, request)
},
func(deadline time.Time, size int) error {
return expectZeroSize(func() (remain int, err error) {
return (&response).readFrom(&c.rbuf, size)
}())
},
)
if err != nil {
return listGroupsResponseV1{}, err
}
if response.ErrorCode != 0 {
return listGroupsResponseV1{}, Error(response.ErrorCode)
}
return response, nil
}
// offsetCommit commits the specified topic partition offsets
//
// See http://kafka.apache.org/protocol.html#The_Messages_OffsetCommit
func (c *Conn) offsetCommit(request offsetCommitRequestV2) (offsetCommitResponseV2, error) {
var response offsetCommitResponseV2
err := c.writeOperation(
func(deadline time.Time, id int32) error {
return c.writeRequest(offsetCommit, v2, id, request)
},
func(deadline time.Time, size int) error {
return expectZeroSize(func() (remain int, err error) {
return (&response).readFrom(&c.rbuf, size)
}())
},
)
if err != nil {
return offsetCommitResponseV2{}, err
}
for _, r := range response.Responses {
for _, pr := range r.PartitionResponses {
if pr.ErrorCode != 0 {
return offsetCommitResponseV2{}, Error(pr.ErrorCode)
}
}
}
return response, nil
}
// offsetFetch fetches the offsets for the specified topic partitions.
// -1 indicates that there is no offset saved for the partition.
//
// See http://kafka.apache.org/protocol.html#The_Messages_OffsetFetch
func (c *Conn) offsetFetch(request offsetFetchRequestV1) (offsetFetchResponseV1, error) {
var response offsetFetchResponseV1
err := c.readOperation(
func(deadline time.Time, id int32) error {
return c.writeRequest(offsetFetch, v1, id, request)
},
func(deadline time.Time, size int) error {
return expectZeroSize(func() (remain int, err error) {
return (&response).readFrom(&c.rbuf, size)
}())
},
)
if err != nil {
return offsetFetchResponseV1{}, err
}
for _, r := range response.Responses {
for _, pr := range r.PartitionResponses {
if pr.ErrorCode != 0 {
return offsetFetchResponseV1{}, Error(pr.ErrorCode)
}
}
}
return response, nil
}
// syncGroup completes the handshake to join a consumer group
//
// See http://kafka.apache.org/protocol.html#The_Messages_SyncGroup
func (c *Conn) syncGroup(request syncGroupRequestV0) (syncGroupResponseV0, error) {
var response syncGroupResponseV0
err := c.readOperation(
func(deadline time.Time, id int32) error {
return c.writeRequest(syncGroup, v0, id, request)
},
func(deadline time.Time, size int) error {
return expectZeroSize(func() (remain int, err error) {
return (&response).readFrom(&c.rbuf, size)
}())
},
)
if err != nil {
return syncGroupResponseV0{}, err
}
if response.ErrorCode != 0 {
return syncGroupResponseV0{}, Error(response.ErrorCode)
}
return response, nil
}
// Close closes the kafka connection.
func (c *Conn) Close() error {
return c.conn.Close()
}
// LocalAddr returns the local network address.
func (c *Conn) LocalAddr() net.Addr {
return c.conn.LocalAddr()
}
// RemoteAddr returns the remote network address.
func (c *Conn) RemoteAddr() net.Addr {
return c.conn.RemoteAddr()
}
// SetDeadline sets the read and write deadlines associated with the connection.
// It is equivalent to calling both SetReadDeadline and SetWriteDeadline.
//
// A deadline is an absolute time after which I/O operations fail with a timeout
// (see type Error) instead of blocking. The deadline applies to all future and
// pending I/O, not just the immediately following call to Read or Write. After
// a deadline has been exceeded, the connection may be closed if it was found to
// be in an unrecoverable state.
//
// A zero value for t means I/O operations will not time out.
func (c *Conn) SetDeadline(t time.Time) error {
c.rdeadline.setDeadline(t)
c.wdeadline.setDeadline(t)
return nil
}
// SetReadDeadline sets the deadline for future Read calls and any
// currently-blocked Read call.
// A zero value for t means Read will not time out.
func (c *Conn) SetReadDeadline(t time.Time) error {
c.rdeadline.setDeadline(t)
return nil
}
// SetWriteDeadline sets the deadline for future Write calls and any
// currently-blocked Write call.
// Even if write times out, it may return n > 0, indicating that some of the
// data was successfully written.
// A zero value for t means Write will not time out.
func (c *Conn) SetWriteDeadline(t time.Time) error {
c.wdeadline.setDeadline(t)
return nil
}
// Offset returns the current offset of the connection as pair of integers,
// where the first one is an offset value and the second one indicates how
// to interpret it.
//
// See Seek for more details about the offset and whence values.
func (c *Conn) Offset() (offset int64, whence int) {
c.mutex.Lock()
offset = c.offset
c.mutex.Unlock()
switch offset {
case FirstOffset:
offset = 0
whence = SeekStart
case LastOffset:
offset = 0
whence = SeekEnd
default:
whence = SeekAbsolute
}
return
}
const (
SeekStart = 0 // Seek relative to the first offset available in the partition.
SeekAbsolute = 1 // Seek to an absolute offset.
SeekEnd = 2 // Seek relative to the last offset available in the partition.
SeekCurrent = 3 // Seek relative to the current offset.
// This flag may be combined to any of the SeekAbsolute and SeekCurrent
// constants to skip the bound check that the connection would do otherwise.
// Programs can use this flag to avoid making a metadata request to the kafka
// broker to read the current first and last offsets of the partition.
SeekDontCheck = 1 << 30
)
// Seek sets the offset for the next read or write operation according to whence, which
// should be one of SeekStart, SeekAbsolute, SeekEnd, or SeekCurrent.
// When seeking relative to the end, the offset is subtracted from the current offset.
// Note that for historical reasons, these do not align with the usual whence constants
// as in lseek(2) or os.Seek.
// The method returns the new absolute offset of the connection.
func (c *Conn) Seek(offset int64, whence int) (int64, error) {
seekDontCheck := (whence & SeekDontCheck) != 0
whence &= ^SeekDontCheck
switch whence {
case SeekStart, SeekAbsolute, SeekEnd, SeekCurrent:
default:
return 0, fmt.Errorf("whence must be one of 0, 1, 2, or 3. (whence = %d)", whence)
}
if seekDontCheck {
if whence == SeekAbsolute {
c.mutex.Lock()
c.offset = offset
c.mutex.Unlock()
return offset, nil
}
if whence == SeekCurrent {
c.mutex.Lock()
c.offset += offset
offset = c.offset
c.mutex.Unlock()
return offset, nil
}
}
if whence == SeekAbsolute {
c.mutex.Lock()
unchanged := offset == c.offset
c.mutex.Unlock()
if unchanged {
return offset, nil
}
}
if whence == SeekCurrent {
c.mutex.Lock()
offset = c.offset + offset
c.mutex.Unlock()
}
first, last, err := c.ReadOffsets()
if err != nil {
return 0, err
}
switch whence {
case SeekStart:
offset = first + offset
case SeekEnd:
offset = last - offset
}
if offset < first || offset > last {
return 0, OffsetOutOfRange
}
c.mutex.Lock()
c.offset = offset
c.mutex.Unlock()
return offset, nil
}
// Read reads the message at the current offset from the connection, advancing
// the offset on success so the next call to a read method will produce the next
// message.
// The method returns the number of bytes read, or an error if something went
// wrong.
//
// While it is safe to call Read concurrently from multiple goroutines it may
// be hard for the program to predict the results as the connection offset will
// be read and written by multiple goroutines, they could read duplicates, or
// messages may be seen by only some of the goroutines.
//
// The method fails with io.ErrShortBuffer if the buffer passed as argument is
// too small to hold the message value.
//
// This method is provided to satisfy the net.Conn interface but is much less
// efficient than using the more general purpose ReadBatch method.
func (c *Conn) Read(b []byte) (int, error) {
batch := c.ReadBatch(1, len(b))
n, err := batch.Read(b)
return n, coalesceErrors(silentEOF(err), batch.Close())
}
// ReadMessage reads the message at the current offset from the connection,
// advancing the offset on success so the next call to a read method will
// produce the next message.
//
// Because this method allocate memory buffers for the message key and value
// it is less memory-efficient than Read, but has the advantage of never
// failing with io.ErrShortBuffer.
//
// While it is safe to call Read concurrently from multiple goroutines it may
// be hard for the program to predict the results as the connection offset will
// be read and written by multiple goroutines, they could read duplicates, or
// messages may be seen by only some of the goroutines.
//
// This method is provided for convenience purposes but is much less efficient
// than using the more general purpose ReadBatch method.
func (c *Conn) ReadMessage(maxBytes int) (Message, error) {
batch := c.ReadBatch(1, maxBytes)
msg, err := batch.ReadMessage()
return msg, coalesceErrors(silentEOF(err), batch.Close())
}
// ReadBatch reads a batch of messages from the kafka server. The method always
// returns a non-nil Batch value. If an error occurred, either sending the fetch
// request or reading the response, the error will be made available by the
// returned value of the batch's Close method.
//
// While it is safe to call ReadBatch concurrently from multiple goroutines it
// may be hard for the program to predict the results as the connection offset
// will be read and written by multiple goroutines, they could read duplicates,
// or messages may be seen by only some of the goroutines.
//
// A program doesn't specify the number of messages in wants from a batch, but
// gives the minimum and maximum number of bytes that it wants to receive from
// the kafka server.
func (c *Conn) ReadBatch(minBytes, maxBytes int) *Batch {
return c.ReadBatchWith(ReadBatchConfig{
MinBytes: minBytes,
MaxBytes: maxBytes,
})
}
// ReadBatchWith in every way is similar to ReadBatch. ReadBatch is configured
// with the default values in ReadBatchConfig except for minBytes and maxBytes.
func (c *Conn) ReadBatchWith(cfg ReadBatchConfig) *Batch {
var adjustedDeadline time.Time
var maxFetch = int(c.fetchMaxBytes)
if cfg.MinBytes < 0 || cfg.MinBytes > maxFetch {
return &Batch{err: fmt.Errorf("kafka.(*Conn).ReadBatch: minBytes of %d out of [1,%d] bounds", cfg.MinBytes, maxFetch)}
}
if cfg.MaxBytes < 0 || cfg.MaxBytes > maxFetch {
return &Batch{err: fmt.Errorf("kafka.(*Conn).ReadBatch: maxBytes of %d out of [1,%d] bounds", cfg.MaxBytes, maxFetch)}
}
if cfg.MinBytes > cfg.MaxBytes {
return &Batch{err: fmt.Errorf("kafka.(*Conn).ReadBatch: minBytes (%d) > maxBytes (%d)", cfg.MinBytes, cfg.MaxBytes)}
}
offset, whence := c.Offset()
offset, err := c.Seek(offset, whence|SeekDontCheck)
if err != nil {
return &Batch{err: dontExpectEOF(err)}
}
fetchVersion, err := c.negotiateVersion(fetch, v2, v5, v10)
if err != nil {
return &Batch{err: dontExpectEOF(err)}
}
id, err := c.doRequest(&c.rdeadline, func(deadline time.Time, id int32) error {
now := time.Now()
var timeout time.Duration
if cfg.MaxWait > 0 {
// explicitly-configured case: no changes are made to the deadline,
// and the timeout is sent exactly as specified.
timeout = cfg.MaxWait
} else {
// default case: use the original logic to adjust the conn's
// deadline.T
deadline = adjustDeadlineForRTT(deadline, now, defaultRTT)
timeout = deadlineToTimeout(deadline, now)
}
// save this variable outside of the closure for later use in detecting
// truncated messages.
adjustedDeadline = deadline
switch fetchVersion {
case v10:
return c.wb.writeFetchRequestV10(
id,
c.clientID,
c.topic,
c.partition,
offset,
cfg.MinBytes,
cfg.MaxBytes+int(c.fetchMinSize),
timeout,
int8(cfg.IsolationLevel),
)
case v5:
return c.wb.writeFetchRequestV5(
id,
c.clientID,
c.topic,
c.partition,
offset,
cfg.MinBytes,
cfg.MaxBytes+int(c.fetchMinSize),
timeout,
int8(cfg.IsolationLevel),
)
default:
return c.wb.writeFetchRequestV2(
id,
c.clientID,
c.topic,
c.partition,
offset,
cfg.MinBytes,
cfg.MaxBytes+int(c.fetchMinSize),
timeout,
)
}
})
if err != nil {
return &Batch{err: dontExpectEOF(err)}
}
_, size, lock, err := c.waitResponse(&c.rdeadline, id)
if err != nil {
return &Batch{err: dontExpectEOF(err)}
}
var throttle int32
var highWaterMark int64
var remain int
switch fetchVersion {
case v10:
throttle, highWaterMark, remain, err = readFetchResponseHeaderV10(&c.rbuf, size)
case v5:
throttle, highWaterMark, remain, err = readFetchResponseHeaderV5(&c.rbuf, size)
default:
throttle, highWaterMark, remain, err = readFetchResponseHeaderV2(&c.rbuf, size)
}
if errors.Is(err, errShortRead) {
err = checkTimeoutErr(adjustedDeadline)
}
var msgs *messageSetReader
if err == nil {
if highWaterMark == offset {
msgs = &messageSetReader{empty: true}
} else {
msgs, err = newMessageSetReader(&c.rbuf, remain)
}
}
if errors.Is(err, errShortRead) {
err = checkTimeoutErr(adjustedDeadline)
}
return &Batch{
conn: c,
msgs: msgs,
deadline: adjustedDeadline,
throttle: makeDuration(throttle),
lock: lock,
topic: c.topic, // topic is copied to Batch to prevent race with Batch.close
partition: int(c.partition), // partition is copied to Batch to prevent race with Batch.close
offset: offset,
highWaterMark: highWaterMark,
// there shouldn't be a short read on initially setting up the batch.
// as such, any io.EOF is re-mapped to an io.ErrUnexpectedEOF so that we
// don't accidentally signal that we successfully reached the end of the
// batch.
err: dontExpectEOF(err),
}
}
// ReadOffset returns the offset of the first message with a timestamp equal or
// greater to t.
func (c *Conn) ReadOffset(t time.Time) (int64, error) {
return c.readOffset(timestamp(t))
}
// ReadFirstOffset returns the first offset available on the connection.
func (c *Conn) ReadFirstOffset() (int64, error) {
return c.readOffset(FirstOffset)
}
// ReadLastOffset returns the last offset available on the connection.
func (c *Conn) ReadLastOffset() (int64, error) {
return c.readOffset(LastOffset)
}
// ReadOffsets returns the absolute first and last offsets of the topic used by
// the connection.
func (c *Conn) ReadOffsets() (first, last int64, err error) {
// We have to submit two different requests to fetch the first and last
// offsets because kafka refuses requests that ask for multiple offsets
// on the same topic and partition.
if first, err = c.ReadFirstOffset(); err != nil {
return
}
if last, err = c.ReadLastOffset(); err != nil {
first = 0 // don't leak the value on error
return
}
return
}
func (c *Conn) readOffset(t int64) (offset int64, err error) {
err = c.readOperation(
func(deadline time.Time, id int32) error {
return c.wb.writeListOffsetRequestV1(id, c.clientID, c.topic, c.partition, t)
},
func(deadline time.Time, size int) error {
return expectZeroSize(readArrayWith(&c.rbuf, size, func(r *bufio.Reader, size int) (int, error) {
// We skip the topic name because we've made a request for
// a single topic.
size, err := discardString(r, size)
if err != nil {
return size, err
}
// Reading the array of partitions, there will be only one
// partition which gives the offset we're looking for.
return readArrayWith(r, size, func(r *bufio.Reader, size int) (int, error) {
var p partitionOffsetV1
size, err := p.readFrom(r, size)
if err != nil {
return size, err
}
if p.ErrorCode != 0 {
return size, Error(p.ErrorCode)
}
offset = p.Offset
return size, nil
})
}))
},
)
return
}
// ReadPartitions returns the list of available partitions for the given list of
// topics.
//
// If the method is called with no topic, it uses the topic configured on the
// connection. If there are none, the method fetches all partitions of the kafka
// cluster.
func (c *Conn) ReadPartitions(topics ...string) (partitions []Partition, err error) {
if len(topics) == 0 {
if len(c.topic) != 0 {
defaultTopics := [...]string{c.topic}
topics = defaultTopics[:]
} else {
// topics needs to be explicitly nil-ed out or the broker will
// interpret it as a request for 0 partitions instead of all.
topics = nil
}
}
metadataVersion, err := c.negotiateVersion(metadata, v1, v6)
if err != nil {
return nil, err
}
err = c.readOperation(
func(deadline time.Time, id int32) error {
switch metadataVersion {
case v6:
return c.writeRequest(metadata, v6, id, topicMetadataRequestV6{Topics: topics, AllowAutoTopicCreation: true})
default:
return c.writeRequest(metadata, v1, id, topicMetadataRequestV1(topics))
}
},
func(deadline time.Time, size int) error {
partitions, err = c.readPartitionsResponse(metadataVersion, size)
return err
},
)
return
}
func (c *Conn) readPartitionsResponse(metadataVersion apiVersion, size int) ([]Partition, error) {