blob: 040cfe9e395225255ff99834a02644e7cb18fb66 [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 {
245 return client.brokers == nil
246}
247
248func (client *client) Topics() ([]string, error) {
249 if client.Closed() {
250 return nil, ErrClosedClient
251 }
252
253 client.lock.RLock()
254 defer client.lock.RUnlock()
255
256 ret := make([]string, 0, len(client.metadata))
257 for topic := range client.metadata {
258 ret = append(ret, topic)
259 }
260
261 return ret, nil
262}
263
264func (client *client) MetadataTopics() ([]string, error) {
265 if client.Closed() {
266 return nil, ErrClosedClient
267 }
268
269 client.lock.RLock()
270 defer client.lock.RUnlock()
271
272 ret := make([]string, 0, len(client.metadataTopics))
273 for topic := range client.metadataTopics {
274 ret = append(ret, topic)
275 }
276
277 return ret, nil
278}
279
280func (client *client) Partitions(topic string) ([]int32, error) {
281 if client.Closed() {
282 return nil, ErrClosedClient
283 }
284
285 partitions := client.cachedPartitions(topic, allPartitions)
286
287 if len(partitions) == 0 {
288 err := client.RefreshMetadata(topic)
289 if err != nil {
290 return nil, err
291 }
292 partitions = client.cachedPartitions(topic, allPartitions)
293 }
294
295 // no partitions found after refresh metadata
296 if len(partitions) == 0 {
297 return nil, ErrUnknownTopicOrPartition
298 }
299
300 return partitions, nil
301}
302
303func (client *client) WritablePartitions(topic string) ([]int32, error) {
304 if client.Closed() {
305 return nil, ErrClosedClient
306 }
307
308 partitions := client.cachedPartitions(topic, writablePartitions)
309
310 // len==0 catches when it's nil (no such topic) and the odd case when every single
311 // partition is undergoing leader election simultaneously. Callers have to be able to handle
312 // this function returning an empty slice (which is a valid return value) but catching it
313 // here the first time (note we *don't* catch it below where we return ErrUnknownTopicOrPartition) triggers
314 // a metadata refresh as a nicety so callers can just try again and don't have to manually
315 // trigger a refresh (otherwise they'd just keep getting a stale cached copy).
316 if len(partitions) == 0 {
317 err := client.RefreshMetadata(topic)
318 if err != nil {
319 return nil, err
320 }
321 partitions = client.cachedPartitions(topic, writablePartitions)
322 }
323
324 if partitions == nil {
325 return nil, ErrUnknownTopicOrPartition
326 }
327
328 return partitions, nil
329}
330
331func (client *client) Replicas(topic string, partitionID int32) ([]int32, error) {
332 if client.Closed() {
333 return nil, ErrClosedClient
334 }
335
336 metadata := client.cachedMetadata(topic, partitionID)
337
338 if metadata == nil {
339 err := client.RefreshMetadata(topic)
340 if err != nil {
341 return nil, err
342 }
343 metadata = client.cachedMetadata(topic, partitionID)
344 }
345
346 if metadata == nil {
347 return nil, ErrUnknownTopicOrPartition
348 }
349
350 if metadata.Err == ErrReplicaNotAvailable {
351 return dupInt32Slice(metadata.Replicas), metadata.Err
352 }
353 return dupInt32Slice(metadata.Replicas), nil
354}
355
356func (client *client) InSyncReplicas(topic string, partitionID int32) ([]int32, error) {
357 if client.Closed() {
358 return nil, ErrClosedClient
359 }
360
361 metadata := client.cachedMetadata(topic, partitionID)
362
363 if metadata == nil {
364 err := client.RefreshMetadata(topic)
365 if err != nil {
366 return nil, err
367 }
368 metadata = client.cachedMetadata(topic, partitionID)
369 }
370
371 if metadata == nil {
372 return nil, ErrUnknownTopicOrPartition
373 }
374
375 if metadata.Err == ErrReplicaNotAvailable {
376 return dupInt32Slice(metadata.Isr), metadata.Err
377 }
378 return dupInt32Slice(metadata.Isr), nil
379}
380
381func (client *client) OfflineReplicas(topic string, partitionID int32) ([]int32, error) {
382 if client.Closed() {
383 return nil, ErrClosedClient
384 }
385
386 metadata := client.cachedMetadata(topic, partitionID)
387
388 if metadata == nil {
389 err := client.RefreshMetadata(topic)
390 if err != nil {
391 return nil, err
392 }
393 metadata = client.cachedMetadata(topic, partitionID)
394 }
395
396 if metadata == nil {
397 return nil, ErrUnknownTopicOrPartition
398 }
399
400 if metadata.Err == ErrReplicaNotAvailable {
401 return dupInt32Slice(metadata.OfflineReplicas), metadata.Err
402 }
403 return dupInt32Slice(metadata.OfflineReplicas), nil
404}
405
406func (client *client) Leader(topic string, partitionID int32) (*Broker, error) {
407 if client.Closed() {
408 return nil, ErrClosedClient
409 }
410
411 leader, err := client.cachedLeader(topic, partitionID)
412
413 if leader == nil {
414 err = client.RefreshMetadata(topic)
415 if err != nil {
416 return nil, err
417 }
418 leader, err = client.cachedLeader(topic, partitionID)
419 }
420
421 return leader, err
422}
423
424func (client *client) RefreshMetadata(topics ...string) error {
425 if client.Closed() {
426 return ErrClosedClient
427 }
428
429 // Prior to 0.8.2, Kafka will throw exceptions on an empty topic and not return a proper
430 // error. This handles the case by returning an error instead of sending it
431 // off to Kafka. See: https://github.com/Shopify/sarama/pull/38#issuecomment-26362310
432 for _, topic := range topics {
433 if len(topic) == 0 {
434 return ErrInvalidTopic // this is the error that 0.8.2 and later correctly return
435 }
436 }
437
438 deadline := time.Time{}
439 if client.conf.Metadata.Timeout > 0 {
440 deadline = time.Now().Add(client.conf.Metadata.Timeout)
441 }
442 return client.tryRefreshMetadata(topics, client.conf.Metadata.Retry.Max, deadline)
443}
444
445func (client *client) GetOffset(topic string, partitionID int32, time int64) (int64, error) {
446 if client.Closed() {
447 return -1, ErrClosedClient
448 }
449
450 offset, err := client.getOffset(topic, partitionID, time)
451
452 if err != nil {
453 if err := client.RefreshMetadata(topic); err != nil {
454 return -1, err
455 }
456 return client.getOffset(topic, partitionID, time)
457 }
458
459 return offset, err
460}
461
462func (client *client) Controller() (*Broker, error) {
463 if client.Closed() {
464 return nil, ErrClosedClient
465 }
466
467 if !client.conf.Version.IsAtLeast(V0_10_0_0) {
468 return nil, ErrUnsupportedVersion
469 }
470
471 controller := client.cachedController()
472 if controller == nil {
473 if err := client.refreshMetadata(); err != nil {
474 return nil, err
475 }
476 controller = client.cachedController()
477 }
478
479 if controller == nil {
480 return nil, ErrControllerNotAvailable
481 }
482
483 _ = controller.Open(client.conf)
484 return controller, nil
485}
486
487func (client *client) Coordinator(consumerGroup string) (*Broker, error) {
488 if client.Closed() {
489 return nil, ErrClosedClient
490 }
491
492 coordinator := client.cachedCoordinator(consumerGroup)
493
494 if coordinator == nil {
495 if err := client.RefreshCoordinator(consumerGroup); err != nil {
496 return nil, err
497 }
498 coordinator = client.cachedCoordinator(consumerGroup)
499 }
500
501 if coordinator == nil {
502 return nil, ErrConsumerCoordinatorNotAvailable
503 }
504
505 _ = coordinator.Open(client.conf)
506 return coordinator, nil
507}
508
509func (client *client) RefreshCoordinator(consumerGroup string) error {
510 if client.Closed() {
511 return ErrClosedClient
512 }
513
514 response, err := client.getConsumerMetadata(consumerGroup, client.conf.Metadata.Retry.Max)
515 if err != nil {
516 return err
517 }
518
519 client.lock.Lock()
520 defer client.lock.Unlock()
521 client.registerBroker(response.Coordinator)
522 client.coordinators[consumerGroup] = response.Coordinator.ID()
523 return nil
524}
525
526// private broker management helpers
527
528// registerBroker makes sure a broker received by a Metadata or Coordinator request is registered
529// in the brokers map. It returns the broker that is registered, which may be the provided broker,
530// or a previously registered Broker instance. You must hold the write lock before calling this function.
531func (client *client) registerBroker(broker *Broker) {
532 if client.brokers[broker.ID()] == nil {
533 client.brokers[broker.ID()] = broker
534 Logger.Printf("client/brokers registered new broker #%d at %s", broker.ID(), broker.Addr())
535 } else if broker.Addr() != client.brokers[broker.ID()].Addr() {
536 safeAsyncClose(client.brokers[broker.ID()])
537 client.brokers[broker.ID()] = broker
538 Logger.Printf("client/brokers replaced registered broker #%d with %s", broker.ID(), broker.Addr())
539 }
540}
541
542// deregisterBroker removes a broker from the seedsBroker list, and if it's
543// not the seedbroker, removes it from brokers map completely.
544func (client *client) deregisterBroker(broker *Broker) {
545 client.lock.Lock()
546 defer client.lock.Unlock()
547
548 if len(client.seedBrokers) > 0 && broker == client.seedBrokers[0] {
549 client.deadSeeds = append(client.deadSeeds, broker)
550 client.seedBrokers = client.seedBrokers[1:]
551 } else {
552 // we do this so that our loop in `tryRefreshMetadata` doesn't go on forever,
553 // but we really shouldn't have to; once that loop is made better this case can be
554 // removed, and the function generally can be renamed from `deregisterBroker` to
555 // `nextSeedBroker` or something
556 Logger.Printf("client/brokers deregistered broker #%d at %s", broker.ID(), broker.Addr())
557 delete(client.brokers, broker.ID())
558 }
559}
560
561func (client *client) resurrectDeadBrokers() {
562 client.lock.Lock()
563 defer client.lock.Unlock()
564
565 Logger.Printf("client/brokers resurrecting %d dead seed brokers", len(client.deadSeeds))
566 client.seedBrokers = append(client.seedBrokers, client.deadSeeds...)
567 client.deadSeeds = nil
568}
569
570func (client *client) any() *Broker {
571 client.lock.RLock()
572 defer client.lock.RUnlock()
573
574 if len(client.seedBrokers) > 0 {
575 _ = client.seedBrokers[0].Open(client.conf)
576 return client.seedBrokers[0]
577 }
578
579 // not guaranteed to be random *or* deterministic
580 for _, broker := range client.brokers {
581 _ = broker.Open(client.conf)
582 return broker
583 }
584
585 return nil
586}
587
588// private caching/lazy metadata helpers
589
590type partitionType int
591
592const (
593 allPartitions partitionType = iota
594 writablePartitions
595 // If you add any more types, update the partition cache in update()
596
597 // Ensure this is the last partition type value
598 maxPartitionIndex
599)
600
601func (client *client) cachedMetadata(topic string, partitionID int32) *PartitionMetadata {
602 client.lock.RLock()
603 defer client.lock.RUnlock()
604
605 partitions := client.metadata[topic]
606 if partitions != nil {
607 return partitions[partitionID]
608 }
609
610 return nil
611}
612
613func (client *client) cachedPartitions(topic string, partitionSet partitionType) []int32 {
614 client.lock.RLock()
615 defer client.lock.RUnlock()
616
617 partitions, exists := client.cachedPartitionsResults[topic]
618
619 if !exists {
620 return nil
621 }
622 return partitions[partitionSet]
623}
624
625func (client *client) setPartitionCache(topic string, partitionSet partitionType) []int32 {
626 partitions := client.metadata[topic]
627
628 if partitions == nil {
629 return nil
630 }
631
632 ret := make([]int32, 0, len(partitions))
633 for _, partition := range partitions {
634 if partitionSet == writablePartitions && partition.Err == ErrLeaderNotAvailable {
635 continue
636 }
637 ret = append(ret, partition.ID)
638 }
639
640 sort.Sort(int32Slice(ret))
641 return ret
642}
643
644func (client *client) cachedLeader(topic string, partitionID int32) (*Broker, error) {
645 client.lock.RLock()
646 defer client.lock.RUnlock()
647
648 partitions := client.metadata[topic]
649 if partitions != nil {
650 metadata, ok := partitions[partitionID]
651 if ok {
652 if metadata.Err == ErrLeaderNotAvailable {
653 return nil, ErrLeaderNotAvailable
654 }
655 b := client.brokers[metadata.Leader]
656 if b == nil {
657 return nil, ErrLeaderNotAvailable
658 }
659 _ = b.Open(client.conf)
660 return b, nil
661 }
662 }
663
664 return nil, ErrUnknownTopicOrPartition
665}
666
667func (client *client) getOffset(topic string, partitionID int32, time int64) (int64, error) {
668 broker, err := client.Leader(topic, partitionID)
669 if err != nil {
670 return -1, err
671 }
672
673 request := &OffsetRequest{}
674 if client.conf.Version.IsAtLeast(V0_10_1_0) {
675 request.Version = 1
676 }
677 request.AddBlock(topic, partitionID, time, 1)
678
679 response, err := broker.GetAvailableOffsets(request)
680 if err != nil {
681 _ = broker.Close()
682 return -1, err
683 }
684
685 block := response.GetBlock(topic, partitionID)
686 if block == nil {
687 _ = broker.Close()
688 return -1, ErrIncompleteResponse
689 }
690 if block.Err != ErrNoError {
691 return -1, block.Err
692 }
693 if len(block.Offsets) != 1 {
694 return -1, ErrOffsetOutOfRange
695 }
696
697 return block.Offsets[0], nil
698}
699
700// core metadata update logic
701
702func (client *client) backgroundMetadataUpdater() {
703 defer close(client.closed)
704
705 if client.conf.Metadata.RefreshFrequency == time.Duration(0) {
706 return
707 }
708
709 ticker := time.NewTicker(client.conf.Metadata.RefreshFrequency)
710 defer ticker.Stop()
711
712 for {
713 select {
714 case <-ticker.C:
715 if err := client.refreshMetadata(); err != nil {
716 Logger.Println("Client background metadata update:", err)
717 }
718 case <-client.closer:
719 return
720 }
721 }
722}
723
724func (client *client) refreshMetadata() error {
725 topics := []string{}
726
727 if !client.conf.Metadata.Full {
728 if specificTopics, err := client.MetadataTopics(); err != nil {
729 return err
730 } else if len(specificTopics) == 0 {
731 return ErrNoTopicsToUpdateMetadata
732 } else {
733 topics = specificTopics
734 }
735 }
736
737 if err := client.RefreshMetadata(topics...); err != nil {
738 return err
739 }
740
741 return nil
742}
743
744func (client *client) tryRefreshMetadata(topics []string, attemptsRemaining int, deadline time.Time) error {
745 pastDeadline := func(backoff time.Duration) bool {
746 if !deadline.IsZero() && time.Now().Add(backoff).After(deadline) {
747 // we are past the deadline
748 return true
749 }
750 return false
751 }
752 retry := func(err error) error {
753 if attemptsRemaining > 0 {
754 backoff := client.computeBackoff(attemptsRemaining)
755 if pastDeadline(backoff) {
756 Logger.Println("client/metadata skipping last retries as we would go past the metadata timeout")
757 return err
758 }
759 Logger.Printf("client/metadata retrying after %dms... (%d attempts remaining)\n", client.conf.Metadata.Retry.Backoff/time.Millisecond, attemptsRemaining)
760 if backoff > 0 {
761 time.Sleep(backoff)
762 }
763 return client.tryRefreshMetadata(topics, attemptsRemaining-1, deadline)
764 }
765 return err
766 }
767
768 broker := client.any()
769 for ; broker != nil && !pastDeadline(0); broker = client.any() {
770 allowAutoTopicCreation := true
771 if len(topics) > 0 {
772 Logger.Printf("client/metadata fetching metadata for %v from broker %s\n", topics, broker.addr)
773 } else {
774 allowAutoTopicCreation = false
775 Logger.Printf("client/metadata fetching metadata for all topics from broker %s\n", broker.addr)
776 }
777
778 req := &MetadataRequest{Topics: topics, AllowAutoTopicCreation: allowAutoTopicCreation}
779 if client.conf.Version.IsAtLeast(V1_0_0_0) {
780 req.Version = 5
781 } else if client.conf.Version.IsAtLeast(V0_10_0_0) {
782 req.Version = 1
783 }
784 response, err := broker.GetMetadata(req)
785 switch err.(type) {
786 case nil:
787 allKnownMetaData := len(topics) == 0
788 // valid response, use it
789 shouldRetry, err := client.updateMetadata(response, allKnownMetaData)
790 if shouldRetry {
791 Logger.Println("client/metadata found some partitions to be leaderless")
792 return retry(err) // note: err can be nil
793 }
794 return err
795
796 case PacketEncodingError:
797 // didn't even send, return the error
798 return err
799
800 case KError:
801 // if SASL auth error return as this _should_ be a non retryable err for all brokers
802 if err.(KError) == ErrSASLAuthenticationFailed {
803 Logger.Println("client/metadata failed SASL authentication")
804 return err
805 }
806
807 if err.(KError) == ErrTopicAuthorizationFailed {
808 Logger.Println("client is not authorized to access this topic. The topics were: ", topics)
809 return err
810 }
811 // else remove that broker and try again
812 Logger.Printf("client/metadata got error from broker %d while fetching metadata: %v\n", broker.ID(), err)
813 _ = broker.Close()
814 client.deregisterBroker(broker)
815
816 default:
817 // some other error, remove that broker and try again
818 Logger.Printf("client/metadata got error from broker %d while fetching metadata: %v\n", broker.ID(), err)
819 _ = broker.Close()
820 client.deregisterBroker(broker)
821 }
822 }
823
824 if broker != nil {
825 Logger.Println("client/metadata not fetching metadata from broker %s as we would go past the metadata timeout\n", broker.addr)
826 return retry(ErrOutOfBrokers)
827 }
828
829 Logger.Println("client/metadata no available broker to send metadata request to")
830 client.resurrectDeadBrokers()
831 return retry(ErrOutOfBrokers)
832}
833
834// if no fatal error, returns a list of topics that need retrying due to ErrLeaderNotAvailable
835func (client *client) updateMetadata(data *MetadataResponse, allKnownMetaData bool) (retry bool, err error) {
836 client.lock.Lock()
837 defer client.lock.Unlock()
838
839 // For all the brokers we received:
840 // - if it is a new ID, save it
841 // - if it is an existing ID, but the address we have is stale, discard the old one and save it
842 // - otherwise ignore it, replacing our existing one would just bounce the connection
843 for _, broker := range data.Brokers {
844 client.registerBroker(broker)
845 }
846
847 client.controllerID = data.ControllerID
848
849 if allKnownMetaData {
850 client.metadata = make(map[string]map[int32]*PartitionMetadata)
851 client.metadataTopics = make(map[string]none)
852 client.cachedPartitionsResults = make(map[string][maxPartitionIndex][]int32)
853 }
854 for _, topic := range data.Topics {
855 // topics must be added firstly to `metadataTopics` to guarantee that all
856 // requested topics must be recorded to keep them trackable for periodically
857 // metadata refresh.
858 if _, exists := client.metadataTopics[topic.Name]; !exists {
859 client.metadataTopics[topic.Name] = none{}
860 }
861 delete(client.metadata, topic.Name)
862 delete(client.cachedPartitionsResults, topic.Name)
863
864 switch topic.Err {
865 case ErrNoError:
866 // no-op
867 case ErrInvalidTopic, ErrTopicAuthorizationFailed: // don't retry, don't store partial results
868 err = topic.Err
869 continue
870 case ErrUnknownTopicOrPartition: // retry, do not store partial partition results
871 err = topic.Err
872 retry = true
873 continue
874 case ErrLeaderNotAvailable: // retry, but store partial partition results
875 retry = true
876 default: // don't retry, don't store partial results
877 Logger.Printf("Unexpected topic-level metadata error: %s", topic.Err)
878 err = topic.Err
879 continue
880 }
881
882 client.metadata[topic.Name] = make(map[int32]*PartitionMetadata, len(topic.Partitions))
883 for _, partition := range topic.Partitions {
884 client.metadata[topic.Name][partition.ID] = partition
885 if partition.Err == ErrLeaderNotAvailable {
886 retry = true
887 }
888 }
889
890 var partitionCache [maxPartitionIndex][]int32
891 partitionCache[allPartitions] = client.setPartitionCache(topic.Name, allPartitions)
892 partitionCache[writablePartitions] = client.setPartitionCache(topic.Name, writablePartitions)
893 client.cachedPartitionsResults[topic.Name] = partitionCache
894 }
895
896 return
897}
898
899func (client *client) cachedCoordinator(consumerGroup string) *Broker {
900 client.lock.RLock()
901 defer client.lock.RUnlock()
902 if coordinatorID, ok := client.coordinators[consumerGroup]; ok {
903 return client.brokers[coordinatorID]
904 }
905 return nil
906}
907
908func (client *client) cachedController() *Broker {
909 client.lock.RLock()
910 defer client.lock.RUnlock()
911
912 return client.brokers[client.controllerID]
913}
914
915func (client *client) computeBackoff(attemptsRemaining int) time.Duration {
916 if client.conf.Metadata.Retry.BackoffFunc != nil {
917 maxRetries := client.conf.Metadata.Retry.Max
918 retries := maxRetries - attemptsRemaining
919 return client.conf.Metadata.Retry.BackoffFunc(retries, maxRetries)
920 }
921 return client.conf.Metadata.Retry.Backoff
922}
923
924func (client *client) getConsumerMetadata(consumerGroup string, attemptsRemaining int) (*FindCoordinatorResponse, error) {
925 retry := func(err error) (*FindCoordinatorResponse, error) {
926 if attemptsRemaining > 0 {
927 backoff := client.computeBackoff(attemptsRemaining)
928 Logger.Printf("client/coordinator retrying after %dms... (%d attempts remaining)\n", backoff/time.Millisecond, attemptsRemaining)
929 time.Sleep(backoff)
930 return client.getConsumerMetadata(consumerGroup, attemptsRemaining-1)
931 }
932 return nil, err
933 }
934
935 for broker := client.any(); broker != nil; broker = client.any() {
936 Logger.Printf("client/coordinator requesting coordinator for consumergroup %s from %s\n", consumerGroup, broker.Addr())
937
938 request := new(FindCoordinatorRequest)
939 request.CoordinatorKey = consumerGroup
940 request.CoordinatorType = CoordinatorGroup
941
942 response, err := broker.FindCoordinator(request)
943
944 if err != nil {
945 Logger.Printf("client/coordinator request to broker %s failed: %s\n", broker.Addr(), err)
946
947 switch err.(type) {
948 case PacketEncodingError:
949 return nil, err
950 default:
951 _ = broker.Close()
952 client.deregisterBroker(broker)
953 continue
954 }
955 }
956
957 switch response.Err {
958 case ErrNoError:
959 Logger.Printf("client/coordinator coordinator for consumergroup %s is #%d (%s)\n", consumerGroup, response.Coordinator.ID(), response.Coordinator.Addr())
960 return response, nil
961
962 case ErrConsumerCoordinatorNotAvailable:
963 Logger.Printf("client/coordinator coordinator for consumer group %s is not available\n", consumerGroup)
964
965 // This is very ugly, but this scenario will only happen once per cluster.
966 // The __consumer_offsets topic only has to be created one time.
967 // The number of partitions not configurable, but partition 0 should always exist.
968 if _, err := client.Leader("__consumer_offsets", 0); err != nil {
969 Logger.Printf("client/coordinator the __consumer_offsets topic is not initialized completely yet. Waiting 2 seconds...\n")
970 time.Sleep(2 * time.Second)
971 }
972
973 return retry(ErrConsumerCoordinatorNotAvailable)
974 case ErrGroupAuthorizationFailed:
975 Logger.Printf("client was not authorized to access group %s while attempting to find coordinator", consumerGroup)
976 return retry(ErrGroupAuthorizationFailed)
977
978 default:
979 return nil, response.Err
980 }
981 }
982
983 Logger.Println("client/coordinator no available broker to send consumer metadata request to")
984 client.resurrectDeadBrokers()
985 return retry(ErrOutOfBrokers)
986}
987
988// nopCloserClient embeds an existing Client, but disables
989// the Close method (yet all other methods pass
990// through unchanged). This is for use in larger structs
991// where it is undesirable to close the client that was
992// passed in by the caller.
993type nopCloserClient struct {
994 Client
995}
996
997// Close intercepts and purposely does not call the underlying
998// client's Close() method.
999func (ncc *nopCloserClient) Close() error {
1000 return nil
1001}