forked from IBM/sarama
-
Notifications
You must be signed in to change notification settings - Fork 0
/
broker.go
1441 lines (1167 loc) · 40.3 KB
/
broker.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 sarama
import (
"crypto/tls"
"encoding/binary"
"fmt"
"io"
"net"
"sort"
"strconv"
"strings"
"sync"
"sync/atomic"
"time"
"github.com/rcrowley/go-metrics"
)
// Broker represents a single Kafka broker connection. All operations on this object are entirely concurrency-safe.
type Broker struct {
conf *Config
rack *string
id int32
addr string
correlationID int32
conn net.Conn
connErr error
lock sync.Mutex
opened int32
responses chan responsePromise
done chan bool
registeredMetrics []string
incomingByteRate metrics.Meter
requestRate metrics.Meter
requestSize metrics.Histogram
requestLatency metrics.Histogram
outgoingByteRate metrics.Meter
responseRate metrics.Meter
responseSize metrics.Histogram
requestsInFlight metrics.Counter
brokerIncomingByteRate metrics.Meter
brokerRequestRate metrics.Meter
brokerRequestSize metrics.Histogram
brokerRequestLatency metrics.Histogram
brokerOutgoingByteRate metrics.Meter
brokerResponseRate metrics.Meter
brokerResponseSize metrics.Histogram
brokerRequestsInFlight metrics.Counter
kerberosAuthenticator GSSAPIKerberosAuth
}
// SASLMechanism specifies the SASL mechanism the client uses to authenticate with the broker
type SASLMechanism string
const (
// SASLTypeOAuth represents the SASL/OAUTHBEARER mechanism (Kafka 2.0.0+)
SASLTypeOAuth = "OAUTHBEARER"
// SASLTypePlaintext represents the SASL/PLAIN mechanism
SASLTypePlaintext = "PLAIN"
// SASLTypeSCRAMSHA256 represents the SCRAM-SHA-256 mechanism.
SASLTypeSCRAMSHA256 = "SCRAM-SHA-256"
// SASLTypeSCRAMSHA512 represents the SCRAM-SHA-512 mechanism.
SASLTypeSCRAMSHA512 = "SCRAM-SHA-512"
SASLTypeGSSAPI = "GSSAPI"
// SASLHandshakeV0 is v0 of the Kafka SASL handshake protocol. Client and
// server negotiate SASL auth using opaque packets.
SASLHandshakeV0 = int16(0)
// SASLHandshakeV1 is v1 of the Kafka SASL handshake protocol. Client and
// server negotiate SASL by wrapping tokens with Kafka protocol headers.
SASLHandshakeV1 = int16(1)
// SASLExtKeyAuth is the reserved extension key name sent as part of the
// SASL/OAUTHBEARER initial client response
SASLExtKeyAuth = "auth"
)
// AccessToken contains an access token used to authenticate a
// SASL/OAUTHBEARER client along with associated metadata.
type AccessToken struct {
// Token is the access token payload.
Token string
// Extensions is a optional map of arbitrary key-value pairs that can be
// sent with the SASL/OAUTHBEARER initial client response. These values are
// ignored by the SASL server if they are unexpected. This feature is only
// supported by Kafka >= 2.1.0.
Extensions map[string]string
}
// AccessTokenProvider is the interface that encapsulates how implementors
// can generate access tokens for Kafka broker authentication.
type AccessTokenProvider interface {
// Token returns an access token. The implementation should ensure token
// reuse so that multiple calls at connect time do not create multiple
// tokens. The implementation should also periodically refresh the token in
// order to guarantee that each call returns an unexpired token. This
// method should not block indefinitely--a timeout error should be returned
// after a short period of inactivity so that the broker connection logic
// can log debugging information and retry.
Token() (*AccessToken, error)
}
// SCRAMClient is a an interface to a SCRAM
// client implementation.
type SCRAMClient interface {
// Begin prepares the client for the SCRAM exchange
// with the server with a user name and a password
Begin(userName, password, authzID string) error
// Step steps client through the SCRAM exchange. It is
// called repeatedly until it errors or `Done` returns true.
Step(challenge string) (response string, err error)
// Done should return true when the SCRAM conversation
// is over.
Done() bool
}
type responsePromise struct {
requestTime time.Time
correlationID int32
headerVersion int16
packets chan []byte
errors chan error
}
// NewBroker creates and returns a Broker targeting the given host:port address.
// This does not attempt to actually connect, you have to call Open() for that.
func NewBroker(addr string) *Broker {
return &Broker{id: -1, addr: addr}
}
// Open tries to connect to the Broker if it is not already connected or connecting, but does not block
// waiting for the connection to complete. This means that any subsequent operations on the broker will
// block waiting for the connection to succeed or fail. To get the effect of a fully synchronous Open call,
// follow it by a call to Connected(). The only errors Open will return directly are ConfigurationError or
// AlreadyConnected. If conf is nil, the result of NewConfig() is used.
func (b *Broker) Open(conf *Config) error {
if !atomic.CompareAndSwapInt32(&b.opened, 0, 1) {
return ErrAlreadyConnected
}
if conf == nil {
conf = NewConfig()
}
err := conf.Validate()
if err != nil {
return err
}
b.lock.Lock()
go withRecover(func() {
defer b.lock.Unlock()
dialer := conf.getDialer()
b.conn, b.connErr = dialer.Dial("tcp", b.addr)
if b.connErr != nil {
Logger.Printf("Failed to connect to broker %s: %s\n", b.addr, b.connErr)
b.conn = nil
atomic.StoreInt32(&b.opened, 0)
return
}
if conf.Net.TLS.Enable {
b.conn = tls.Client(b.conn, validServerNameTLS(b.addr, conf.Net.TLS.Config))
}
b.conn = newBufConn(b.conn)
b.conf = conf
// Create or reuse the global metrics shared between brokers
b.incomingByteRate = metrics.GetOrRegisterMeter("incoming-byte-rate", conf.MetricRegistry)
b.requestRate = metrics.GetOrRegisterMeter("request-rate", conf.MetricRegistry)
b.requestSize = getOrRegisterHistogram("request-size", conf.MetricRegistry)
b.requestLatency = getOrRegisterHistogram("request-latency-in-ms", conf.MetricRegistry)
b.outgoingByteRate = metrics.GetOrRegisterMeter("outgoing-byte-rate", conf.MetricRegistry)
b.responseRate = metrics.GetOrRegisterMeter("response-rate", conf.MetricRegistry)
b.responseSize = getOrRegisterHistogram("response-size", conf.MetricRegistry)
b.requestsInFlight = metrics.GetOrRegisterCounter("requests-in-flight", conf.MetricRegistry)
// Do not gather metrics for seeded broker (only used during bootstrap) because they share
// the same id (-1) and are already exposed through the global metrics above
if b.id >= 0 {
b.registerMetrics()
}
if conf.Net.SASL.Enable {
b.connErr = b.authenticateViaSASL()
if b.connErr != nil {
err = b.conn.Close()
if err == nil {
Logger.Printf("Closed connection to broker %s\n", b.addr)
} else {
Logger.Printf("Error while closing connection to broker %s: %s\n", b.addr, err)
}
b.conn = nil
atomic.StoreInt32(&b.opened, 0)
return
}
}
b.done = make(chan bool)
b.responses = make(chan responsePromise, b.conf.Net.MaxOpenRequests-1)
if b.id >= 0 {
Logger.Printf("Connected to broker at %s (registered as #%d)\n", b.addr, b.id)
} else {
Logger.Printf("Connected to broker at %s (unregistered)\n", b.addr)
}
go withRecover(b.responseReceiver)
})
return nil
}
// Connected returns true if the broker is connected and false otherwise. If the broker is not
// connected but it had tried to connect, the error from that connection attempt is also returned.
func (b *Broker) Connected() (bool, error) {
b.lock.Lock()
defer b.lock.Unlock()
return b.conn != nil, b.connErr
}
//Close closes the broker resources
func (b *Broker) Close() error {
b.lock.Lock()
defer b.lock.Unlock()
if b.conn == nil {
return ErrNotConnected
}
close(b.responses)
<-b.done
err := b.conn.Close()
b.conn = nil
b.connErr = nil
b.done = nil
b.responses = nil
b.unregisterMetrics()
if err == nil {
Logger.Printf("Closed connection to broker %s\n", b.addr)
} else {
Logger.Printf("Error while closing connection to broker %s: %s\n", b.addr, err)
}
atomic.StoreInt32(&b.opened, 0)
return err
}
// ID returns the broker ID retrieved from Kafka's metadata, or -1 if that is not known.
func (b *Broker) ID() int32 {
return b.id
}
// Addr returns the broker address as either retrieved from Kafka's metadata or passed to NewBroker.
func (b *Broker) Addr() string {
return b.addr
}
// Rack returns the broker's rack as retrieved from Kafka's metadata or the
// empty string if it is not known. The returned value corresponds to the
// broker's broker.rack configuration setting. Requires protocol version to be
// at least v0.10.0.0.
func (b *Broker) Rack() string {
if b.rack == nil {
return ""
}
return *b.rack
}
//GetMetadata send a metadata request and returns a metadata response or error
func (b *Broker) GetMetadata(request *MetadataRequest) (*MetadataResponse, error) {
response := new(MetadataResponse)
err := b.sendAndReceive(request, response)
if err != nil {
return nil, err
}
return response, nil
}
//GetConsumerMetadata send a consumer metadata request and returns a consumer metadata response or error
func (b *Broker) GetConsumerMetadata(request *ConsumerMetadataRequest) (*ConsumerMetadataResponse, error) {
response := new(ConsumerMetadataResponse)
err := b.sendAndReceive(request, response)
if err != nil {
return nil, err
}
return response, nil
}
//FindCoordinator sends a find coordinate request and returns a response or error
func (b *Broker) FindCoordinator(request *FindCoordinatorRequest) (*FindCoordinatorResponse, error) {
response := new(FindCoordinatorResponse)
err := b.sendAndReceive(request, response)
if err != nil {
return nil, err
}
return response, nil
}
//GetAvailableOffsets return an offset response or error
func (b *Broker) GetAvailableOffsets(request *OffsetRequest) (*OffsetResponse, error) {
response := new(OffsetResponse)
err := b.sendAndReceive(request, response)
if err != nil {
return nil, err
}
return response, nil
}
//Produce returns a produce response or error
func (b *Broker) Produce(request *ProduceRequest) (*ProduceResponse, error) {
var (
response *ProduceResponse
err error
)
if request.RequiredAcks == NoResponse {
err = b.sendAndReceive(request, nil)
} else {
response = new(ProduceResponse)
err = b.sendAndReceive(request, response)
}
if err != nil {
return nil, err
}
return response, nil
}
//Fetch returns a FetchResponse or error
func (b *Broker) Fetch(request *FetchRequest) (*FetchResponse, error) {
response := new(FetchResponse)
err := b.sendAndReceive(request, response)
if err != nil {
return nil, err
}
return response, nil
}
//CommitOffset return an Offset commit response or error
func (b *Broker) CommitOffset(request *OffsetCommitRequest) (*OffsetCommitResponse, error) {
response := new(OffsetCommitResponse)
err := b.sendAndReceive(request, response)
if err != nil {
return nil, err
}
return response, nil
}
//FetchOffset returns an offset fetch response or error
func (b *Broker) FetchOffset(request *OffsetFetchRequest) (*OffsetFetchResponse, error) {
response := new(OffsetFetchResponse)
err := b.sendAndReceive(request, response)
if err != nil {
return nil, err
}
return response, nil
}
//JoinGroup returns a join group response or error
func (b *Broker) JoinGroup(request *JoinGroupRequest) (*JoinGroupResponse, error) {
response := new(JoinGroupResponse)
err := b.sendAndReceive(request, response)
if err != nil {
return nil, err
}
return response, nil
}
//SyncGroup returns a sync group response or error
func (b *Broker) SyncGroup(request *SyncGroupRequest) (*SyncGroupResponse, error) {
response := new(SyncGroupResponse)
err := b.sendAndReceive(request, response)
if err != nil {
return nil, err
}
return response, nil
}
//LeaveGroup return a leave group response or error
func (b *Broker) LeaveGroup(request *LeaveGroupRequest) (*LeaveGroupResponse, error) {
response := new(LeaveGroupResponse)
err := b.sendAndReceive(request, response)
if err != nil {
return nil, err
}
return response, nil
}
//Heartbeat returns a heartbeat response or error
func (b *Broker) Heartbeat(request *HeartbeatRequest) (*HeartbeatResponse, error) {
response := new(HeartbeatResponse)
err := b.sendAndReceive(request, response)
if err != nil {
return nil, err
}
return response, nil
}
//ListGroups return a list group response or error
func (b *Broker) ListGroups(request *ListGroupsRequest) (*ListGroupsResponse, error) {
response := new(ListGroupsResponse)
err := b.sendAndReceive(request, response)
if err != nil {
return nil, err
}
return response, nil
}
//DescribeGroups return describe group response or error
func (b *Broker) DescribeGroups(request *DescribeGroupsRequest) (*DescribeGroupsResponse, error) {
response := new(DescribeGroupsResponse)
err := b.sendAndReceive(request, response)
if err != nil {
return nil, err
}
return response, nil
}
//ApiVersions return api version response or error
func (b *Broker) ApiVersions(request *ApiVersionsRequest) (*ApiVersionsResponse, error) {
response := new(ApiVersionsResponse)
err := b.sendAndReceive(request, response)
if err != nil {
return nil, err
}
return response, nil
}
//CreateTopics send a create topic request and returns create topic response
func (b *Broker) CreateTopics(request *CreateTopicsRequest) (*CreateTopicsResponse, error) {
response := new(CreateTopicsResponse)
err := b.sendAndReceive(request, response)
if err != nil {
return nil, err
}
return response, nil
}
//DeleteTopics sends a delete topic request and returns delete topic response
func (b *Broker) DeleteTopics(request *DeleteTopicsRequest) (*DeleteTopicsResponse, error) {
response := new(DeleteTopicsResponse)
err := b.sendAndReceive(request, response)
if err != nil {
return nil, err
}
return response, nil
}
//CreatePartitions sends a create partition request and returns create
//partitions response or error
func (b *Broker) CreatePartitions(request *CreatePartitionsRequest) (*CreatePartitionsResponse, error) {
response := new(CreatePartitionsResponse)
err := b.sendAndReceive(request, response)
if err != nil {
return nil, err
}
return response, nil
}
//AlterPartitionReassignments sends a alter partition reassignments request and
//returns alter partition reassignments response
func (b *Broker) AlterPartitionReassignments(request *AlterPartitionReassignmentsRequest) (*AlterPartitionReassignmentsResponse, error) {
response := new(AlterPartitionReassignmentsResponse)
err := b.sendAndReceive(request, response)
if err != nil {
return nil, err
}
return response, nil
}
//ListPartitionReassignments sends a list partition reassignments request and
//returns list partition reassignments response
func (b *Broker) ListPartitionReassignments(request *ListPartitionReassignmentsRequest) (*ListPartitionReassignmentsResponse, error) {
response := new(ListPartitionReassignmentsResponse)
err := b.sendAndReceive(request, response)
if err != nil {
return nil, err
}
return response, nil
}
//DeleteRecords send a request to delete records and return delete record
//response or error
func (b *Broker) DeleteRecords(request *DeleteRecordsRequest) (*DeleteRecordsResponse, error) {
response := new(DeleteRecordsResponse)
err := b.sendAndReceive(request, response)
if err != nil {
return nil, err
}
return response, nil
}
//DescribeAcls sends a describe acl request and returns a response or error
func (b *Broker) DescribeAcls(request *DescribeAclsRequest) (*DescribeAclsResponse, error) {
response := new(DescribeAclsResponse)
err := b.sendAndReceive(request, response)
if err != nil {
return nil, err
}
return response, nil
}
//CreateAcls sends a create acl request and returns a response or error
func (b *Broker) CreateAcls(request *CreateAclsRequest) (*CreateAclsResponse, error) {
response := new(CreateAclsResponse)
err := b.sendAndReceive(request, response)
if err != nil {
return nil, err
}
return response, nil
}
//DeleteAcls sends a delete acl request and returns a response or error
func (b *Broker) DeleteAcls(request *DeleteAclsRequest) (*DeleteAclsResponse, error) {
response := new(DeleteAclsResponse)
err := b.sendAndReceive(request, response)
if err != nil {
return nil, err
}
return response, nil
}
//InitProducerID sends an init producer request and returns a response or error
func (b *Broker) InitProducerID(request *InitProducerIDRequest) (*InitProducerIDResponse, error) {
response := new(InitProducerIDResponse)
err := b.sendAndReceive(request, response)
if err != nil {
return nil, err
}
return response, nil
}
//AddPartitionsToTxn send a request to add partition to txn and returns
//a response or error
func (b *Broker) AddPartitionsToTxn(request *AddPartitionsToTxnRequest) (*AddPartitionsToTxnResponse, error) {
response := new(AddPartitionsToTxnResponse)
err := b.sendAndReceive(request, response)
if err != nil {
return nil, err
}
return response, nil
}
//AddOffsetsToTxn sends a request to add offsets to txn and returns a response
//or error
func (b *Broker) AddOffsetsToTxn(request *AddOffsetsToTxnRequest) (*AddOffsetsToTxnResponse, error) {
response := new(AddOffsetsToTxnResponse)
err := b.sendAndReceive(request, response)
if err != nil {
return nil, err
}
return response, nil
}
//EndTxn sends a request to end txn and returns a response or error
func (b *Broker) EndTxn(request *EndTxnRequest) (*EndTxnResponse, error) {
response := new(EndTxnResponse)
err := b.sendAndReceive(request, response)
if err != nil {
return nil, err
}
return response, nil
}
//TxnOffsetCommit sends a request to commit transaction offsets and returns
//a response or error
func (b *Broker) TxnOffsetCommit(request *TxnOffsetCommitRequest) (*TxnOffsetCommitResponse, error) {
response := new(TxnOffsetCommitResponse)
err := b.sendAndReceive(request, response)
if err != nil {
return nil, err
}
return response, nil
}
//DescribeConfigs sends a request to describe config and returns a response or
//error
func (b *Broker) DescribeConfigs(request *DescribeConfigsRequest) (*DescribeConfigsResponse, error) {
response := new(DescribeConfigsResponse)
err := b.sendAndReceive(request, response)
if err != nil {
return nil, err
}
return response, nil
}
//AlterConfigs sends a request to alter config and return a response or error
func (b *Broker) AlterConfigs(request *AlterConfigsRequest) (*AlterConfigsResponse, error) {
response := new(AlterConfigsResponse)
err := b.sendAndReceive(request, response)
if err != nil {
return nil, err
}
return response, nil
}
//DeleteGroups sends a request to delete groups and returns a response or error
func (b *Broker) DeleteGroups(request *DeleteGroupsRequest) (*DeleteGroupsResponse, error) {
response := new(DeleteGroupsResponse)
if err := b.sendAndReceive(request, response); err != nil {
return nil, err
}
return response, nil
}
//DescribeLogDirs sends a request to get the broker's log dir paths and sizes
func (b *Broker) DescribeLogDirs(request *DescribeLogDirsRequest) (*DescribeLogDirsResponse, error) {
response := new(DescribeLogDirsResponse)
err := b.sendAndReceive(request, response)
if err != nil {
return nil, err
}
return response, nil
}
// readFull ensures the conn ReadDeadline has been setup before making a
// call to io.ReadFull
func (b *Broker) readFull(buf []byte) (n int, err error) {
if err := b.conn.SetReadDeadline(time.Now().Add(b.conf.Net.ReadTimeout)); err != nil {
return 0, err
}
return io.ReadFull(b.conn, buf)
}
// write ensures the conn WriteDeadline has been setup before making a
// call to conn.Write
func (b *Broker) write(buf []byte) (n int, err error) {
if err := b.conn.SetWriteDeadline(time.Now().Add(b.conf.Net.WriteTimeout)); err != nil {
return 0, err
}
return b.conn.Write(buf)
}
func (b *Broker) send(rb protocolBody, promiseResponse bool, responseHeaderVersion int16) (*responsePromise, error) {
b.lock.Lock()
defer b.lock.Unlock()
if b.conn == nil {
if b.connErr != nil {
return nil, b.connErr
}
return nil, ErrNotConnected
}
if !b.conf.Version.IsAtLeast(rb.requiredVersion()) {
return nil, ErrUnsupportedVersion
}
req := &request{correlationID: b.correlationID, clientID: b.conf.ClientID, body: rb}
buf, err := encode(req, b.conf.MetricRegistry)
if err != nil {
return nil, err
}
requestTime := time.Now()
// Will be decremented in responseReceiver (except error or request with NoResponse)
b.addRequestInFlightMetrics(1)
bytes, err := b.write(buf)
b.updateOutgoingCommunicationMetrics(bytes)
if err != nil {
b.addRequestInFlightMetrics(-1)
return nil, err
}
b.correlationID++
if !promiseResponse {
// Record request latency without the response
b.updateRequestLatencyAndInFlightMetrics(time.Since(requestTime))
return nil, nil
}
promise := responsePromise{requestTime, req.correlationID, responseHeaderVersion, make(chan []byte), make(chan error)}
b.responses <- promise
return &promise, nil
}
func (b *Broker) sendAndReceive(req protocolBody, res protocolBody) error {
responseHeaderVersion := int16(-1)
if res != nil {
responseHeaderVersion = res.headerVersion()
}
promise, err := b.send(req, res != nil, responseHeaderVersion)
if err != nil {
return err
}
if promise == nil {
return nil
}
select {
case buf := <-promise.packets:
return versionedDecode(buf, res, req.version())
case err = <-promise.errors:
return err
}
}
func (b *Broker) decode(pd packetDecoder, version int16) (err error) {
b.id, err = pd.getInt32()
if err != nil {
return err
}
host, err := pd.getString()
if err != nil {
return err
}
port, err := pd.getInt32()
if err != nil {
return err
}
if version >= 1 {
b.rack, err = pd.getNullableString()
if err != nil {
return err
}
}
b.addr = net.JoinHostPort(host, fmt.Sprint(port))
if _, _, err := net.SplitHostPort(b.addr); err != nil {
return err
}
return nil
}
func (b *Broker) encode(pe packetEncoder, version int16) (err error) {
host, portstr, err := net.SplitHostPort(b.addr)
if err != nil {
return err
}
port, err := strconv.Atoi(portstr)
if err != nil {
return err
}
pe.putInt32(b.id)
err = pe.putString(host)
if err != nil {
return err
}
pe.putInt32(int32(port))
if version >= 1 {
err = pe.putNullableString(b.rack)
if err != nil {
return err
}
}
return nil
}
func (b *Broker) responseReceiver() {
var dead error
for response := range b.responses {
if dead != nil {
// This was previously incremented in send() and
// we are not calling updateIncomingCommunicationMetrics()
b.addRequestInFlightMetrics(-1)
response.errors <- dead
continue
}
var headerLength = getHeaderLength(response.headerVersion)
header := make([]byte, headerLength)
bytesReadHeader, err := b.readFull(header)
requestLatency := time.Since(response.requestTime)
if err != nil {
b.updateIncomingCommunicationMetrics(bytesReadHeader, requestLatency)
dead = err
response.errors <- err
continue
}
decodedHeader := responseHeader{}
err = versionedDecode(header, &decodedHeader, response.headerVersion)
if err != nil {
b.updateIncomingCommunicationMetrics(bytesReadHeader, requestLatency)
dead = err
response.errors <- err
continue
}
if decodedHeader.correlationID != response.correlationID {
b.updateIncomingCommunicationMetrics(bytesReadHeader, requestLatency)
// TODO if decoded ID < cur ID, discard until we catch up
// TODO if decoded ID > cur ID, save it so when cur ID catches up we have a response
dead = PacketDecodingError{fmt.Sprintf("correlation ID didn't match, wanted %d, got %d", response.correlationID, decodedHeader.correlationID)}
response.errors <- dead
continue
}
buf := make([]byte, decodedHeader.length-int32(headerLength)+4)
bytesReadBody, err := b.readFull(buf)
b.updateIncomingCommunicationMetrics(bytesReadHeader+bytesReadBody, requestLatency)
if err != nil {
dead = err
response.errors <- err
continue
}
response.packets <- buf
}
close(b.done)
}
func getHeaderLength(headerVersion int16) int8 {
if headerVersion < 1 {
return 8
} else {
// header contains additional tagged field length (0), we don't support actual tags yet.
return 9
}
}
func (b *Broker) authenticateViaSASL() error {
switch b.conf.Net.SASL.Mechanism {
case SASLTypeOAuth:
return b.sendAndReceiveSASLOAuth(b.conf.Net.SASL.TokenProvider)
case SASLTypeSCRAMSHA256, SASLTypeSCRAMSHA512:
return b.sendAndReceiveSASLSCRAMv1()
case SASLTypeGSSAPI:
return b.sendAndReceiveKerberos()
default:
return b.sendAndReceiveSASLPlainAuth()
}
}
func (b *Broker) sendAndReceiveKerberos() error {
b.kerberosAuthenticator.Config = &b.conf.Net.SASL.GSSAPI
if b.kerberosAuthenticator.NewKerberosClientFunc == nil {
b.kerberosAuthenticator.NewKerberosClientFunc = NewKerberosClient
}
return b.kerberosAuthenticator.Authorize(b)
}
func (b *Broker) sendAndReceiveSASLHandshake(saslType SASLMechanism, version int16) error {
rb := &SaslHandshakeRequest{Mechanism: string(saslType), Version: version}
req := &request{correlationID: b.correlationID, clientID: b.conf.ClientID, body: rb}
buf, err := encode(req, b.conf.MetricRegistry)
if err != nil {
return err
}
requestTime := time.Now()
// Will be decremented in updateIncomingCommunicationMetrics (except error)
b.addRequestInFlightMetrics(1)
bytes, err := b.write(buf)
b.updateOutgoingCommunicationMetrics(bytes)
if err != nil {
b.addRequestInFlightMetrics(-1)
Logger.Printf("Failed to send SASL handshake %s: %s\n", b.addr, err.Error())
return err
}
b.correlationID++
header := make([]byte, 8) // response header
_, err = b.readFull(header)
if err != nil {
b.addRequestInFlightMetrics(-1)
Logger.Printf("Failed to read SASL handshake header : %s\n", err.Error())
return err
}
length := binary.BigEndian.Uint32(header[:4])
payload := make([]byte, length-4)
n, err := b.readFull(payload)
if err != nil {
b.addRequestInFlightMetrics(-1)
Logger.Printf("Failed to read SASL handshake payload : %s\n", err.Error())
return err
}
b.updateIncomingCommunicationMetrics(n+8, time.Since(requestTime))
res := &SaslHandshakeResponse{}
err = versionedDecode(payload, res, 0)
if err != nil {
Logger.Printf("Failed to parse SASL handshake : %s\n", err.Error())
return err
}
if res.Err != ErrNoError {
Logger.Printf("Invalid SASL Mechanism : %s\n", res.Err.Error())
return res.Err
}
Logger.Print("Successful SASL handshake. Available mechanisms: ", res.EnabledMechanisms)
return nil
}
// Kafka 0.10.x supported SASL PLAIN/Kerberos via KAFKA-3149 (KIP-43).
// Kafka 1.x.x onward added a SaslAuthenticate request/response message which
// wraps the SASL flow in the Kafka protocol, which allows for returning
// meaningful errors on authentication failure.
//
// In SASL Plain, Kafka expects the auth header to be in the following format
// Message format (from https://tools.ietf.org/html/rfc4616):
//
// message = [authzid] UTF8NUL authcid UTF8NUL passwd
// authcid = 1*SAFE ; MUST accept up to 255 octets
// authzid = 1*SAFE ; MUST accept up to 255 octets
// passwd = 1*SAFE ; MUST accept up to 255 octets
// UTF8NUL = %x00 ; UTF-8 encoded NUL character
//
// SAFE = UTF1 / UTF2 / UTF3 / UTF4
// ;; any UTF-8 encoded Unicode character except NUL
//