cubefs
1124 строки · 34.3 Кб
1package sarama2
3import (4"errors"5"math/rand"6"sort"7"sync"8"time"9)
10
11// Client is a generic Kafka client. It manages connections to one or more Kafka brokers.
12// You MUST call Close() on a client to avoid leaks, it will not be garbage-collected
13// automatically when it passes out of scope. It is safe to share a client amongst many
14// users, however Kafka will process requests from a single client strictly in serial,
15// so it is generally more efficient to use the default one client per producer/consumer.
16type Client interface {17// Config returns the Config struct of the client. This struct should not be18// altered after it has been created.19Config() *Config20
21// Controller returns the cluster controller broker. It will return a22// locally cached value if it's available. You can call RefreshController23// to update the cached value. Requires Kafka 0.10 or higher.24Controller() (*Broker, error)25
26// RefreshController retrieves the cluster controller from fresh metadata27// and stores it in the local cache. Requires Kafka 0.10 or higher.28RefreshController() (*Broker, error)29
30// Brokers returns the current set of active brokers as retrieved from cluster metadata.31Brokers() []*Broker32
33// Broker returns the active Broker if available for the broker ID.34Broker(brokerID int32) (*Broker, error)35
36// Topics returns the set of available topics as retrieved from cluster metadata.37Topics() ([]string, error)38
39// Partitions returns the sorted list of all partition IDs for the given topic.40Partitions(topic string) ([]int32, error)41
42// WritablePartitions returns the sorted list of all writable partition IDs for43// the given topic, where "writable" means "having a valid leader accepting44// writes".45WritablePartitions(topic string) ([]int32, error)46
47// Leader returns the broker object that is the leader of the current48// topic/partition, as determined by querying the cluster metadata.49Leader(topic string, partitionID int32) (*Broker, error)50
51// Replicas returns the set of all replica IDs for the given partition.52Replicas(topic string, partitionID int32) ([]int32, error)53
54// InSyncReplicas returns the set of all in-sync replica IDs for the given55// partition. In-sync replicas are replicas which are fully caught up with56// the partition leader.57InSyncReplicas(topic string, partitionID int32) ([]int32, error)58
59// OfflineReplicas returns the set of all offline replica IDs for the given60// partition. Offline replicas are replicas which are offline61OfflineReplicas(topic string, partitionID int32) ([]int32, error)62
63// RefreshBrokers takes a list of addresses to be used as seed brokers.64// Existing broker connections are closed and the updated list of seed brokers65// will be used for the next metadata fetch.66RefreshBrokers(addrs []string) error67
68// RefreshMetadata takes a list of topics and queries the cluster to refresh the69// available metadata for those topics. If no topics are provided, it will refresh70// metadata for all topics.71RefreshMetadata(topics ...string) error72
73// GetOffset queries the cluster to get the most recent available offset at the74// given time (in milliseconds) on the topic/partition combination.75// Time should be OffsetOldest for the earliest available offset,76// OffsetNewest for the offset of the message that will be produced next, or a time.77GetOffset(topic string, partitionID int32, time int64) (int64, error)78
79// Coordinator returns the coordinating broker for a consumer group. It will80// return a locally cached value if it's available. You can call81// RefreshCoordinator to update the cached value. This function only works on82// Kafka 0.8.2 and higher.83Coordinator(consumerGroup string) (*Broker, error)84
85// RefreshCoordinator retrieves the coordinator for a consumer group and stores it86// in local cache. This function only works on Kafka 0.8.2 and higher.87RefreshCoordinator(consumerGroup string) error88
89// InitProducerID retrieves information required for Idempotent Producer90InitProducerID() (*InitProducerIDResponse, error)91
92// Close shuts down all broker connections managed by this client. It is required93// to call this function before a client object passes out of scope, as it will94// otherwise leak memory. You must close any Producers or Consumers using a client95// before you close the client.96Close() error97
98// Closed returns true if the client has already had Close called on it99Closed() bool100}
101
102const (103// OffsetNewest stands for the log head offset, i.e. the offset that will be104// assigned to the next message that will be produced to the partition. You105// can send this to a client's GetOffset method to get this offset, or when106// calling ConsumePartition to start consuming new messages.107OffsetNewest int64 = -1108// OffsetOldest stands for the oldest offset available on the broker for a109// partition. You can send this to a client's GetOffset method to get this110// offset, or when calling ConsumePartition to start consuming from the111// oldest offset that is still available on the broker.112OffsetOldest int64 = -2113)
114
115type client struct {116conf *Config117closer, closed chan none // for shutting down background metadata updater118
119// the broker addresses given to us through the constructor are not guaranteed to be returned in120// the cluster metadata (I *think* it only returns brokers who are currently leading partitions?)121// so we store them separately122seedBrokers []*Broker123deadSeeds []*Broker124
125controllerID int32 // cluster controller broker id126brokers map[int32]*Broker // maps broker ids to brokers127metadata map[string]map[int32]*PartitionMetadata // maps topics to partition ids to metadata128metadataTopics map[string]none // topics that need to collect metadata129coordinators map[string]int32 // Maps consumer group names to coordinating broker IDs130
131// If the number of partitions is large, we can get some churn calling cachedPartitions,132// so the result is cached. It is important to update this value whenever metadata is changed133cachedPartitionsResults map[string][maxPartitionIndex][]int32134
135lock sync.RWMutex // protects access to the maps that hold cluster state.136}
137
138// NewClient creates a new Client. It connects to one of the given broker addresses
139// and uses that broker to automatically fetch metadata on the rest of the kafka cluster. If metadata cannot
140// be retrieved from any of the given broker addresses, the client is not created.
141func NewClient(addrs []string, conf *Config) (Client, error) {142DebugLogger.Println("Initializing new client")143
144if conf == nil {145conf = NewConfig()146}147
148if err := conf.Validate(); err != nil {149return nil, err150}151
152if len(addrs) < 1 {153return nil, ConfigurationError("You must provide at least one broker address")154}155
156client := &client{157conf: conf,158closer: make(chan none),159closed: make(chan none),160brokers: make(map[int32]*Broker),161metadata: make(map[string]map[int32]*PartitionMetadata),162metadataTopics: make(map[string]none),163cachedPartitionsResults: make(map[string][maxPartitionIndex][]int32),164coordinators: make(map[string]int32),165}166
167client.randomizeSeedBrokers(addrs)168
169if conf.Metadata.Full {170// do an initial fetch of all cluster metadata by specifying an empty list of topics171err := client.RefreshMetadata()172if err == nil {173} else if errors.Is(err, ErrLeaderNotAvailable) || errors.Is(err, ErrReplicaNotAvailable) || errors.Is(err, ErrTopicAuthorizationFailed) || errors.Is(err, ErrClusterAuthorizationFailed) {174// indicates that maybe part of the cluster is down, but is not fatal to creating the client175Logger.Println(err)176} else {177close(client.closed) // we haven't started the background updater yet, so we have to do this manually178_ = client.Close()179return nil, err180}181}182go withRecover(client.backgroundMetadataUpdater)183
184DebugLogger.Println("Successfully initialized new client")185
186return client, nil187}
188
189func (client *client) Config() *Config {190return client.conf191}
192
193func (client *client) Brokers() []*Broker {194client.lock.RLock()195defer client.lock.RUnlock()196brokers := make([]*Broker, 0, len(client.brokers))197for _, broker := range client.brokers {198brokers = append(brokers, broker)199}200return brokers201}
202
203func (client *client) Broker(brokerID int32) (*Broker, error) {204client.lock.RLock()205defer client.lock.RUnlock()206broker, ok := client.brokers[brokerID]207if !ok {208return nil, ErrBrokerNotFound209}210_ = broker.Open(client.conf)211return broker, nil212}
213
214func (client *client) InitProducerID() (*InitProducerIDResponse, error) {215brokerErrors := make([]error, 0)216for broker := client.any(); broker != nil; broker = client.any() {217var response *InitProducerIDResponse218req := &InitProducerIDRequest{}219
220response, err := broker.InitProducerID(req)221if err == nil {222return response, nil223} else {224// some error, remove that broker and try again225Logger.Printf("Client got error from broker %d when issuing InitProducerID : %v\n", broker.ID(), err)226_ = broker.Close()227brokerErrors = append(brokerErrors, err)228client.deregisterBroker(broker)229}230}231
232return nil, Wrap(ErrOutOfBrokers, brokerErrors...)233}
234
235func (client *client) Close() error {236if client.Closed() {237// Chances are this is being called from a defer() and the error will go unobserved238// so we go ahead and log the event in this case.239Logger.Printf("Close() called on already closed client")240return ErrClosedClient241}242
243// shutdown and wait for the background thread before we take the lock, to avoid races244close(client.closer)245<-client.closed246
247client.lock.Lock()248defer client.lock.Unlock()249DebugLogger.Println("Closing Client")250
251for _, broker := range client.brokers {252safeAsyncClose(broker)253}254
255for _, broker := range client.seedBrokers {256safeAsyncClose(broker)257}258
259client.brokers = nil260client.metadata = nil261client.metadataTopics = nil262
263return nil264}
265
266func (client *client) Closed() bool {267client.lock.RLock()268defer client.lock.RUnlock()269
270return client.brokers == nil271}
272
273func (client *client) Topics() ([]string, error) {274if client.Closed() {275return nil, ErrClosedClient276}277
278client.lock.RLock()279defer client.lock.RUnlock()280
281ret := make([]string, 0, len(client.metadata))282for topic := range client.metadata {283ret = append(ret, topic)284}285
286return ret, nil287}
288
289func (client *client) MetadataTopics() ([]string, error) {290if client.Closed() {291return nil, ErrClosedClient292}293
294client.lock.RLock()295defer client.lock.RUnlock()296
297ret := make([]string, 0, len(client.metadataTopics))298for topic := range client.metadataTopics {299ret = append(ret, topic)300}301
302return ret, nil303}
304
305func (client *client) Partitions(topic string) ([]int32, error) {306if client.Closed() {307return nil, ErrClosedClient308}309
310partitions := client.cachedPartitions(topic, allPartitions)311
312if len(partitions) == 0 {313err := client.RefreshMetadata(topic)314if err != nil {315return nil, err316}317partitions = client.cachedPartitions(topic, allPartitions)318}319
320// no partitions found after refresh metadata321if len(partitions) == 0 {322return nil, ErrUnknownTopicOrPartition323}324
325return partitions, nil326}
327
328func (client *client) WritablePartitions(topic string) ([]int32, error) {329if client.Closed() {330return nil, ErrClosedClient331}332
333partitions := client.cachedPartitions(topic, writablePartitions)334
335// len==0 catches when it's nil (no such topic) and the odd case when every single336// partition is undergoing leader election simultaneously. Callers have to be able to handle337// this function returning an empty slice (which is a valid return value) but catching it338// here the first time (note we *don't* catch it below where we return ErrUnknownTopicOrPartition) triggers339// a metadata refresh as a nicety so callers can just try again and don't have to manually340// trigger a refresh (otherwise they'd just keep getting a stale cached copy).341if len(partitions) == 0 {342err := client.RefreshMetadata(topic)343if err != nil {344return nil, err345}346partitions = client.cachedPartitions(topic, writablePartitions)347}348
349if partitions == nil {350return nil, ErrUnknownTopicOrPartition351}352
353return partitions, nil354}
355
356func (client *client) Replicas(topic string, partitionID int32) ([]int32, error) {357if client.Closed() {358return nil, ErrClosedClient359}360
361metadata := client.cachedMetadata(topic, partitionID)362
363if metadata == nil {364err := client.RefreshMetadata(topic)365if err != nil {366return nil, err367}368metadata = client.cachedMetadata(topic, partitionID)369}370
371if metadata == nil {372return nil, ErrUnknownTopicOrPartition373}374
375if errors.Is(metadata.Err, ErrReplicaNotAvailable) {376return dupInt32Slice(metadata.Replicas), metadata.Err377}378return dupInt32Slice(metadata.Replicas), nil379}
380
381func (client *client) InSyncReplicas(topic string, partitionID int32) ([]int32, error) {382if client.Closed() {383return nil, ErrClosedClient384}385
386metadata := client.cachedMetadata(topic, partitionID)387
388if metadata == nil {389err := client.RefreshMetadata(topic)390if err != nil {391return nil, err392}393metadata = client.cachedMetadata(topic, partitionID)394}395
396if metadata == nil {397return nil, ErrUnknownTopicOrPartition398}399
400if errors.Is(metadata.Err, ErrReplicaNotAvailable) {401return dupInt32Slice(metadata.Isr), metadata.Err402}403return dupInt32Slice(metadata.Isr), nil404}
405
406func (client *client) OfflineReplicas(topic string, partitionID int32) ([]int32, error) {407if client.Closed() {408return nil, ErrClosedClient409}410
411metadata := client.cachedMetadata(topic, partitionID)412
413if metadata == nil {414err := client.RefreshMetadata(topic)415if err != nil {416return nil, err417}418metadata = client.cachedMetadata(topic, partitionID)419}420
421if metadata == nil {422return nil, ErrUnknownTopicOrPartition423}424
425if errors.Is(metadata.Err, ErrReplicaNotAvailable) {426return dupInt32Slice(metadata.OfflineReplicas), metadata.Err427}428return dupInt32Slice(metadata.OfflineReplicas), nil429}
430
431func (client *client) Leader(topic string, partitionID int32) (*Broker, error) {432if client.Closed() {433return nil, ErrClosedClient434}435
436leader, err := client.cachedLeader(topic, partitionID)437
438if leader == nil {439err = client.RefreshMetadata(topic)440if err != nil {441return nil, err442}443leader, err = client.cachedLeader(topic, partitionID)444}445
446return leader, err447}
448
449func (client *client) RefreshBrokers(addrs []string) error {450if client.Closed() {451return ErrClosedClient452}453
454client.lock.Lock()455defer client.lock.Unlock()456
457for _, broker := range client.brokers {458_ = broker.Close()459delete(client.brokers, broker.ID())460}461
462for _, broker := range client.seedBrokers {463_ = broker.Close()464}465
466for _, broker := range client.deadSeeds {467_ = broker.Close()468}469
470client.seedBrokers = nil471client.deadSeeds = nil472
473client.randomizeSeedBrokers(addrs)474
475return nil476}
477
478func (client *client) RefreshMetadata(topics ...string) error {479if client.Closed() {480return ErrClosedClient481}482
483// Prior to 0.8.2, Kafka will throw exceptions on an empty topic and not return a proper484// error. This handles the case by returning an error instead of sending it485// off to Kafka. See: https://github.com/Shopify/sarama/pull/38#issuecomment-26362310486for _, topic := range topics {487if topic == "" {488return ErrInvalidTopic // this is the error that 0.8.2 and later correctly return489}490}491
492deadline := time.Time{}493if client.conf.Metadata.Timeout > 0 {494deadline = time.Now().Add(client.conf.Metadata.Timeout)495}496return client.tryRefreshMetadata(topics, client.conf.Metadata.Retry.Max, deadline)497}
498
499func (client *client) GetOffset(topic string, partitionID int32, time int64) (int64, error) {500if client.Closed() {501return -1, ErrClosedClient502}503
504offset, err := client.getOffset(topic, partitionID, time)505if err != nil {506if err := client.RefreshMetadata(topic); err != nil {507return -1, err508}509return client.getOffset(topic, partitionID, time)510}511
512return offset, err513}
514
515func (client *client) Controller() (*Broker, error) {516if client.Closed() {517return nil, ErrClosedClient518}519
520if !client.conf.Version.IsAtLeast(V0_10_0_0) {521return nil, ErrUnsupportedVersion522}523
524controller := client.cachedController()525if controller == nil {526if err := client.refreshMetadata(); err != nil {527return nil, err528}529controller = client.cachedController()530}531
532if controller == nil {533return nil, ErrControllerNotAvailable534}535
536_ = controller.Open(client.conf)537return controller, nil538}
539
540// deregisterController removes the cached controllerID
541func (client *client) deregisterController() {542client.lock.Lock()543defer client.lock.Unlock()544if controller, ok := client.brokers[client.controllerID]; ok {545_ = controller.Close()546delete(client.brokers, client.controllerID)547}548}
549
550// RefreshController retrieves the cluster controller from fresh metadata
551// and stores it in the local cache. Requires Kafka 0.10 or higher.
552func (client *client) RefreshController() (*Broker, error) {553if client.Closed() {554return nil, ErrClosedClient555}556
557client.deregisterController()558
559if err := client.refreshMetadata(); err != nil {560return nil, err561}562
563controller := client.cachedController()564if controller == nil {565return nil, ErrControllerNotAvailable566}567
568_ = controller.Open(client.conf)569return controller, nil570}
571
572func (client *client) Coordinator(consumerGroup string) (*Broker, error) {573if client.Closed() {574return nil, ErrClosedClient575}576
577coordinator := client.cachedCoordinator(consumerGroup)578
579if coordinator == nil {580if err := client.RefreshCoordinator(consumerGroup); err != nil {581return nil, err582}583coordinator = client.cachedCoordinator(consumerGroup)584}585
586if coordinator == nil {587return nil, ErrConsumerCoordinatorNotAvailable588}589
590_ = coordinator.Open(client.conf)591return coordinator, nil592}
593
594func (client *client) RefreshCoordinator(consumerGroup string) error {595if client.Closed() {596return ErrClosedClient597}598
599response, err := client.getConsumerMetadata(consumerGroup, client.conf.Metadata.Retry.Max)600if err != nil {601return err602}603
604client.lock.Lock()605defer client.lock.Unlock()606client.registerBroker(response.Coordinator)607client.coordinators[consumerGroup] = response.Coordinator.ID()608return nil609}
610
611// private broker management helpers
612
613func (client *client) randomizeSeedBrokers(addrs []string) {614random := rand.New(rand.NewSource(time.Now().UnixNano()))615for _, index := range random.Perm(len(addrs)) {616client.seedBrokers = append(client.seedBrokers, NewBroker(addrs[index]))617}618}
619
620func (client *client) updateBroker(brokers []*Broker) {621currentBroker := make(map[int32]*Broker, len(brokers))622
623for _, broker := range brokers {624currentBroker[broker.ID()] = broker625if client.brokers[broker.ID()] == nil { // add new broker626client.brokers[broker.ID()] = broker627DebugLogger.Printf("client/brokers registered new broker #%d at %s", broker.ID(), broker.Addr())628} else if broker.Addr() != client.brokers[broker.ID()].Addr() { // replace broker with new address629safeAsyncClose(client.brokers[broker.ID()])630client.brokers[broker.ID()] = broker631Logger.Printf("client/brokers replaced registered broker #%d with %s", broker.ID(), broker.Addr())632}633}634
635for id, broker := range client.brokers {636if _, exist := currentBroker[id]; !exist { // remove old broker637safeAsyncClose(broker)638delete(client.brokers, id)639Logger.Printf("client/broker remove invalid broker #%d with %s", broker.ID(), broker.Addr())640}641}642}
643
644// registerBroker makes sure a broker received by a Metadata or Coordinator request is registered
645// in the brokers map. It returns the broker that is registered, which may be the provided broker,
646// or a previously registered Broker instance. You must hold the write lock before calling this function.
647func (client *client) registerBroker(broker *Broker) {648if client.brokers == nil {649Logger.Printf("cannot register broker #%d at %s, client already closed", broker.ID(), broker.Addr())650return651}652
653if client.brokers[broker.ID()] == nil {654client.brokers[broker.ID()] = broker655DebugLogger.Printf("client/brokers registered new broker #%d at %s", broker.ID(), broker.Addr())656} else if broker.Addr() != client.brokers[broker.ID()].Addr() {657safeAsyncClose(client.brokers[broker.ID()])658client.brokers[broker.ID()] = broker659Logger.Printf("client/brokers replaced registered broker #%d with %s", broker.ID(), broker.Addr())660}661}
662
663// deregisterBroker removes a broker from the seedsBroker list, and if it's
664// not the seedbroker, removes it from brokers map completely.
665func (client *client) deregisterBroker(broker *Broker) {666client.lock.Lock()667defer client.lock.Unlock()668
669if len(client.seedBrokers) > 0 && broker == client.seedBrokers[0] {670client.deadSeeds = append(client.deadSeeds, broker)671client.seedBrokers = client.seedBrokers[1:]672} else {673// we do this so that our loop in `tryRefreshMetadata` doesn't go on forever,674// but we really shouldn't have to; once that loop is made better this case can be675// removed, and the function generally can be renamed from `deregisterBroker` to676// `nextSeedBroker` or something677DebugLogger.Printf("client/brokers deregistered broker #%d at %s", broker.ID(), broker.Addr())678delete(client.brokers, broker.ID())679}680}
681
682func (client *client) resurrectDeadBrokers() {683client.lock.Lock()684defer client.lock.Unlock()685
686Logger.Printf("client/brokers resurrecting %d dead seed brokers", len(client.deadSeeds))687client.seedBrokers = append(client.seedBrokers, client.deadSeeds...)688client.deadSeeds = nil689}
690
691func (client *client) any() *Broker {692client.lock.RLock()693defer client.lock.RUnlock()694
695if len(client.seedBrokers) > 0 {696_ = client.seedBrokers[0].Open(client.conf)697return client.seedBrokers[0]698}699
700// not guaranteed to be random *or* deterministic701for _, broker := range client.brokers {702_ = broker.Open(client.conf)703return broker704}705
706return nil707}
708
709// private caching/lazy metadata helpers
710
711type partitionType int712
713const (714allPartitions partitionType = iota715writablePartitions
716// If you add any more types, update the partition cache in update()717
718// Ensure this is the last partition type value719maxPartitionIndex
720)
721
722func (client *client) cachedMetadata(topic string, partitionID int32) *PartitionMetadata {723client.lock.RLock()724defer client.lock.RUnlock()725
726partitions := client.metadata[topic]727if partitions != nil {728return partitions[partitionID]729}730
731return nil732}
733
734func (client *client) cachedPartitions(topic string, partitionSet partitionType) []int32 {735client.lock.RLock()736defer client.lock.RUnlock()737
738partitions, exists := client.cachedPartitionsResults[topic]739
740if !exists {741return nil742}743return partitions[partitionSet]744}
745
746func (client *client) setPartitionCache(topic string, partitionSet partitionType) []int32 {747partitions := client.metadata[topic]748
749if partitions == nil {750return nil751}752
753ret := make([]int32, 0, len(partitions))754for _, partition := range partitions {755if partitionSet == writablePartitions && errors.Is(partition.Err, ErrLeaderNotAvailable) {756continue757}758ret = append(ret, partition.ID)759}760
761sort.Sort(int32Slice(ret))762return ret763}
764
765func (client *client) cachedLeader(topic string, partitionID int32) (*Broker, error) {766client.lock.RLock()767defer client.lock.RUnlock()768
769partitions := client.metadata[topic]770if partitions != nil {771metadata, ok := partitions[partitionID]772if ok {773if errors.Is(metadata.Err, ErrLeaderNotAvailable) {774return nil, ErrLeaderNotAvailable775}776b := client.brokers[metadata.Leader]777if b == nil {778return nil, ErrLeaderNotAvailable779}780_ = b.Open(client.conf)781return b, nil782}783}784
785return nil, ErrUnknownTopicOrPartition786}
787
788func (client *client) getOffset(topic string, partitionID int32, time int64) (int64, error) {789broker, err := client.Leader(topic, partitionID)790if err != nil {791return -1, err792}793
794request := &OffsetRequest{}795if client.conf.Version.IsAtLeast(V0_10_1_0) {796request.Version = 1797}798request.AddBlock(topic, partitionID, time, 1)799
800response, err := broker.GetAvailableOffsets(request)801if err != nil {802_ = broker.Close()803return -1, err804}805
806block := response.GetBlock(topic, partitionID)807if block == nil {808_ = broker.Close()809return -1, ErrIncompleteResponse810}811if !errors.Is(block.Err, ErrNoError) {812return -1, block.Err813}814if len(block.Offsets) != 1 {815return -1, ErrOffsetOutOfRange816}817
818return block.Offsets[0], nil819}
820
821// core metadata update logic
822
823func (client *client) backgroundMetadataUpdater() {824defer close(client.closed)825
826if client.conf.Metadata.RefreshFrequency == time.Duration(0) {827return828}829
830ticker := time.NewTicker(client.conf.Metadata.RefreshFrequency)831defer ticker.Stop()832
833for {834select {835case <-ticker.C:836if err := client.refreshMetadata(); err != nil {837Logger.Println("Client background metadata update:", err)838}839case <-client.closer:840return841}842}843}
844
845func (client *client) refreshMetadata() error {846var topics []string847
848if !client.conf.Metadata.Full {849if specificTopics, err := client.MetadataTopics(); err != nil {850return err851} else if len(specificTopics) == 0 {852return ErrNoTopicsToUpdateMetadata853} else {854topics = specificTopics855}856}857
858if err := client.RefreshMetadata(topics...); err != nil {859return err860}861
862return nil863}
864
865func (client *client) tryRefreshMetadata(topics []string, attemptsRemaining int, deadline time.Time) error {866pastDeadline := func(backoff time.Duration) bool {867if !deadline.IsZero() && time.Now().Add(backoff).After(deadline) {868// we are past the deadline869return true870}871return false872}873retry := func(err error) error {874if attemptsRemaining > 0 {875backoff := client.computeBackoff(attemptsRemaining)876if pastDeadline(backoff) {877Logger.Println("client/metadata skipping last retries as we would go past the metadata timeout")878return err879}880Logger.Printf("client/metadata retrying after %dms... (%d attempts remaining)\n", backoff/time.Millisecond, attemptsRemaining)881if backoff > 0 {882time.Sleep(backoff)883}884return client.tryRefreshMetadata(topics, attemptsRemaining-1, deadline)885}886return err887}888
889broker := client.any()890brokerErrors := make([]error, 0)891for ; broker != nil && !pastDeadline(0); broker = client.any() {892allowAutoTopicCreation := client.conf.Metadata.AllowAutoTopicCreation893if len(topics) > 0 {894DebugLogger.Printf("client/metadata fetching metadata for %v from broker %s\n", topics, broker.addr)895} else {896allowAutoTopicCreation = false897DebugLogger.Printf("client/metadata fetching metadata for all topics from broker %s\n", broker.addr)898}899
900req := &MetadataRequest{Topics: topics, AllowAutoTopicCreation: allowAutoTopicCreation}901if client.conf.Version.IsAtLeast(V1_0_0_0) {902req.Version = 5903} else if client.conf.Version.IsAtLeast(V0_10_0_0) {904req.Version = 1905}906response, err := broker.GetMetadata(req)907var kerror KError908var packetEncodingError PacketEncodingError909if err == nil {910allKnownMetaData := len(topics) == 0911// valid response, use it912shouldRetry, err := client.updateMetadata(response, allKnownMetaData)913if shouldRetry {914Logger.Println("client/metadata found some partitions to be leaderless")915return retry(err) // note: err can be nil916}917return err918} else if errors.As(err, &packetEncodingError) {919// didn't even send, return the error920return err921} else if errors.As(err, &kerror) {922// if SASL auth error return as this _should_ be a non retryable err for all brokers923if errors.Is(err, ErrSASLAuthenticationFailed) {924Logger.Println("client/metadata failed SASL authentication")925return err926}927
928if errors.Is(err, ErrTopicAuthorizationFailed) {929Logger.Println("client is not authorized to access this topic. The topics were: ", topics)930return err931}932// else remove that broker and try again933Logger.Printf("client/metadata got error from broker %d while fetching metadata: %v\n", broker.ID(), err)934_ = broker.Close()935client.deregisterBroker(broker)936} else {937// some other error, remove that broker and try again938Logger.Printf("client/metadata got error from broker %d while fetching metadata: %v\n", broker.ID(), err)939brokerErrors = append(brokerErrors, err)940_ = broker.Close()941client.deregisterBroker(broker)942}943}944
945error := Wrap(ErrOutOfBrokers, brokerErrors...)946if broker != nil {947Logger.Printf("client/metadata not fetching metadata from broker %s as we would go past the metadata timeout\n", broker.addr)948return retry(error)949}950
951Logger.Println("client/metadata no available broker to send metadata request to")952client.resurrectDeadBrokers()953return retry(error)954}
955
956// if no fatal error, returns a list of topics that need retrying due to ErrLeaderNotAvailable
957func (client *client) updateMetadata(data *MetadataResponse, allKnownMetaData bool) (retry bool, err error) {958if client.Closed() {959return960}961
962client.lock.Lock()963defer client.lock.Unlock()964
965// For all the brokers we received:966// - if it is a new ID, save it967// - if it is an existing ID, but the address we have is stale, discard the old one and save it968// - if some brokers is not exist in it, remove old broker969// - otherwise ignore it, replacing our existing one would just bounce the connection970client.updateBroker(data.Brokers)971
972client.controllerID = data.ControllerID973
974if allKnownMetaData {975client.metadata = make(map[string]map[int32]*PartitionMetadata)976client.metadataTopics = make(map[string]none)977client.cachedPartitionsResults = make(map[string][maxPartitionIndex][]int32)978}979for _, topic := range data.Topics {980// topics must be added firstly to `metadataTopics` to guarantee that all981// requested topics must be recorded to keep them trackable for periodically982// metadata refresh.983if _, exists := client.metadataTopics[topic.Name]; !exists {984client.metadataTopics[topic.Name] = none{}985}986delete(client.metadata, topic.Name)987delete(client.cachedPartitionsResults, topic.Name)988
989switch topic.Err {990case ErrNoError:991// no-op992case ErrInvalidTopic, ErrTopicAuthorizationFailed: // don't retry, don't store partial results993err = topic.Err994continue995case ErrUnknownTopicOrPartition: // retry, do not store partial partition results996err = topic.Err997retry = true998continue999case ErrLeaderNotAvailable: // retry, but store partial partition results1000retry = true1001default: // don't retry, don't store partial results1002Logger.Printf("Unexpected topic-level metadata error: %s", topic.Err)1003err = topic.Err1004continue1005}1006
1007client.metadata[topic.Name] = make(map[int32]*PartitionMetadata, len(topic.Partitions))1008for _, partition := range topic.Partitions {1009client.metadata[topic.Name][partition.ID] = partition1010if errors.Is(partition.Err, ErrLeaderNotAvailable) {1011retry = true1012}1013}1014
1015var partitionCache [maxPartitionIndex][]int321016partitionCache[allPartitions] = client.setPartitionCache(topic.Name, allPartitions)1017partitionCache[writablePartitions] = client.setPartitionCache(topic.Name, writablePartitions)1018client.cachedPartitionsResults[topic.Name] = partitionCache1019}1020
1021return1022}
1023
1024func (client *client) cachedCoordinator(consumerGroup string) *Broker {1025client.lock.RLock()1026defer client.lock.RUnlock()1027if coordinatorID, ok := client.coordinators[consumerGroup]; ok {1028return client.brokers[coordinatorID]1029}1030return nil1031}
1032
1033func (client *client) cachedController() *Broker {1034client.lock.RLock()1035defer client.lock.RUnlock()1036
1037return client.brokers[client.controllerID]1038}
1039
1040func (client *client) computeBackoff(attemptsRemaining int) time.Duration {1041if client.conf.Metadata.Retry.BackoffFunc != nil {1042maxRetries := client.conf.Metadata.Retry.Max1043retries := maxRetries - attemptsRemaining1044return client.conf.Metadata.Retry.BackoffFunc(retries, maxRetries)1045}1046return client.conf.Metadata.Retry.Backoff1047}
1048
1049func (client *client) getConsumerMetadata(consumerGroup string, attemptsRemaining int) (*FindCoordinatorResponse, error) {1050retry := func(err error) (*FindCoordinatorResponse, error) {1051if attemptsRemaining > 0 {1052backoff := client.computeBackoff(attemptsRemaining)1053Logger.Printf("client/coordinator retrying after %dms... (%d attempts remaining)\n", backoff/time.Millisecond, attemptsRemaining)1054time.Sleep(backoff)1055return client.getConsumerMetadata(consumerGroup, attemptsRemaining-1)1056}1057return nil, err1058}1059
1060brokerErrors := make([]error, 0)1061for broker := client.any(); broker != nil; broker = client.any() {1062DebugLogger.Printf("client/coordinator requesting coordinator for consumergroup %s from %s\n", consumerGroup, broker.Addr())1063
1064request := new(FindCoordinatorRequest)1065request.CoordinatorKey = consumerGroup1066request.CoordinatorType = CoordinatorGroup1067
1068response, err := broker.FindCoordinator(request)1069if err != nil {1070Logger.Printf("client/coordinator request to broker %s failed: %s\n", broker.Addr(), err)1071
1072var packetEncodingError PacketEncodingError1073if errors.As(err, &packetEncodingError) {1074return nil, err1075} else {1076_ = broker.Close()1077brokerErrors = append(brokerErrors, err)1078client.deregisterBroker(broker)1079continue1080}1081}1082
1083if errors.Is(response.Err, ErrNoError) {1084DebugLogger.Printf("client/coordinator coordinator for consumergroup %s is #%d (%s)\n", consumerGroup, response.Coordinator.ID(), response.Coordinator.Addr())1085return response, nil1086} else if errors.Is(response.Err, ErrConsumerCoordinatorNotAvailable) {1087Logger.Printf("client/coordinator coordinator for consumer group %s is not available\n", consumerGroup)1088
1089// This is very ugly, but this scenario will only happen once per cluster.1090// The __consumer_offsets topic only has to be created one time.1091// The number of partitions not configurable, but partition 0 should always exist.1092if _, err := client.Leader("__consumer_offsets", 0); err != nil {1093Logger.Printf("client/coordinator the __consumer_offsets topic is not initialized completely yet. Waiting 2 seconds...\n")1094time.Sleep(2 * time.Second)1095}1096
1097return retry(ErrConsumerCoordinatorNotAvailable)1098} else if errors.Is(response.Err, ErrGroupAuthorizationFailed) {1099Logger.Printf("client was not authorized to access group %s while attempting to find coordinator", consumerGroup)1100return retry(ErrGroupAuthorizationFailed)1101} else {1102return nil, response.Err1103}1104}1105
1106Logger.Println("client/coordinator no available broker to send consumer metadata request to")1107client.resurrectDeadBrokers()1108return retry(Wrap(ErrOutOfBrokers, brokerErrors...))1109}
1110
1111// nopCloserClient embeds an existing Client, but disables
1112// the Close method (yet all other methods pass
1113// through unchanged). This is for use in larger structs
1114// where it is undesirable to close the client that was
1115// passed in by the caller.
1116type nopCloserClient struct {1117Client
1118}
1119
1120// Close intercepts and purposely does not call the underlying
1121// client's Close() method.
1122func (ncc *nopCloserClient) Close() error {1123return nil1124}
1125