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