blob: e5b3557b8c4496a5271221de518eeede99716f2c [file] [log] [blame]
Dinesh Belwalkare63f7f92019-11-22 23:11:16 +00001package sarama
2
3import (
4 "math/rand"
5 "sort"
6 "sync"
7 "time"
8)
9
10// Client is a generic Kafka client. It manages connections to one or more Kafka brokers.
11// You MUST call Close() on a client to avoid leaks, it will not be garbage-collected
12// automatically when it passes out of scope. It is safe to share a client amongst many
13// users, however Kafka will process requests from a single client strictly in serial,
14// so it is generally more efficient to use the default one client per producer/consumer.
15type Client interface {
16 // Config returns the Config struct of the client. This struct should not be
17 // altered after it has been created.
18 Config() *Config
19
20 // Controller returns the cluster controller broker. Requires Kafka 0.10 or higher.
21 Controller() (*Broker, error)
22
23 // Brokers returns the current set of active brokers as retrieved from cluster metadata.
24 Brokers() []*Broker
25
26 // Topics returns the set of available topics as retrieved from cluster metadata.
27 Topics() ([]string, error)
28
29 // Partitions returns the sorted list of all partition IDs for the given topic.
30 Partitions(topic string) ([]int32, error)
31
32 // WritablePartitions returns the sorted list of all writable partition IDs for
33 // the given topic, where "writable" means "having a valid leader accepting
34 // writes".
35 WritablePartitions(topic string) ([]int32, error)
36
37 // Leader returns the broker object that is the leader of the current
38 // topic/partition, as determined by querying the cluster metadata.
39 Leader(topic string, partitionID int32) (*Broker, error)
40
41 // Replicas returns the set of all replica IDs for the given partition.
42 Replicas(topic string, partitionID int32) ([]int32, error)
43
44 // InSyncReplicas returns the set of all in-sync replica IDs for the given
45 // partition. In-sync replicas are replicas which are fully caught up with
46 // the partition leader.
47 InSyncReplicas(topic string, partitionID int32) ([]int32, error)
48
49 // OfflineReplicas returns the set of all offline replica IDs for the given
50 // partition. Offline replicas are replicas which are offline
51 OfflineReplicas(topic string, partitionID int32) ([]int32, error)
52
53 // RefreshMetadata takes a list of topics and queries the cluster to refresh the
54 // available metadata for those topics. If no topics are provided, it will refresh
55 // metadata for all topics.
56 RefreshMetadata(topics ...string) error
57
58 // GetOffset queries the cluster to get the most recent available offset at the
59 // given time (in milliseconds) on the topic/partition combination.
60 // Time should be OffsetOldest for the earliest available offset,
61 // OffsetNewest for the offset of the message that will be produced next, or a time.
62 GetOffset(topic string, partitionID int32, time int64) (int64, error)
63
64 // Coordinator returns the coordinating broker for a consumer group. It will
65 // return a locally cached value if it's available. You can call
66 // RefreshCoordinator to update the cached value. This function only works on
67 // Kafka 0.8.2 and higher.
68 Coordinator(consumerGroup string) (*Broker, error)
69
70 // RefreshCoordinator retrieves the coordinator for a consumer group and stores it
71 // in local cache. This function only works on Kafka 0.8.2 and higher.
72 RefreshCoordinator(consumerGroup string) error
73
74 // InitProducerID retrieves information required for Idempotent Producer
75 InitProducerID() (*InitProducerIDResponse, error)
76
77 // Close shuts down all broker connections managed by this client. It is required
78 // to call this function before a client object passes out of scope, as it will
79 // otherwise leak memory. You must close any Producers or Consumers using a client
80 // before you close the client.
81 Close() error
82
83 // Closed returns true if the client has already had Close called on it
84 Closed() bool
85}
86
87const (
88 // OffsetNewest stands for the log head offset, i.e. the offset that will be
89 // assigned to the next message that will be produced to the partition. You
90 // can send this to a client's GetOffset method to get this offset, or when
91 // calling ConsumePartition to start consuming new messages.
92 OffsetNewest int64 = -1
93 // OffsetOldest stands for the oldest offset available on the broker for a
94 // partition. You can send this to a client's GetOffset method to get this
95 // offset, or when calling ConsumePartition to start consuming from the
96 // oldest offset that is still available on the broker.
97 OffsetOldest int64 = -2
98)
99
100type client struct {
101 conf *Config
102 closer, closed chan none // for shutting down background metadata updater
103
104 // the broker addresses given to us through the constructor are not guaranteed to be returned in
105 // the cluster metadata (I *think* it only returns brokers who are currently leading partitions?)
106 // so we store them separately
107 seedBrokers []*Broker
108 deadSeeds []*Broker
109
110 controllerID int32 // cluster controller broker id
111 brokers map[int32]*Broker // maps broker ids to brokers
112 metadata map[string]map[int32]*PartitionMetadata // maps topics to partition ids to metadata
113 metadataTopics map[string]none // topics that need to collect metadata
114 coordinators map[string]int32 // Maps consumer group names to coordinating broker IDs
115
116 // If the number of partitions is large, we can get some churn calling cachedPartitions,
117 // so the result is cached. It is important to update this value whenever metadata is changed
118 cachedPartitionsResults map[string][maxPartitionIndex][]int32
119
120 lock sync.RWMutex // protects access to the maps that hold cluster state.
121}
122
123// NewClient creates a new Client. It connects to one of the given broker addresses
124// and uses that broker to automatically fetch metadata on the rest of the kafka cluster. If metadata cannot
125// be retrieved from any of the given broker addresses, the client is not created.
126func NewClient(addrs []string, conf *Config) (Client, error) {
127 Logger.Println("Initializing new client")
128
129 if conf == nil {
130 conf = NewConfig()
131 }
132
133 if err := conf.Validate(); err != nil {
134 return nil, err
135 }
136
137 if len(addrs) < 1 {
138 return nil, ConfigurationError("You must provide at least one broker address")
139 }
140
141 client := &client{
142 conf: conf,
143 closer: make(chan none),
144 closed: make(chan none),
145 brokers: make(map[int32]*Broker),
146 metadata: make(map[string]map[int32]*PartitionMetadata),
147 metadataTopics: make(map[string]none),
148 cachedPartitionsResults: make(map[string][maxPartitionIndex][]int32),
149 coordinators: make(map[string]int32),
150 }
151
152 random := rand.New(rand.NewSource(time.Now().UnixNano()))
153 for _, index := range random.Perm(len(addrs)) {
154 client.seedBrokers = append(client.seedBrokers, NewBroker(addrs[index]))
155 }
156
157 if conf.Metadata.Full {
158 // do an initial fetch of all cluster metadata by specifying an empty list of topics
159 err := client.RefreshMetadata()
160 switch err {
161 case nil:
162 break
163 case ErrLeaderNotAvailable, ErrReplicaNotAvailable, ErrTopicAuthorizationFailed, ErrClusterAuthorizationFailed:
164 // indicates that maybe part of the cluster is down, but is not fatal to creating the client
165 Logger.Println(err)
166 default:
167 close(client.closed) // we haven't started the background updater yet, so we have to do this manually
168 _ = client.Close()
169 return nil, err
170 }
171 }
172 go withRecover(client.backgroundMetadataUpdater)
173
174 Logger.Println("Successfully initialized new client")
175
176 return client, nil
177}
178
179func (client *client) Config() *Config {
180 return client.conf
181}
182
183func (client *client) Brokers() []*Broker {
184 client.lock.RLock()
185 defer client.lock.RUnlock()
186 brokers := make([]*Broker, 0, len(client.brokers))
187 for _, broker := range client.brokers {
188 brokers = append(brokers, broker)
189 }
190 return brokers
191}
192
193func (client *client) InitProducerID() (*InitProducerIDResponse, error) {
194 var err error
195 for broker := client.any(); broker != nil; broker = client.any() {
196
197 req := &InitProducerIDRequest{}
198
199 response, err := broker.InitProducerID(req)
200 switch err.(type) {
201 case nil:
202 return response, nil
203 default:
204 // some error, remove that broker and try again
205 Logger.Printf("Client got error from broker %d when issuing InitProducerID : %v\n", broker.ID(), err)
206 _ = broker.Close()
207 client.deregisterBroker(broker)
208 }
209 }
210 return nil, err
211}
212
213func (client *client) Close() error {
214 if client.Closed() {
215 // Chances are this is being called from a defer() and the error will go unobserved
216 // so we go ahead and log the event in this case.
217 Logger.Printf("Close() called on already closed client")
218 return ErrClosedClient
219 }
220
221 // shutdown and wait for the background thread before we take the lock, to avoid races
222 close(client.closer)
223 <-client.closed
224
225 client.lock.Lock()
226 defer client.lock.Unlock()
227 Logger.Println("Closing Client")
228
229 for _, broker := range client.brokers {
230 safeAsyncClose(broker)
231 }
232
233 for _, broker := range client.seedBrokers {
234 safeAsyncClose(broker)
235 }
236
237 client.brokers = nil
238 client.metadata = nil
239 client.metadataTopics = nil
240
241 return nil
242}
243
244func (client *client) Closed() bool {
Dinesh Belwalkar396b6522020-02-06 22:11:53 +0000245 client.lock.RLock()
246 defer client.lock.RUnlock()
247
Dinesh Belwalkare63f7f92019-11-22 23:11:16 +0000248 return client.brokers == nil
249}
250
251func (client *client) Topics() ([]string, error) {
252 if client.Closed() {
253 return nil, ErrClosedClient
254 }
255
256 client.lock.RLock()
257 defer client.lock.RUnlock()
258
259 ret := make([]string, 0, len(client.metadata))
260 for topic := range client.metadata {
261 ret = append(ret, topic)
262 }
263
264 return ret, nil
265}
266
267func (client *client) MetadataTopics() ([]string, error) {
268 if client.Closed() {
269 return nil, ErrClosedClient
270 }
271
272 client.lock.RLock()
273 defer client.lock.RUnlock()
274
275 ret := make([]string, 0, len(client.metadataTopics))
276 for topic := range client.metadataTopics {
277 ret = append(ret, topic)
278 }
279
280 return ret, nil
281}
282
283func (client *client) Partitions(topic string) ([]int32, error) {
284 if client.Closed() {
285 return nil, ErrClosedClient
286 }
287
288 partitions := client.cachedPartitions(topic, allPartitions)
289
290 if len(partitions) == 0 {
291 err := client.RefreshMetadata(topic)
292 if err != nil {
293 return nil, err
294 }
295 partitions = client.cachedPartitions(topic, allPartitions)
296 }
297
298 // no partitions found after refresh metadata
299 if len(partitions) == 0 {
300 return nil, ErrUnknownTopicOrPartition
301 }
302
303 return partitions, nil
304}
305
306func (client *client) WritablePartitions(topic string) ([]int32, error) {
307 if client.Closed() {
308 return nil, ErrClosedClient
309 }
310
311 partitions := client.cachedPartitions(topic, writablePartitions)
312
313 // len==0 catches when it's nil (no such topic) and the odd case when every single
314 // partition is undergoing leader election simultaneously. Callers have to be able to handle
315 // this function returning an empty slice (which is a valid return value) but catching it
316 // here the first time (note we *don't* catch it below where we return ErrUnknownTopicOrPartition) triggers
317 // a metadata refresh as a nicety so callers can just try again and don't have to manually
318 // trigger a refresh (otherwise they'd just keep getting a stale cached copy).
319 if len(partitions) == 0 {
320 err := client.RefreshMetadata(topic)
321 if err != nil {
322 return nil, err
323 }
324 partitions = client.cachedPartitions(topic, writablePartitions)
325 }
326
327 if partitions == nil {
328 return nil, ErrUnknownTopicOrPartition
329 }
330
331 return partitions, nil
332}
333
334func (client *client) Replicas(topic string, partitionID int32) ([]int32, error) {
335 if client.Closed() {
336 return nil, ErrClosedClient
337 }
338
339 metadata := client.cachedMetadata(topic, partitionID)
340
341 if metadata == nil {
342 err := client.RefreshMetadata(topic)
343 if err != nil {
344 return nil, err
345 }
346 metadata = client.cachedMetadata(topic, partitionID)
347 }
348
349 if metadata == nil {
350 return nil, ErrUnknownTopicOrPartition
351 }
352
353 if metadata.Err == ErrReplicaNotAvailable {
354 return dupInt32Slice(metadata.Replicas), metadata.Err
355 }
356 return dupInt32Slice(metadata.Replicas), nil
357}
358
359func (client *client) InSyncReplicas(topic string, partitionID int32) ([]int32, error) {
360 if client.Closed() {
361 return nil, ErrClosedClient
362 }
363
364 metadata := client.cachedMetadata(topic, partitionID)
365
366 if metadata == nil {
367 err := client.RefreshMetadata(topic)
368 if err != nil {
369 return nil, err
370 }
371 metadata = client.cachedMetadata(topic, partitionID)
372 }
373
374 if metadata == nil {
375 return nil, ErrUnknownTopicOrPartition
376 }
377
378 if metadata.Err == ErrReplicaNotAvailable {
379 return dupInt32Slice(metadata.Isr), metadata.Err
380 }
381 return dupInt32Slice(metadata.Isr), nil
382}
383
384func (client *client) OfflineReplicas(topic string, partitionID int32) ([]int32, error) {
385 if client.Closed() {
386 return nil, ErrClosedClient
387 }
388
389 metadata := client.cachedMetadata(topic, partitionID)
390
391 if metadata == nil {
392 err := client.RefreshMetadata(topic)
393 if err != nil {
394 return nil, err
395 }
396 metadata = client.cachedMetadata(topic, partitionID)
397 }
398
399 if metadata == nil {
400 return nil, ErrUnknownTopicOrPartition
401 }
402
403 if metadata.Err == ErrReplicaNotAvailable {
404 return dupInt32Slice(metadata.OfflineReplicas), metadata.Err
405 }
406 return dupInt32Slice(metadata.OfflineReplicas), nil
407}
408
409func (client *client) Leader(topic string, partitionID int32) (*Broker, error) {
410 if client.Closed() {
411 return nil, ErrClosedClient
412 }
413
414 leader, err := client.cachedLeader(topic, partitionID)
415
416 if leader == nil {
417 err = client.RefreshMetadata(topic)
418 if err != nil {
419 return nil, err
420 }
421 leader, err = client.cachedLeader(topic, partitionID)
422 }
423
424 return leader, err
425}
426
427func (client *client) RefreshMetadata(topics ...string) error {
428 if client.Closed() {
429 return ErrClosedClient
430 }
431
432 // Prior to 0.8.2, Kafka will throw exceptions on an empty topic and not return a proper
433 // error. This handles the case by returning an error instead of sending it
434 // off to Kafka. See: https://github.com/Shopify/sarama/pull/38#issuecomment-26362310
435 for _, topic := range topics {
436 if len(topic) == 0 {
437 return ErrInvalidTopic // this is the error that 0.8.2 and later correctly return
438 }
439 }
440
441 deadline := time.Time{}
442 if client.conf.Metadata.Timeout > 0 {
443 deadline = time.Now().Add(client.conf.Metadata.Timeout)
444 }
445 return client.tryRefreshMetadata(topics, client.conf.Metadata.Retry.Max, deadline)
446}
447
448func (client *client) GetOffset(topic string, partitionID int32, time int64) (int64, error) {
449 if client.Closed() {
450 return -1, ErrClosedClient
451 }
452
453 offset, err := client.getOffset(topic, partitionID, time)
454
455 if err != nil {
456 if err := client.RefreshMetadata(topic); err != nil {
457 return -1, err
458 }
459 return client.getOffset(topic, partitionID, time)
460 }
461
462 return offset, err
463}
464
465func (client *client) Controller() (*Broker, error) {
466 if client.Closed() {
467 return nil, ErrClosedClient
468 }
469
470 if !client.conf.Version.IsAtLeast(V0_10_0_0) {
471 return nil, ErrUnsupportedVersion
472 }
473
474 controller := client.cachedController()
475 if controller == nil {
476 if err := client.refreshMetadata(); err != nil {
477 return nil, err
478 }
479 controller = client.cachedController()
480 }
481
482 if controller == nil {
483 return nil, ErrControllerNotAvailable
484 }
485
486 _ = controller.Open(client.conf)
487 return controller, nil
488}
489
490func (client *client) Coordinator(consumerGroup string) (*Broker, error) {
491 if client.Closed() {
492 return nil, ErrClosedClient
493 }
494
495 coordinator := client.cachedCoordinator(consumerGroup)
496
497 if coordinator == nil {
498 if err := client.RefreshCoordinator(consumerGroup); err != nil {
499 return nil, err
500 }
501 coordinator = client.cachedCoordinator(consumerGroup)
502 }
503
504 if coordinator == nil {
505 return nil, ErrConsumerCoordinatorNotAvailable
506 }
507
508 _ = coordinator.Open(client.conf)
509 return coordinator, nil
510}
511
512func (client *client) RefreshCoordinator(consumerGroup string) error {
513 if client.Closed() {
514 return ErrClosedClient
515 }
516
517 response, err := client.getConsumerMetadata(consumerGroup, client.conf.Metadata.Retry.Max)
518 if err != nil {
519 return err
520 }
521
522 client.lock.Lock()
523 defer client.lock.Unlock()
524 client.registerBroker(response.Coordinator)
525 client.coordinators[consumerGroup] = response.Coordinator.ID()
526 return nil
527}
528
529// private broker management helpers
530
531// registerBroker makes sure a broker received by a Metadata or Coordinator request is registered
532// in the brokers map. It returns the broker that is registered, which may be the provided broker,
533// or a previously registered Broker instance. You must hold the write lock before calling this function.
534func (client *client) registerBroker(broker *Broker) {
Dinesh Belwalkar396b6522020-02-06 22:11:53 +0000535 if client.brokers == nil {
536 Logger.Printf("cannot register broker #%d at %s, client already closed", broker.ID(), broker.Addr())
537 return
538 }
539
Dinesh Belwalkare63f7f92019-11-22 23:11:16 +0000540 if client.brokers[broker.ID()] == nil {
541 client.brokers[broker.ID()] = broker
542 Logger.Printf("client/brokers registered new broker #%d at %s", broker.ID(), broker.Addr())
543 } else if broker.Addr() != client.brokers[broker.ID()].Addr() {
544 safeAsyncClose(client.brokers[broker.ID()])
545 client.brokers[broker.ID()] = broker
546 Logger.Printf("client/brokers replaced registered broker #%d with %s", broker.ID(), broker.Addr())
547 }
548}
549
550// deregisterBroker removes a broker from the seedsBroker list, and if it's
551// not the seedbroker, removes it from brokers map completely.
552func (client *client) deregisterBroker(broker *Broker) {
553 client.lock.Lock()
554 defer client.lock.Unlock()
555
556 if len(client.seedBrokers) > 0 && broker == client.seedBrokers[0] {
557 client.deadSeeds = append(client.deadSeeds, broker)
558 client.seedBrokers = client.seedBrokers[1:]
559 } else {
560 // we do this so that our loop in `tryRefreshMetadata` doesn't go on forever,
561 // but we really shouldn't have to; once that loop is made better this case can be
562 // removed, and the function generally can be renamed from `deregisterBroker` to
563 // `nextSeedBroker` or something
564 Logger.Printf("client/brokers deregistered broker #%d at %s", broker.ID(), broker.Addr())
565 delete(client.brokers, broker.ID())
566 }
567}
568
569func (client *client) resurrectDeadBrokers() {
570 client.lock.Lock()
571 defer client.lock.Unlock()
572
573 Logger.Printf("client/brokers resurrecting %d dead seed brokers", len(client.deadSeeds))
574 client.seedBrokers = append(client.seedBrokers, client.deadSeeds...)
575 client.deadSeeds = nil
576}
577
578func (client *client) any() *Broker {
579 client.lock.RLock()
580 defer client.lock.RUnlock()
581
582 if len(client.seedBrokers) > 0 {
583 _ = client.seedBrokers[0].Open(client.conf)
584 return client.seedBrokers[0]
585 }
586
587 // not guaranteed to be random *or* deterministic
588 for _, broker := range client.brokers {
589 _ = broker.Open(client.conf)
590 return broker
591 }
592
593 return nil
594}
595
596// private caching/lazy metadata helpers
597
598type partitionType int
599
600const (
601 allPartitions partitionType = iota
602 writablePartitions
603 // If you add any more types, update the partition cache in update()
604
605 // Ensure this is the last partition type value
606 maxPartitionIndex
607)
608
609func (client *client) cachedMetadata(topic string, partitionID int32) *PartitionMetadata {
610 client.lock.RLock()
611 defer client.lock.RUnlock()
612
613 partitions := client.metadata[topic]
614 if partitions != nil {
615 return partitions[partitionID]
616 }
617
618 return nil
619}
620
621func (client *client) cachedPartitions(topic string, partitionSet partitionType) []int32 {
622 client.lock.RLock()
623 defer client.lock.RUnlock()
624
625 partitions, exists := client.cachedPartitionsResults[topic]
626
627 if !exists {
628 return nil
629 }
630 return partitions[partitionSet]
631}
632
633func (client *client) setPartitionCache(topic string, partitionSet partitionType) []int32 {
634 partitions := client.metadata[topic]
635
636 if partitions == nil {
637 return nil
638 }
639
640 ret := make([]int32, 0, len(partitions))
641 for _, partition := range partitions {
642 if partitionSet == writablePartitions && partition.Err == ErrLeaderNotAvailable {
643 continue
644 }
645 ret = append(ret, partition.ID)
646 }
647
648 sort.Sort(int32Slice(ret))
649 return ret
650}
651
652func (client *client) cachedLeader(topic string, partitionID int32) (*Broker, error) {
653 client.lock.RLock()
654 defer client.lock.RUnlock()
655
656 partitions := client.metadata[topic]
657 if partitions != nil {
658 metadata, ok := partitions[partitionID]
659 if ok {
660 if metadata.Err == ErrLeaderNotAvailable {
661 return nil, ErrLeaderNotAvailable
662 }
663 b := client.brokers[metadata.Leader]
664 if b == nil {
665 return nil, ErrLeaderNotAvailable
666 }
667 _ = b.Open(client.conf)
668 return b, nil
669 }
670 }
671
672 return nil, ErrUnknownTopicOrPartition
673}
674
675func (client *client) getOffset(topic string, partitionID int32, time int64) (int64, error) {
676 broker, err := client.Leader(topic, partitionID)
677 if err != nil {
678 return -1, err
679 }
680
681 request := &OffsetRequest{}
682 if client.conf.Version.IsAtLeast(V0_10_1_0) {
683 request.Version = 1
684 }
685 request.AddBlock(topic, partitionID, time, 1)
686
687 response, err := broker.GetAvailableOffsets(request)
688 if err != nil {
689 _ = broker.Close()
690 return -1, err
691 }
692
693 block := response.GetBlock(topic, partitionID)
694 if block == nil {
695 _ = broker.Close()
696 return -1, ErrIncompleteResponse
697 }
698 if block.Err != ErrNoError {
699 return -1, block.Err
700 }
701 if len(block.Offsets) != 1 {
702 return -1, ErrOffsetOutOfRange
703 }
704
705 return block.Offsets[0], nil
706}
707
708// core metadata update logic
709
710func (client *client) backgroundMetadataUpdater() {
711 defer close(client.closed)
712
713 if client.conf.Metadata.RefreshFrequency == time.Duration(0) {
714 return
715 }
716
717 ticker := time.NewTicker(client.conf.Metadata.RefreshFrequency)
718 defer ticker.Stop()
719
720 for {
721 select {
722 case <-ticker.C:
723 if err := client.refreshMetadata(); err != nil {
724 Logger.Println("Client background metadata update:", err)
725 }
726 case <-client.closer:
727 return
728 }
729 }
730}
731
732func (client *client) refreshMetadata() error {
733 topics := []string{}
734
735 if !client.conf.Metadata.Full {
736 if specificTopics, err := client.MetadataTopics(); err != nil {
737 return err
738 } else if len(specificTopics) == 0 {
739 return ErrNoTopicsToUpdateMetadata
740 } else {
741 topics = specificTopics
742 }
743 }
744
745 if err := client.RefreshMetadata(topics...); err != nil {
746 return err
747 }
748
749 return nil
750}
751
752func (client *client) tryRefreshMetadata(topics []string, attemptsRemaining int, deadline time.Time) error {
753 pastDeadline := func(backoff time.Duration) bool {
754 if !deadline.IsZero() && time.Now().Add(backoff).After(deadline) {
755 // we are past the deadline
756 return true
757 }
758 return false
759 }
760 retry := func(err error) error {
761 if attemptsRemaining > 0 {
762 backoff := client.computeBackoff(attemptsRemaining)
763 if pastDeadline(backoff) {
764 Logger.Println("client/metadata skipping last retries as we would go past the metadata timeout")
765 return err
766 }
767 Logger.Printf("client/metadata retrying after %dms... (%d attempts remaining)\n", client.conf.Metadata.Retry.Backoff/time.Millisecond, attemptsRemaining)
768 if backoff > 0 {
769 time.Sleep(backoff)
770 }
771 return client.tryRefreshMetadata(topics, attemptsRemaining-1, deadline)
772 }
773 return err
774 }
775
776 broker := client.any()
777 for ; broker != nil && !pastDeadline(0); broker = client.any() {
778 allowAutoTopicCreation := true
779 if len(topics) > 0 {
780 Logger.Printf("client/metadata fetching metadata for %v from broker %s\n", topics, broker.addr)
781 } else {
782 allowAutoTopicCreation = false
783 Logger.Printf("client/metadata fetching metadata for all topics from broker %s\n", broker.addr)
784 }
785
786 req := &MetadataRequest{Topics: topics, AllowAutoTopicCreation: allowAutoTopicCreation}
787 if client.conf.Version.IsAtLeast(V1_0_0_0) {
788 req.Version = 5
789 } else if client.conf.Version.IsAtLeast(V0_10_0_0) {
790 req.Version = 1
791 }
792 response, err := broker.GetMetadata(req)
793 switch err.(type) {
794 case nil:
795 allKnownMetaData := len(topics) == 0
796 // valid response, use it
797 shouldRetry, err := client.updateMetadata(response, allKnownMetaData)
798 if shouldRetry {
799 Logger.Println("client/metadata found some partitions to be leaderless")
800 return retry(err) // note: err can be nil
801 }
802 return err
803
804 case PacketEncodingError:
805 // didn't even send, return the error
806 return err
807
808 case KError:
809 // if SASL auth error return as this _should_ be a non retryable err for all brokers
810 if err.(KError) == ErrSASLAuthenticationFailed {
811 Logger.Println("client/metadata failed SASL authentication")
812 return err
813 }
814
815 if err.(KError) == ErrTopicAuthorizationFailed {
816 Logger.Println("client is not authorized to access this topic. The topics were: ", topics)
817 return err
818 }
819 // else remove that broker and try again
820 Logger.Printf("client/metadata got error from broker %d while fetching metadata: %v\n", broker.ID(), err)
821 _ = broker.Close()
822 client.deregisterBroker(broker)
823
824 default:
825 // some other error, remove that broker and try again
826 Logger.Printf("client/metadata got error from broker %d while fetching metadata: %v\n", broker.ID(), err)
827 _ = broker.Close()
828 client.deregisterBroker(broker)
829 }
830 }
831
832 if broker != nil {
Dinesh Belwalkar396b6522020-02-06 22:11:53 +0000833 Logger.Printf("client/metadata not fetching metadata from broker %s as we would go past the metadata timeout\n", broker.addr)
Dinesh Belwalkare63f7f92019-11-22 23:11:16 +0000834 return retry(ErrOutOfBrokers)
835 }
836
837 Logger.Println("client/metadata no available broker to send metadata request to")
838 client.resurrectDeadBrokers()
839 return retry(ErrOutOfBrokers)
840}
841
842// if no fatal error, returns a list of topics that need retrying due to ErrLeaderNotAvailable
843func (client *client) updateMetadata(data *MetadataResponse, allKnownMetaData bool) (retry bool, err error) {
Dinesh Belwalkar396b6522020-02-06 22:11:53 +0000844 if client.Closed() {
845 return
846 }
847
Dinesh Belwalkare63f7f92019-11-22 23:11:16 +0000848 client.lock.Lock()
849 defer client.lock.Unlock()
850
851 // For all the brokers we received:
852 // - if it is a new ID, save it
853 // - if it is an existing ID, but the address we have is stale, discard the old one and save it
854 // - otherwise ignore it, replacing our existing one would just bounce the connection
855 for _, broker := range data.Brokers {
856 client.registerBroker(broker)
857 }
858
859 client.controllerID = data.ControllerID
860
861 if allKnownMetaData {
862 client.metadata = make(map[string]map[int32]*PartitionMetadata)
863 client.metadataTopics = make(map[string]none)
864 client.cachedPartitionsResults = make(map[string][maxPartitionIndex][]int32)
865 }
866 for _, topic := range data.Topics {
867 // topics must be added firstly to `metadataTopics` to guarantee that all
868 // requested topics must be recorded to keep them trackable for periodically
869 // metadata refresh.
870 if _, exists := client.metadataTopics[topic.Name]; !exists {
871 client.metadataTopics[topic.Name] = none{}
872 }
873 delete(client.metadata, topic.Name)
874 delete(client.cachedPartitionsResults, topic.Name)
875
876 switch topic.Err {
877 case ErrNoError:
878 // no-op
879 case ErrInvalidTopic, ErrTopicAuthorizationFailed: // don't retry, don't store partial results
880 err = topic.Err
881 continue
882 case ErrUnknownTopicOrPartition: // retry, do not store partial partition results
883 err = topic.Err
884 retry = true
885 continue
886 case ErrLeaderNotAvailable: // retry, but store partial partition results
887 retry = true
888 default: // don't retry, don't store partial results
889 Logger.Printf("Unexpected topic-level metadata error: %s", topic.Err)
890 err = topic.Err
891 continue
892 }
893
894 client.metadata[topic.Name] = make(map[int32]*PartitionMetadata, len(topic.Partitions))
895 for _, partition := range topic.Partitions {
896 client.metadata[topic.Name][partition.ID] = partition
897 if partition.Err == ErrLeaderNotAvailable {
898 retry = true
899 }
900 }
901
902 var partitionCache [maxPartitionIndex][]int32
903 partitionCache[allPartitions] = client.setPartitionCache(topic.Name, allPartitions)
904 partitionCache[writablePartitions] = client.setPartitionCache(topic.Name, writablePartitions)
905 client.cachedPartitionsResults[topic.Name] = partitionCache
906 }
907
908 return
909}
910
911func (client *client) cachedCoordinator(consumerGroup string) *Broker {
912 client.lock.RLock()
913 defer client.lock.RUnlock()
914 if coordinatorID, ok := client.coordinators[consumerGroup]; ok {
915 return client.brokers[coordinatorID]
916 }
917 return nil
918}
919
920func (client *client) cachedController() *Broker {
921 client.lock.RLock()
922 defer client.lock.RUnlock()
923
924 return client.brokers[client.controllerID]
925}
926
927func (client *client) computeBackoff(attemptsRemaining int) time.Duration {
928 if client.conf.Metadata.Retry.BackoffFunc != nil {
929 maxRetries := client.conf.Metadata.Retry.Max
930 retries := maxRetries - attemptsRemaining
931 return client.conf.Metadata.Retry.BackoffFunc(retries, maxRetries)
932 }
933 return client.conf.Metadata.Retry.Backoff
934}
935
936func (client *client) getConsumerMetadata(consumerGroup string, attemptsRemaining int) (*FindCoordinatorResponse, error) {
937 retry := func(err error) (*FindCoordinatorResponse, error) {
938 if attemptsRemaining > 0 {
939 backoff := client.computeBackoff(attemptsRemaining)
940 Logger.Printf("client/coordinator retrying after %dms... (%d attempts remaining)\n", backoff/time.Millisecond, attemptsRemaining)
941 time.Sleep(backoff)
942 return client.getConsumerMetadata(consumerGroup, attemptsRemaining-1)
943 }
944 return nil, err
945 }
946
947 for broker := client.any(); broker != nil; broker = client.any() {
948 Logger.Printf("client/coordinator requesting coordinator for consumergroup %s from %s\n", consumerGroup, broker.Addr())
949
950 request := new(FindCoordinatorRequest)
951 request.CoordinatorKey = consumerGroup
952 request.CoordinatorType = CoordinatorGroup
953
954 response, err := broker.FindCoordinator(request)
955
956 if err != nil {
957 Logger.Printf("client/coordinator request to broker %s failed: %s\n", broker.Addr(), err)
958
959 switch err.(type) {
960 case PacketEncodingError:
961 return nil, err
962 default:
963 _ = broker.Close()
964 client.deregisterBroker(broker)
965 continue
966 }
967 }
968
969 switch response.Err {
970 case ErrNoError:
971 Logger.Printf("client/coordinator coordinator for consumergroup %s is #%d (%s)\n", consumerGroup, response.Coordinator.ID(), response.Coordinator.Addr())
972 return response, nil
973
974 case ErrConsumerCoordinatorNotAvailable:
975 Logger.Printf("client/coordinator coordinator for consumer group %s is not available\n", consumerGroup)
976
977 // This is very ugly, but this scenario will only happen once per cluster.
978 // The __consumer_offsets topic only has to be created one time.
979 // The number of partitions not configurable, but partition 0 should always exist.
980 if _, err := client.Leader("__consumer_offsets", 0); err != nil {
981 Logger.Printf("client/coordinator the __consumer_offsets topic is not initialized completely yet. Waiting 2 seconds...\n")
982 time.Sleep(2 * time.Second)
983 }
984
985 return retry(ErrConsumerCoordinatorNotAvailable)
986 case ErrGroupAuthorizationFailed:
987 Logger.Printf("client was not authorized to access group %s while attempting to find coordinator", consumerGroup)
988 return retry(ErrGroupAuthorizationFailed)
989
990 default:
991 return nil, response.Err
992 }
993 }
994
995 Logger.Println("client/coordinator no available broker to send consumer metadata request to")
996 client.resurrectDeadBrokers()
997 return retry(ErrOutOfBrokers)
998}
999
1000// nopCloserClient embeds an existing Client, but disables
1001// the Close method (yet all other methods pass
1002// through unchanged). This is for use in larger structs
1003// where it is undesirable to close the client that was
1004// passed in by the caller.
1005type nopCloserClient struct {
1006 Client
1007}
1008
1009// Close intercepts and purposely does not call the underlying
1010// client's Close() method.
1011func (ncc *nopCloserClient) Close() error {
1012 return nil
1013}