File: transport.go

package info (click to toggle)
golang-github-segmentio-kafka-go 0.4.49%2Bds1-1
  • links: PTS, VCS
  • area: main
  • in suites: sid
  • size: 2,292 kB
  • sloc: sh: 17; makefile: 10
file content (1363 lines) | stat: -rw-r--r-- 35,211 bytes parent folder | download
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
1001
1002
1003
1004
1005
1006
1007
1008
1009
1010
1011
1012
1013
1014
1015
1016
1017
1018
1019
1020
1021
1022
1023
1024
1025
1026
1027
1028
1029
1030
1031
1032
1033
1034
1035
1036
1037
1038
1039
1040
1041
1042
1043
1044
1045
1046
1047
1048
1049
1050
1051
1052
1053
1054
1055
1056
1057
1058
1059
1060
1061
1062
1063
1064
1065
1066
1067
1068
1069
1070
1071
1072
1073
1074
1075
1076
1077
1078
1079
1080
1081
1082
1083
1084
1085
1086
1087
1088
1089
1090
1091
1092
1093
1094
1095
1096
1097
1098
1099
1100
1101
1102
1103
1104
1105
1106
1107
1108
1109
1110
1111
1112
1113
1114
1115
1116
1117
1118
1119
1120
1121
1122
1123
1124
1125
1126
1127
1128
1129
1130
1131
1132
1133
1134
1135
1136
1137
1138
1139
1140
1141
1142
1143
1144
1145
1146
1147
1148
1149
1150
1151
1152
1153
1154
1155
1156
1157
1158
1159
1160
1161
1162
1163
1164
1165
1166
1167
1168
1169
1170
1171
1172
1173
1174
1175
1176
1177
1178
1179
1180
1181
1182
1183
1184
1185
1186
1187
1188
1189
1190
1191
1192
1193
1194
1195
1196
1197
1198
1199
1200
1201
1202
1203
1204
1205
1206
1207
1208
1209
1210
1211
1212
1213
1214
1215
1216
1217
1218
1219
1220
1221
1222
1223
1224
1225
1226
1227
1228
1229
1230
1231
1232
1233
1234
1235
1236
1237
1238
1239
1240
1241
1242
1243
1244
1245
1246
1247
1248
1249
1250
1251
1252
1253
1254
1255
1256
1257
1258
1259
1260
1261
1262
1263
1264
1265
1266
1267
1268
1269
1270
1271
1272
1273
1274
1275
1276
1277
1278
1279
1280
1281
1282
1283
1284
1285
1286
1287
1288
1289
1290
1291
1292
1293
1294
1295
1296
1297
1298
1299
1300
1301
1302
1303
1304
1305
1306
1307
1308
1309
1310
1311
1312
1313
1314
1315
1316
1317
1318
1319
1320
1321
1322
1323
1324
1325
1326
1327
1328
1329
1330
1331
1332
1333
1334
1335
1336
1337
1338
1339
1340
1341
1342
1343
1344
1345
1346
1347
1348
1349
1350
1351
1352
1353
1354
1355
1356
1357
1358
1359
1360
1361
1362
1363
package kafka

import (
	"context"
	"crypto/tls"
	"errors"
	"fmt"
	"io"
	"math/rand"
	"net"
	"runtime/pprof"
	"sort"
	"strconv"
	"strings"
	"sync"
	"sync/atomic"
	"time"

	"github.com/segmentio/kafka-go/protocol"
	"github.com/segmentio/kafka-go/protocol/apiversions"
	"github.com/segmentio/kafka-go/protocol/createtopics"
	"github.com/segmentio/kafka-go/protocol/findcoordinator"
	meta "github.com/segmentio/kafka-go/protocol/metadata"
	"github.com/segmentio/kafka-go/protocol/saslauthenticate"
	"github.com/segmentio/kafka-go/protocol/saslhandshake"
	"github.com/segmentio/kafka-go/sasl"
)

// Request is an interface implemented by types that represent messages sent
// from kafka clients to brokers.
type Request = protocol.Message

// Response is an interface implemented by types that represent messages sent
// from kafka brokers in response to client requests.
type Response = protocol.Message

// RoundTripper is an interface implemented by types which support interacting
// with kafka brokers.
type RoundTripper interface {
	// RoundTrip sends a request to a kafka broker and returns the response that
	// was received, or a non-nil error.
	//
	// The context passed as first argument can be used to asynchronnously abort
	// the call if needed.
	RoundTrip(context.Context, net.Addr, Request) (Response, error)
}

// Transport is an implementation of the RoundTripper interface.
//
// Transport values manage a pool of connections and automatically discovers the
// clusters layout to route requests to the appropriate brokers.
//
// Transport values are safe to use concurrently from multiple goroutines.
//
// Note: The intent is for the Transport to become the underlying layer of the
// kafka.Reader and kafka.Writer types.
type Transport struct {
	// A function used to establish connections to the kafka cluster.
	Dial func(context.Context, string, string) (net.Conn, error)

	// Time limit set for establishing connections to the kafka cluster. This
	// limit includes all round trips done to establish the connections (TLS
	// handshake, SASL negotiation, etc...).
	//
	// Defaults to 5s.
	DialTimeout time.Duration

	// Maximum amount of time that connections will remain open and unused.
	// The transport will manage to automatically close connections that have
	// been idle for too long, and re-open them on demand when the transport is
	// used again.
	//
	// Defaults to 30s.
	IdleTimeout time.Duration

	// TTL for the metadata cached by this transport. Note that the value
	// configured here is an upper bound, the transport randomizes the TTLs to
	// avoid getting into states where multiple clients end up synchronized and
	// cause bursts of requests to the kafka broker.
	//
	// Default to 6s.
	MetadataTTL time.Duration

	// Topic names for the metadata cached by this transport. If this field is left blank,
	// metadata information of all topics in the cluster will be retrieved.
	MetadataTopics []string

	// Unique identifier that the transport communicates to the brokers when it
	// sends requests.
	ClientID string

	// An optional configuration for TLS connections established by this
	// transport.
	//
	// If the Server
	TLS *tls.Config

	// SASL configures the Transfer to use SASL authentication.
	SASL sasl.Mechanism

	// An optional resolver used to translate broker host names into network
	// addresses.
	//
	// The resolver will be called for every request (not every connection),
	// making it possible to implement ACL policies by validating that the
	// program is allowed to connect to the kafka broker. This also means that
	// the resolver should probably provide a caching layer to avoid storming
	// the service discovery backend with requests.
	//
	// When set, the Dial function is not responsible for performing name
	// resolution, and is always called with a pre-resolved address.
	Resolver BrokerResolver

	// The background context used to control goroutines started internally by
	// the transport.
	//
	// If nil, context.Background() is used instead.
	Context context.Context

	mutex sync.RWMutex
	pools map[networkAddress]*connPool
}

// DefaultTransport is the default transport used by kafka clients in this
// package.
var DefaultTransport RoundTripper = &Transport{
	Dial: (&net.Dialer{
		Timeout:   3 * time.Second,
		DualStack: true,
	}).DialContext,
}

// CloseIdleConnections closes all idle connections immediately, and marks all
// connections that are in use to be closed when they become idle again.
func (t *Transport) CloseIdleConnections() {
	t.mutex.Lock()
	defer t.mutex.Unlock()

	for _, pool := range t.pools {
		pool.unref()
	}

	for k := range t.pools {
		delete(t.pools, k)
	}
}

// RoundTrip sends a request to a kafka cluster and returns the response, or an
// error if no responses were received.
//
// Message types are available in sub-packages of the protocol package. Each
// kafka API is implemented in a different sub-package. For example, the request
// and response types for the Fetch API are available in the protocol/fetch
// package.
//
// The type of the response message will match the type of the request. For
// example, if RoundTrip was called with a *fetch.Request as argument, the value
// returned will be of type *fetch.Response. It is safe for the program to do a
// type assertion after checking that no error was returned.
//
// This example illustrates the way this method is expected to be used:
//
//	r, err := transport.RoundTrip(ctx, addr, &fetch.Request{ ... })
//	if err != nil {
//		...
//	} else {
//		res := r.(*fetch.Response)
//		...
//	}
//
// The transport automatically selects the highest version of the API that is
// supported by both the kafka-go package and the kafka broker. The negotiation
// happens transparently once when connections are established.
//
// This API was introduced in version 0.4 as a way to leverage the lower-level
// features of the kafka protocol, but also provide a more efficient way of
// managing connections to kafka brokers.
func (t *Transport) RoundTrip(ctx context.Context, addr net.Addr, req Request) (Response, error) {
	p := t.grabPool(addr)
	defer p.unref()
	return p.roundTrip(ctx, req)
}

func (t *Transport) dial() func(context.Context, string, string) (net.Conn, error) {
	if t.Dial != nil {
		return t.Dial
	}
	return defaultDialer.DialContext
}

func (t *Transport) dialTimeout() time.Duration {
	if t.DialTimeout > 0 {
		return t.DialTimeout
	}
	return 5 * time.Second
}

func (t *Transport) idleTimeout() time.Duration {
	if t.IdleTimeout > 0 {
		return t.IdleTimeout
	}
	return 30 * time.Second
}

func (t *Transport) metadataTTL() time.Duration {
	if t.MetadataTTL > 0 {
		return t.MetadataTTL
	}
	return 6 * time.Second
}

func (t *Transport) grabPool(addr net.Addr) *connPool {
	k := networkAddress{
		network: addr.Network(),
		address: addr.String(),
	}

	t.mutex.RLock()
	p := t.pools[k]
	if p != nil {
		p.ref()
	}
	t.mutex.RUnlock()

	if p != nil {
		return p
	}

	t.mutex.Lock()
	defer t.mutex.Unlock()

	if p := t.pools[k]; p != nil {
		p.ref()
		return p
	}

	ctx, cancel := context.WithCancel(t.context())

	p = &connPool{
		refc: 2,

		dial:           t.dial(),
		dialTimeout:    t.dialTimeout(),
		idleTimeout:    t.idleTimeout(),
		metadataTTL:    t.metadataTTL(),
		metadataTopics: t.MetadataTopics,
		clientID:       t.ClientID,
		tls:            t.TLS,
		sasl:           t.SASL,
		resolver:       t.Resolver,

		ready:  make(event),
		wake:   make(chan event),
		conns:  make(map[int32]*connGroup),
		cancel: cancel,
	}

	p.ctrl = p.newConnGroup(addr)
	go p.discover(ctx, p.wake)

	if t.pools == nil {
		t.pools = make(map[networkAddress]*connPool)
	}
	t.pools[k] = p
	return p
}

func (t *Transport) context() context.Context {
	if t.Context != nil {
		return t.Context
	}
	return context.Background()
}

type event chan struct{}

func (e event) trigger() { close(e) }

type connPool struct {
	refc uintptr
	// Immutable fields of the connection pool. Connections access these field
	// on their parent pool in a ready-only fashion, so no synchronization is
	// required.
	dial           func(context.Context, string, string) (net.Conn, error)
	dialTimeout    time.Duration
	idleTimeout    time.Duration
	metadataTTL    time.Duration
	metadataTopics []string
	clientID       string
	tls            *tls.Config
	sasl           sasl.Mechanism
	resolver       BrokerResolver
	// Signaling mechanisms to orchestrate communications between the pool and
	// the rest of the program.
	once   sync.Once  // ensure that `ready` is triggered only once
	ready  event      // triggered after the first metadata update
	wake   chan event // used to force metadata updates
	cancel context.CancelFunc
	// Mutable fields of the connection pool, access must be synchronized.
	mutex sync.RWMutex
	conns map[int32]*connGroup // data connections used for produce/fetch/etc...
	ctrl  *connGroup           // control connections used for metadata requests
	state atomic.Value         // cached cluster state
}

type connPoolState struct {
	metadata *meta.Response   // last metadata response seen by the pool
	err      error            // last error from metadata requests
	layout   protocol.Cluster // cluster layout built from metadata response
}

func (p *connPool) grabState() connPoolState {
	state, _ := p.state.Load().(connPoolState)
	return state
}

func (p *connPool) setState(state connPoolState) {
	p.state.Store(state)
}

func (p *connPool) ref() {
	atomic.AddUintptr(&p.refc, +1)
}

func (p *connPool) unref() {
	if atomic.AddUintptr(&p.refc, ^uintptr(0)) == 0 {
		p.mutex.Lock()
		defer p.mutex.Unlock()

		for _, conns := range p.conns {
			conns.closeIdleConns()
		}

		p.ctrl.closeIdleConns()
		p.cancel()
	}
}

func (p *connPool) roundTrip(ctx context.Context, req Request) (Response, error) {
	// This first select should never block after the first metadata response
	// that would mark the pool as `ready`.
	select {
	case <-p.ready:
	case <-ctx.Done():
		return nil, ctx.Err()
	}

	state := p.grabState()
	var response promise

	switch m := req.(type) {
	case *meta.Request:
		// We serve metadata requests directly from the transport cache unless
		// we would like to auto create a topic that isn't in our cache.
		//
		// This reduces the number of round trips to kafka brokers while keeping
		// the logic simple when applying partitioning strategies.
		if state.err != nil {
			return nil, state.err
		}

		cachedMeta := filterMetadataResponse(m, state.metadata)
		// requestNeeded indicates if we need to send this metadata request to the server.
		// It's true when we want to auto-create topics and we don't have the topic in our
		// cache.
		var requestNeeded bool
		if m.AllowAutoTopicCreation {
			for _, topic := range cachedMeta.Topics {
				if topic.ErrorCode == int16(UnknownTopicOrPartition) {
					requestNeeded = true
					break
				}
			}
		}

		if !requestNeeded {
			return cachedMeta, nil
		}

	case protocol.Splitter:
		// Messages that implement the Splitter interface trigger the creation of
		// multiple requests that are all merged back into a single results by
		// a merger.
		messages, merger, err := m.Split(state.layout)
		if err != nil {
			return nil, err
		}
		promises := make([]promise, len(messages))
		for i, m := range messages {
			promises[i] = p.sendRequest(ctx, m, state)
		}
		response = join(promises, messages, merger)
	}

	if response == nil {
		response = p.sendRequest(ctx, req, state)
	}

	r, err := response.await(ctx)
	if err != nil {
		return r, err
	}

	switch resp := r.(type) {
	case *createtopics.Response:
		// Force an update of the metadata when adding topics,
		// otherwise the cached state would get out of sync.
		topicsToRefresh := make([]string, 0, len(resp.Topics))
		for _, topic := range resp.Topics {
			// fixes issue 672: don't refresh topics that failed to create, it causes the library to hang indefinitely
			if topic.ErrorCode != 0 {
				continue
			}

			topicsToRefresh = append(topicsToRefresh, topic.Name)
		}

		p.refreshMetadata(ctx, topicsToRefresh)
	case *meta.Response:
		m := req.(*meta.Request)
		// If we get here with allow auto topic creation then
		// we didn't have that topic in our cache, so we should update
		// the cache.
		if m.AllowAutoTopicCreation {
			topicsToRefresh := make([]string, 0, len(resp.Topics))
			for _, topic := range resp.Topics {
				// Don't refresh topics that failed to create, since that may
				// mean that enable automatic topic creation is not enabled.
				// That causes the library to hang indefinitely, same as
				// don't refresh topics that failed to create,
				// createtopics process. Fixes issue 806.
				if topic.ErrorCode != 0 {
					continue
				}

				topicsToRefresh = append(topicsToRefresh, topic.Name)
			}
			p.refreshMetadata(ctx, topicsToRefresh)
		}
	}

	return r, nil
}

// refreshMetadata forces an update of the cached cluster metadata, and waits
// for the given list of topics to appear. This waiting mechanism is necessary
// to account for the fact that topic creation is asynchronous in kafka, and
// causes subsequent requests to fail while the cluster state is propagated to
// all the brokers.
func (p *connPool) refreshMetadata(ctx context.Context, expectTopics []string) {
	minBackoff := 100 * time.Millisecond
	maxBackoff := 2 * time.Second
	cancel := ctx.Done()

	for ctx.Err() == nil {
		notify := make(event)
		select {
		case <-cancel:
			return
		case p.wake <- notify:
			select {
			case <-notify:
			case <-cancel:
				return
			}
		}

		state := p.grabState()
		found := 0

		for _, topic := range expectTopics {
			if _, ok := state.layout.Topics[topic]; ok {
				found++
			}
		}

		if found == len(expectTopics) {
			return
		}

		if delay := time.Duration(rand.Int63n(int64(minBackoff))); delay > 0 {
			timer := time.NewTimer(minBackoff)
			select {
			case <-cancel:
			case <-timer.C:
			}
			timer.Stop()

			if minBackoff *= 2; minBackoff > maxBackoff {
				minBackoff = maxBackoff
			}
		}
	}
}

func (p *connPool) setReady() {
	p.once.Do(p.ready.trigger)
}

// update is called periodically by the goroutine running the discover method
// to refresh the cluster layout information used by the transport to route
// requests to brokers.
func (p *connPool) update(ctx context.Context, metadata *meta.Response, err error) {
	var layout protocol.Cluster

	if metadata != nil {
		metadata.ThrottleTimeMs = 0

		// Normalize the lists so we can apply binary search on them.
		sortMetadataBrokers(metadata.Brokers)
		sortMetadataTopics(metadata.Topics)

		for i := range metadata.Topics {
			t := &metadata.Topics[i]
			sortMetadataPartitions(t.Partitions)
		}

		layout = makeLayout(metadata)
	}

	state := p.grabState()
	addBrokers := make(map[int32]struct{})
	delBrokers := make(map[int32]struct{})

	if err != nil {
		// Only update the error on the transport if the cluster layout was
		// unknown. This ensures that we prioritize a previously known state
		// of the cluster to reduce the impact of transient failures.
		if state.metadata != nil {
			return
		}
		state.err = err
	} else {
		for id, b2 := range layout.Brokers {
			if b1, ok := state.layout.Brokers[id]; !ok {
				addBrokers[id] = struct{}{}
			} else if b1 != b2 {
				addBrokers[id] = struct{}{}
				delBrokers[id] = struct{}{}
			}
		}

		for id := range state.layout.Brokers {
			if _, ok := layout.Brokers[id]; !ok {
				delBrokers[id] = struct{}{}
			}
		}

		state.metadata, state.layout = metadata, layout
		state.err = nil
	}

	defer p.setReady()
	defer p.setState(state)

	if len(addBrokers) != 0 || len(delBrokers) != 0 {
		// Only acquire the lock when there is a change of layout. This is an
		// infrequent event so we don't risk introducing regular contention on
		// the mutex if we were to lock it on every update.
		p.mutex.Lock()
		defer p.mutex.Unlock()

		if ctx.Err() != nil {
			return // the pool has been closed, no need to update
		}

		for id := range delBrokers {
			if broker := p.conns[id]; broker != nil {
				broker.closeIdleConns()
				delete(p.conns, id)
			}
		}

		for id := range addBrokers {
			broker := layout.Brokers[id]
			p.conns[id] = p.newBrokerConnGroup(Broker{
				Rack: broker.Rack,
				Host: broker.Host,
				Port: int(broker.Port),
				ID:   int(broker.ID),
			})
		}
	}
}

// discover is the entry point of an internal goroutine for the transport which
// periodically requests updates of the cluster metadata and refreshes the
// transport cached cluster layout.
func (p *connPool) discover(ctx context.Context, wake <-chan event) {
	prng := rand.New(rand.NewSource(time.Now().UnixNano()))
	metadataTTL := func() time.Duration {
		return time.Duration(prng.Int63n(int64(p.metadataTTL)))
	}

	timer := time.NewTimer(metadataTTL())
	defer timer.Stop()

	var notify event
	done := ctx.Done()

	req := &meta.Request{
		TopicNames: p.metadataTopics,
	}

	for {
		c, err := p.grabClusterConn(ctx)
		if err != nil {
			p.update(ctx, nil, err)
		} else {
			res := make(async, 1)
			deadline, cancel := context.WithTimeout(ctx, p.metadataTTL)
			c.reqs <- connRequest{
				ctx: deadline,
				req: req,
				res: res,
			}
			r, err := res.await(deadline)
			cancel()
			if err != nil && errors.Is(err, ctx.Err()) {
				return
			}
			ret, _ := r.(*meta.Response)
			p.update(ctx, ret, err)
		}

		if notify != nil {
			notify.trigger()
			notify = nil
		}

		select {
		case <-timer.C:
			timer.Reset(metadataTTL())
		case <-done:
			return
		case notify = <-wake:
		}
	}
}

// grabBrokerConn returns a connection to a specific broker represented by the
// broker id passed as argument. If the broker id was not known, an error is
// returned.
func (p *connPool) grabBrokerConn(ctx context.Context, brokerID int32) (*conn, error) {
	p.mutex.RLock()
	g := p.conns[brokerID]
	p.mutex.RUnlock()
	if g == nil {
		return nil, BrokerNotAvailable
	}
	return g.grabConnOrConnect(ctx)
}

// grabClusterConn returns the connection to the kafka cluster that the pool is
// configured to connect to.
//
// The transport uses a shared `control` connection to the cluster for any
// requests that aren't supposed to be sent to specific brokers (e.g. Fetch or
// Produce requests). Requests intended to be routed to specific brokers are
// dispatched on a separate pool of connections that the transport maintains.
// This split help avoid head-of-line blocking situations where control requests
// like Metadata would be queued behind large responses from Fetch requests for
// example.
//
// In either cases, the requests are multiplexed so we can keep a minimal number
// of connections open (N+1, where N is the number of brokers in the cluster).
func (p *connPool) grabClusterConn(ctx context.Context) (*conn, error) {
	return p.ctrl.grabConnOrConnect(ctx)
}

func (p *connPool) sendRequest(ctx context.Context, req Request, state connPoolState) promise {
	brokerID := int32(-1)

	switch m := req.(type) {
	case protocol.BrokerMessage:
		// Some requests are supposed to be sent to specific brokers (e.g. the
		// partition leaders). They implement the BrokerMessage interface to
		// delegate the routing decision to each message type.
		broker, err := m.Broker(state.layout)
		if err != nil {
			return reject(err)
		}
		brokerID = broker.ID

	case protocol.GroupMessage:
		// Some requests are supposed to be sent to a group coordinator,
		// look up which broker is currently the coordinator for the group
		// so we can get a connection to that broker.
		//
		// TODO: should we cache the coordinator info?
		p := p.sendRequest(ctx, &findcoordinator.Request{Key: m.Group()}, state)
		r, err := p.await(ctx)
		if err != nil {
			return reject(err)
		}
		brokerID = r.(*findcoordinator.Response).NodeID
	case protocol.TransactionalMessage:
		p := p.sendRequest(ctx, &findcoordinator.Request{
			Key:     m.Transaction(),
			KeyType: int8(CoordinatorKeyTypeTransaction),
		}, state)
		r, err := p.await(ctx)
		if err != nil {
			return reject(err)
		}
		brokerID = r.(*findcoordinator.Response).NodeID
	}

	var c *conn
	var err error
	if brokerID >= 0 {
		c, err = p.grabBrokerConn(ctx, brokerID)
	} else {
		c, err = p.grabClusterConn(ctx)
	}
	if err != nil {
		return reject(err)
	}

	res := make(async, 1)

	c.reqs <- connRequest{
		ctx: ctx,
		req: req,
		res: res,
	}

	return res
}

func filterMetadataResponse(req *meta.Request, res *meta.Response) *meta.Response {
	ret := *res

	if req.TopicNames != nil {
		ret.Topics = make([]meta.ResponseTopic, len(req.TopicNames))

		for i, topicName := range req.TopicNames {
			j, ok := findMetadataTopic(res.Topics, topicName)
			if ok {
				ret.Topics[i] = res.Topics[j]
			} else {
				ret.Topics[i] = meta.ResponseTopic{
					ErrorCode: int16(UnknownTopicOrPartition),
					Name:      topicName,
				}
			}
		}
	}

	return &ret
}

func findMetadataTopic(topics []meta.ResponseTopic, topicName string) (int, bool) {
	i := sort.Search(len(topics), func(i int) bool {
		return topics[i].Name >= topicName
	})
	return i, i >= 0 && i < len(topics) && topics[i].Name == topicName
}

func sortMetadataBrokers(brokers []meta.ResponseBroker) {
	sort.Slice(brokers, func(i, j int) bool {
		return brokers[i].NodeID < brokers[j].NodeID
	})
}

func sortMetadataTopics(topics []meta.ResponseTopic) {
	sort.Slice(topics, func(i, j int) bool {
		return topics[i].Name < topics[j].Name
	})
}

func sortMetadataPartitions(partitions []meta.ResponsePartition) {
	sort.Slice(partitions, func(i, j int) bool {
		return partitions[i].PartitionIndex < partitions[j].PartitionIndex
	})
}

func makeLayout(metadataResponse *meta.Response) protocol.Cluster {
	layout := protocol.Cluster{
		Controller: metadataResponse.ControllerID,
		Brokers:    make(map[int32]protocol.Broker),
		Topics:     make(map[string]protocol.Topic),
	}

	for _, broker := range metadataResponse.Brokers {
		layout.Brokers[broker.NodeID] = protocol.Broker{
			Rack: broker.Rack,
			Host: broker.Host,
			Port: broker.Port,
			ID:   broker.NodeID,
		}
	}

	for _, topic := range metadataResponse.Topics {
		if topic.IsInternal {
			continue // TODO: do we need to expose those?
		}
		layout.Topics[topic.Name] = protocol.Topic{
			Name:       topic.Name,
			Error:      topic.ErrorCode,
			Partitions: makePartitions(topic.Partitions),
		}
	}

	return layout
}

func makePartitions(metadataPartitions []meta.ResponsePartition) map[int32]protocol.Partition {
	protocolPartitions := make(map[int32]protocol.Partition, len(metadataPartitions))
	numBrokerIDs := 0

	for _, p := range metadataPartitions {
		numBrokerIDs += len(p.ReplicaNodes) + len(p.IsrNodes) + len(p.OfflineReplicas)
	}

	// Reduce the memory footprint a bit by allocating a single buffer to write
	// all broker ids.
	brokerIDs := make([]int32, 0, numBrokerIDs)

	for _, p := range metadataPartitions {
		var rep, isr, off []int32
		brokerIDs, rep = appendBrokerIDs(brokerIDs, p.ReplicaNodes)
		brokerIDs, isr = appendBrokerIDs(brokerIDs, p.IsrNodes)
		brokerIDs, off = appendBrokerIDs(brokerIDs, p.OfflineReplicas)

		protocolPartitions[p.PartitionIndex] = protocol.Partition{
			ID:       p.PartitionIndex,
			Error:    p.ErrorCode,
			Leader:   p.LeaderID,
			Replicas: rep,
			ISR:      isr,
			Offline:  off,
		}
	}

	return protocolPartitions
}

func appendBrokerIDs(ids, brokers []int32) ([]int32, []int32) {
	i := len(ids)
	ids = append(ids, brokers...)
	return ids, ids[i:len(ids):len(ids)]
}

func (p *connPool) newConnGroup(a net.Addr) *connGroup {
	return &connGroup{
		addr: a,
		pool: p,
		broker: Broker{
			ID: -1,
		},
	}
}

func (p *connPool) newBrokerConnGroup(broker Broker) *connGroup {
	return &connGroup{
		addr: &networkAddress{
			network: "tcp",
			address: net.JoinHostPort(broker.Host, strconv.Itoa(broker.Port)),
		},
		pool:   p,
		broker: broker,
	}
}

type connRequest struct {
	ctx context.Context
	req Request
	res async
}

// The promise interface is used as a message passing abstraction to coordinate
// between goroutines that handle requests and responses.
type promise interface {
	// Waits until the promise is resolved, rejected, or the context canceled.
	await(context.Context) (Response, error)
}

// async is an implementation of the promise interface which supports resolving
// or rejecting the await call asynchronously.
type async chan interface{}

func (p async) await(ctx context.Context) (Response, error) {
	select {
	case x := <-p:
		switch v := x.(type) {
		case nil:
			return nil, nil // A nil response is ok (e.g. when RequiredAcks is None)
		case Response:
			return v, nil
		case error:
			return nil, v
		default:
			panic(fmt.Errorf("BUG: promise resolved with impossible value of type %T", v))
		}
	case <-ctx.Done():
		return nil, ctx.Err()
	}
}

func (p async) resolve(res Response) { p <- res }

func (p async) reject(err error) { p <- err }

// rejected is an implementation of the promise interface which is always
// returns an error. Values of this type are constructed using the reject
// function.
type rejected struct{ err error }

func reject(err error) promise { return &rejected{err: err} }

func (p *rejected) await(ctx context.Context) (Response, error) {
	return nil, p.err
}

// joined is an implementation of the promise interface which merges results
// from multiple promises into one await call using a merger.
type joined struct {
	promises []promise
	requests []Request
	merger   protocol.Merger
}

func join(promises []promise, requests []Request, merger protocol.Merger) promise {
	return &joined{
		promises: promises,
		requests: requests,
		merger:   merger,
	}
}

func (p *joined) await(ctx context.Context) (Response, error) {
	results := make([]interface{}, len(p.promises))

	for i, sub := range p.promises {
		m, err := sub.await(ctx)
		if err != nil {
			results[i] = err
		} else {
			results[i] = m
		}
	}

	return p.merger.Merge(p.requests, results)
}

// Default dialer used by the transport connections when no Dial function
// was configured by the program.
var defaultDialer = net.Dialer{
	Timeout:   3 * time.Second,
	DualStack: true,
}

// connGroup represents a logical connection group to a kafka broker. The
// actual network connections are lazily open before sending requests, and
// closed if they are unused for longer than the idle timeout.
type connGroup struct {
	addr   net.Addr
	broker Broker
	// Immutable state of the connection.
	pool *connPool
	// Shared state of the connection, this is synchronized on the mutex through
	// calls to the synchronized method. Both goroutines of the connection share
	// the state maintained in these fields.
	mutex     sync.Mutex
	closed    bool
	idleConns []*conn // stack of idle connections
}

func (g *connGroup) closeIdleConns() {
	g.mutex.Lock()
	conns := g.idleConns
	g.idleConns = nil
	g.closed = true
	g.mutex.Unlock()

	for _, c := range conns {
		c.close()
	}
}

func (g *connGroup) grabConnOrConnect(ctx context.Context) (*conn, error) {
	rslv := g.pool.resolver
	addr := g.addr
	var c *conn

	if rslv == nil {
		c = g.grabConn()
	} else {
		var err error
		broker := g.broker

		if broker.ID < 0 {
			host, port, err := splitHostPortNumber(addr.String())
			if err != nil {
				return nil, err
			}
			broker.Host = host
			broker.Port = port
		}

		ipAddrs, err := rslv.LookupBrokerIPAddr(ctx, broker)
		if err != nil {
			return nil, err
		}

		for _, ipAddr := range ipAddrs {
			network := addr.Network()
			address := net.JoinHostPort(ipAddr.String(), strconv.Itoa(broker.Port))

			if c = g.grabConnTo(network, address); c != nil {
				break
			}
		}
	}

	if c == nil {
		connChan := make(chan *conn)
		errChan := make(chan error)

		go func() {
			c, err := g.connect(ctx, addr)
			if err != nil {
				select {
				case errChan <- err:
				case <-ctx.Done():
				}
			} else {
				select {
				case connChan <- c:
				case <-ctx.Done():
					if !g.releaseConn(c) {
						c.close()
					}
				}
			}
		}()

		select {
		case c = <-connChan:
		case err := <-errChan:
			return nil, err
		case <-ctx.Done():
			return nil, ctx.Err()
		}
	}

	return c, nil
}

func (g *connGroup) grabConnTo(network, address string) *conn {
	g.mutex.Lock()
	defer g.mutex.Unlock()

	for i := len(g.idleConns) - 1; i >= 0; i-- {
		c := g.idleConns[i]

		if c.network == network && c.address == address {
			copy(g.idleConns[i:], g.idleConns[i+1:])
			n := len(g.idleConns) - 1
			g.idleConns[n] = nil
			g.idleConns = g.idleConns[:n]

			if c.timer != nil {
				c.timer.Stop()
			}

			return c
		}
	}

	return nil
}

func (g *connGroup) grabConn() *conn {
	g.mutex.Lock()
	defer g.mutex.Unlock()

	if len(g.idleConns) == 0 {
		return nil
	}

	n := len(g.idleConns) - 1
	c := g.idleConns[n]
	g.idleConns[n] = nil
	g.idleConns = g.idleConns[:n]

	if c.timer != nil {
		c.timer.Stop()
	}

	return c
}

func (g *connGroup) removeConn(c *conn) bool {
	g.mutex.Lock()
	defer g.mutex.Unlock()

	if c.timer != nil {
		c.timer.Stop()
	}

	for i, x := range g.idleConns {
		if x == c {
			copy(g.idleConns[i:], g.idleConns[i+1:])
			n := len(g.idleConns) - 1
			g.idleConns[n] = nil
			g.idleConns = g.idleConns[:n]
			return true
		}
	}

	return false
}

func (g *connGroup) releaseConn(c *conn) bool {
	idleTimeout := g.pool.idleTimeout

	g.mutex.Lock()
	defer g.mutex.Unlock()

	if g.closed {
		return false
	}

	if c.timer != nil {
		c.timer.Reset(idleTimeout)
	} else {
		c.timer = time.AfterFunc(idleTimeout, func() {
			if g.removeConn(c) {
				c.close()
			}
		})
	}

	g.idleConns = append(g.idleConns, c)
	return true
}

func (g *connGroup) connect(ctx context.Context, addr net.Addr) (*conn, error) {
	deadline := time.Now().Add(g.pool.dialTimeout)

	ctx, cancel := context.WithDeadline(ctx, deadline)
	defer cancel()

	network := strings.Split(addr.Network(), ",")
	address := strings.Split(addr.String(), ",")
	var netConn net.Conn
	var netAddr net.Addr
	var err error

	if len(address) > 1 {
		// Shuffle the list of addresses to randomize the order in which
		// connections are attempted. This prevents routing all connections
		// to the first broker (which will usually succeed).
		rand.Shuffle(len(address), func(i, j int) {
			network[i], network[j] = network[j], network[i]
			address[i], address[j] = address[j], address[i]
		})
	}

	for i := range address {
		netConn, err = g.pool.dial(ctx, network[i], address[i])
		if err == nil {
			netAddr = &networkAddress{
				network: network[i],
				address: address[i],
			}
			break
		}
	}

	if err != nil {
		return nil, err
	}

	defer func() {
		if netConn != nil {
			netConn.Close()
		}
	}()

	if tlsConfig := g.pool.tls; tlsConfig != nil {
		if tlsConfig.ServerName == "" {
			host, _ := splitHostPort(netAddr.String())
			tlsConfig = tlsConfig.Clone()
			tlsConfig.ServerName = host
		}
		netConn = tls.Client(netConn, tlsConfig)
	}

	pc := protocol.NewConn(netConn, g.pool.clientID)
	pc.SetDeadline(deadline)

	r, err := pc.RoundTrip(new(apiversions.Request))
	if err != nil {
		return nil, err
	}
	res := r.(*apiversions.Response)
	ver := make(map[protocol.ApiKey]int16, len(res.ApiKeys))

	if res.ErrorCode != 0 {
		return nil, fmt.Errorf("negotating API versions with kafka broker at %s: %w", g.addr, Error(res.ErrorCode))
	}

	for _, r := range res.ApiKeys {
		apiKey := protocol.ApiKey(r.ApiKey)
		ver[apiKey] = apiKey.SelectVersion(r.MinVersion, r.MaxVersion)
	}

	pc.SetVersions(ver)
	pc.SetDeadline(time.Time{})

	if g.pool.sasl != nil {
		host, port, err := splitHostPortNumber(netAddr.String())
		if err != nil {
			return nil, err
		}
		metadata := &sasl.Metadata{
			Host: host,
			Port: port,
		}
		if err := authenticateSASL(sasl.WithMetadata(ctx, metadata), pc, g.pool.sasl); err != nil {
			return nil, err
		}
	}

	reqs := make(chan connRequest)
	c := &conn{
		network: netAddr.Network(),
		address: netAddr.String(),
		reqs:    reqs,
		group:   g,
	}
	go c.run(pc, reqs)

	netConn = nil
	return c, nil
}

type conn struct {
	reqs    chan<- connRequest
	network string
	address string
	once    sync.Once
	group   *connGroup
	timer   *time.Timer
}

func (c *conn) close() {
	c.once.Do(func() { close(c.reqs) })
}

func (c *conn) run(pc *protocol.Conn, reqs <-chan connRequest) {
	defer pc.Close()

	for cr := range reqs {
		r, err := c.roundTrip(cr.ctx, pc, cr.req)
		if err != nil {
			cr.res.reject(err)
			if !errors.Is(err, protocol.ErrNoRecord) {
				break
			}
		} else {
			cr.res.resolve(r)
		}
		if !c.group.releaseConn(c) {
			break
		}
	}
}

func (c *conn) roundTrip(ctx context.Context, pc *protocol.Conn, req Request) (Response, error) {
	pprof.SetGoroutineLabels(ctx)
	defer pprof.SetGoroutineLabels(context.Background())

	if deadline, hasDeadline := ctx.Deadline(); hasDeadline {
		pc.SetDeadline(deadline)
		defer pc.SetDeadline(time.Time{})
	}

	return pc.RoundTrip(req)
}

// authenticateSASL performs all of the required requests to authenticate this
// connection.  If any step fails, this function returns with an error.  A nil
// error indicates successful authentication.
func authenticateSASL(ctx context.Context, pc *protocol.Conn, mechanism sasl.Mechanism) error {
	if err := saslHandshakeRoundTrip(pc, mechanism.Name()); err != nil {
		return err
	}

	sess, state, err := mechanism.Start(ctx)
	if err != nil {
		return err
	}

	for completed := false; !completed; {
		challenge, err := saslAuthenticateRoundTrip(pc, state)
		if err != nil {
			if errors.Is(err, io.EOF) {
				// the broker may communicate a failed exchange by closing the
				// connection (esp. in the case where we're passing opaque sasl
				// data over the wire since there's no protocol info).
				return SASLAuthenticationFailed
			}

			return err
		}

		completed, state, err = sess.Next(ctx, challenge)
		if err != nil {
			return err
		}
	}

	return nil
}

// saslHandshake sends the SASL handshake message.  This will determine whether
// the Mechanism is supported by the cluster.  If it's not, this function will
// error out with UnsupportedSASLMechanism.
//
// If the mechanism is unsupported, the handshake request will reply with the
// list of the cluster's configured mechanisms, which could potentially be used
// to facilitate negotiation.  At the moment, we are not negotiating the
// mechanism as we believe that brokers are usually known to the client, and
// therefore the client should already know which mechanisms are supported.
//
// See http://kafka.apache.org/protocol.html#The_Messages_SaslHandshake
func saslHandshakeRoundTrip(pc *protocol.Conn, mechanism string) error {
	msg, err := pc.RoundTrip(&saslhandshake.Request{
		Mechanism: mechanism,
	})
	if err != nil {
		return err
	}
	res := msg.(*saslhandshake.Response)
	if res.ErrorCode != 0 {
		err = Error(res.ErrorCode)
	}
	return err
}

// saslAuthenticate sends the SASL authenticate message.  This function must
// be immediately preceded by a successful saslHandshake.
//
// See http://kafka.apache.org/protocol.html#The_Messages_SaslAuthenticate
func saslAuthenticateRoundTrip(pc *protocol.Conn, data []byte) ([]byte, error) {
	msg, err := pc.RoundTrip(&saslauthenticate.Request{
		AuthBytes: data,
	})
	if err != nil {
		return nil, err
	}
	res := msg.(*saslauthenticate.Response)
	if res.ErrorCode != 0 {
		err = makeError(res.ErrorCode, res.ErrorMessage)
	}
	return res.AuthBytes, err
}

var _ RoundTripper = (*Transport)(nil)