client.go 22 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676677678679680681682683684685686687688689690691692693694695696697698699700701702703704705706707708709710711712713714715716717718719720
  1. package sarama
  2. import (
  3. "sort"
  4. "sync"
  5. "time"
  6. )
  7. // Client is a generic Kafka client. It manages connections to one or more Kafka brokers.
  8. // You MUST call Close() on a client to avoid leaks, it will not be garbage-collected
  9. // automatically when it passes out of scope. A single client can be safely shared by
  10. // multiple concurrent Producers and Consumers.
  11. type Client interface {
  12. // Config returns the Config struct of the client. This struct should not be altered after it
  13. // has been created.
  14. Config() *Config
  15. // Topics returns the set of available topics as retrieved from the cluster metadata.
  16. Topics() ([]string, error)
  17. // Partitions returns the sorted list of all partition IDs for the given topic.
  18. Partitions(topic string) ([]int32, error)
  19. // WritablePartitions returns the sorted list of all writable partition IDs for the given topic,
  20. // where "writable" means "having a valid leader accepting writes".
  21. WritablePartitions(topic string) ([]int32, error)
  22. // Leader returns the broker object that is the leader of the current topic/partition, as
  23. // determined by querying the cluster metadata.
  24. Leader(topic string, partitionID int32) (*Broker, error)
  25. // Replicas returns the set of all replica IDs for the given partition.
  26. Replicas(topic string, partitionID int32) ([]int32, error)
  27. // RefreshMetadata takes a list of topics and queries the cluster to refresh the
  28. // available metadata for those topics. If no topics are provided, it will refresh metadata
  29. // for all topics.
  30. RefreshMetadata(topics ...string) error
  31. // GetOffset queries the cluster to get the most recent available offset at the given
  32. // time on the topic/partition combination. Time should be OffsetOldest for the earliest available
  33. // offset, OffsetNewest for the offset of the message that will be produced next, or a time.
  34. GetOffset(topic string, partitionID int32, time int64) (int64, error)
  35. // Coordinator returns the coordinating broker for a consumer group. It will return a locally cached
  36. // value if it's available. You can call RefreshCoordinator to update the cached value.
  37. Coordinator(consumerGroup string) (*Broker, error)
  38. // RefreshCoordinator retrieves the coordinator for a consumer group and stores it in local cache.
  39. RefreshCoordinator(consumerGroup string) error
  40. // Close shuts down all broker connections managed by this client. It is required to call this function before
  41. // a client object passes out of scope, as it will otherwise leak memory. You must close any Producers or Consumers
  42. // using a client before you close the client.
  43. Close() error
  44. // Closed returns true if the client has already had Close called on it
  45. Closed() bool
  46. }
  47. const (
  48. // OffsetNewest stands for the log head offset, i.e. the offset that will be assigned to the next message
  49. // that will be produced to the partition. You can send this to a client's GetOffset method to get this
  50. // offset, or when calling ConsumePartition to start consuming new messages.
  51. OffsetNewest int64 = -1
  52. // OffsetOldest stands for the oldest offset available on the broker for a partition. You can send this
  53. // to a client's GetOffset method to get this offset, or when calling ConsumePartition to start consuming
  54. // from the oldest offset that is still available on the broker.
  55. OffsetOldest int64 = -2
  56. )
  57. type client struct {
  58. conf *Config
  59. closer chan none
  60. // the broker addresses given to us through the constructor are not guaranteed to be returned in
  61. // the cluster metadata (I *think* it only returns brokers who are currently leading partitions?)
  62. // so we store them separately
  63. seedBrokers []*Broker
  64. deadSeeds []*Broker
  65. brokers map[int32]*Broker // maps broker ids to brokers
  66. metadata map[string]map[int32]*PartitionMetadata // maps topics to partition ids to metadata
  67. coordinators map[string]int32 // Maps consumer group names to coordinating broker IDs
  68. // If the number of partitions is large, we can get some churn calling cachedPartitions,
  69. // so the result is cached. It is important to update this value whenever metadata is changed
  70. cachedPartitionsResults map[string][maxPartitionIndex][]int32
  71. lock sync.RWMutex // protects access to the maps that hold cluster state.
  72. }
  73. // NewClient creates a new Client. It connects to one of the given broker addresses
  74. // and uses that broker to automatically fetch metadata on the rest of the kafka cluster. If metadata cannot
  75. // be retrieved from any of the given broker addresses, the client is not created.
  76. func NewClient(addrs []string, conf *Config) (Client, error) {
  77. Logger.Println("Initializing new client")
  78. if conf == nil {
  79. conf = NewConfig()
  80. }
  81. if err := conf.Validate(); err != nil {
  82. return nil, err
  83. }
  84. if len(addrs) < 1 {
  85. return nil, ConfigurationError("You must provide at least one broker address")
  86. }
  87. client := &client{
  88. conf: conf,
  89. closer: make(chan none),
  90. brokers: make(map[int32]*Broker),
  91. metadata: make(map[string]map[int32]*PartitionMetadata),
  92. cachedPartitionsResults: make(map[string][maxPartitionIndex][]int32),
  93. coordinators: make(map[string]int32),
  94. }
  95. for _, addr := range addrs {
  96. client.seedBrokers = append(client.seedBrokers, NewBroker(addr))
  97. }
  98. // do an initial fetch of all cluster metadata by specifing an empty list of topics
  99. err := client.RefreshMetadata()
  100. switch err {
  101. case nil:
  102. break
  103. case ErrLeaderNotAvailable, ErrReplicaNotAvailable:
  104. // indicates that maybe part of the cluster is down, but is not fatal to creating the client
  105. Logger.Println(err)
  106. default:
  107. _ = client.Close()
  108. return nil, err
  109. }
  110. go withRecover(client.backgroundMetadataUpdater)
  111. Logger.Println("Successfully initialized new client")
  112. return client, nil
  113. }
  114. func (client *client) Config() *Config {
  115. return client.conf
  116. }
  117. func (client *client) Close() error {
  118. // Check to see whether the client is closed
  119. if client.Closed() {
  120. // Chances are this is being called from a defer() and the error will go unobserved
  121. // so we go ahead and log the event in this case.
  122. Logger.Printf("Close() called on already closed client")
  123. return ErrClosedClient
  124. }
  125. client.lock.Lock()
  126. defer client.lock.Unlock()
  127. Logger.Println("Closing Client")
  128. for _, broker := range client.brokers {
  129. safeAsyncClose(broker)
  130. }
  131. for _, broker := range client.seedBrokers {
  132. safeAsyncClose(broker)
  133. }
  134. client.brokers = nil
  135. client.metadata = nil
  136. close(client.closer)
  137. return nil
  138. }
  139. func (client *client) Closed() bool {
  140. return client.brokers == nil
  141. }
  142. func (client *client) Topics() ([]string, error) {
  143. // Check to see whether the client is closed
  144. if client.Closed() {
  145. return nil, ErrClosedClient
  146. }
  147. client.lock.RLock()
  148. defer client.lock.RUnlock()
  149. ret := make([]string, 0, len(client.metadata))
  150. for topic := range client.metadata {
  151. ret = append(ret, topic)
  152. }
  153. return ret, nil
  154. }
  155. func (client *client) Partitions(topic string) ([]int32, error) {
  156. // Check to see whether the client is closed
  157. if client.Closed() {
  158. return nil, ErrClosedClient
  159. }
  160. partitions := client.cachedPartitions(topic, allPartitions)
  161. if len(partitions) == 0 {
  162. err := client.RefreshMetadata(topic)
  163. if err != nil {
  164. return nil, err
  165. }
  166. partitions = client.cachedPartitions(topic, allPartitions)
  167. }
  168. if partitions == nil {
  169. return nil, ErrUnknownTopicOrPartition
  170. }
  171. return partitions, nil
  172. }
  173. func (client *client) WritablePartitions(topic string) ([]int32, error) {
  174. // Check to see whether the client is closed
  175. if client.Closed() {
  176. return nil, ErrClosedClient
  177. }
  178. partitions := client.cachedPartitions(topic, writablePartitions)
  179. // len==0 catches when it's nil (no such topic) and the odd case when every single
  180. // partition is undergoing leader election simultaneously. Callers have to be able to handle
  181. // this function returning an empty slice (which is a valid return value) but catching it
  182. // here the first time (note we *don't* catch it below where we return ErrUnknownTopicOrPartition) triggers
  183. // a metadata refresh as a nicety so callers can just try again and don't have to manually
  184. // trigger a refresh (otherwise they'd just keep getting a stale cached copy).
  185. if len(partitions) == 0 {
  186. err := client.RefreshMetadata(topic)
  187. if err != nil {
  188. return nil, err
  189. }
  190. partitions = client.cachedPartitions(topic, writablePartitions)
  191. }
  192. if partitions == nil {
  193. return nil, ErrUnknownTopicOrPartition
  194. }
  195. return partitions, nil
  196. }
  197. func (client *client) Replicas(topic string, partitionID int32) ([]int32, error) {
  198. if client.Closed() {
  199. return nil, ErrClosedClient
  200. }
  201. metadata := client.cachedMetadata(topic, partitionID)
  202. if metadata == nil {
  203. err := client.RefreshMetadata(topic)
  204. if err != nil {
  205. return nil, err
  206. }
  207. metadata = client.cachedMetadata(topic, partitionID)
  208. }
  209. if metadata == nil {
  210. return nil, ErrUnknownTopicOrPartition
  211. }
  212. if metadata.Err == ErrReplicaNotAvailable {
  213. return nil, metadata.Err
  214. }
  215. return dupeAndSort(metadata.Replicas), nil
  216. }
  217. func (client *client) Leader(topic string, partitionID int32) (*Broker, error) {
  218. leader, err := client.cachedLeader(topic, partitionID)
  219. if leader == nil {
  220. err := client.RefreshMetadata(topic)
  221. if err != nil {
  222. return nil, err
  223. }
  224. leader, err = client.cachedLeader(topic, partitionID)
  225. }
  226. return leader, err
  227. }
  228. func (client *client) RefreshMetadata(topics ...string) error {
  229. if client.Closed() {
  230. return ErrClosedClient
  231. }
  232. // Prior to 0.8.2, Kafka will throw exceptions on an empty topic and not return a proper
  233. // error. This handles the case by returning an error instead of sending it
  234. // off to Kafka. See: https://github.com/Shopify/sarama/pull/38#issuecomment-26362310
  235. for _, topic := range topics {
  236. if len(topic) == 0 {
  237. return ErrInvalidTopic // this is the error that 0.8.2 and later correctly return
  238. }
  239. }
  240. return client.tryRefreshMetadata(topics, client.conf.Metadata.Retry.Max)
  241. }
  242. func (client *client) GetOffset(topic string, partitionID int32, time int64) (int64, error) {
  243. offset, err := client.getOffset(topic, partitionID, time)
  244. if err != nil {
  245. if err := client.RefreshMetadata(topic); err != nil {
  246. return -1, err
  247. }
  248. return client.getOffset(topic, partitionID, time)
  249. }
  250. return offset, err
  251. }
  252. func (client *client) Coordinator(consumerGroup string) (*Broker, error) {
  253. coordinator := client.cachedCoordinator(consumerGroup)
  254. if coordinator == nil {
  255. if err := client.RefreshCoordinator(consumerGroup); err != nil {
  256. return nil, err
  257. }
  258. coordinator = client.cachedCoordinator(consumerGroup)
  259. }
  260. if coordinator == nil {
  261. return nil, ErrConsumerCoordinatorNotAvailable
  262. }
  263. _ = coordinator.Open(client.conf)
  264. return coordinator, nil
  265. }
  266. func (client *client) RefreshCoordinator(consumerGroup string) error {
  267. response, err := client.getConsumerMetadata(consumerGroup, client.conf.Metadata.Retry.Max)
  268. if err != nil {
  269. return err
  270. }
  271. client.lock.Lock()
  272. defer client.lock.Unlock()
  273. client.registerBroker(response.Coordinator)
  274. client.coordinators[consumerGroup] = response.Coordinator.ID()
  275. return nil
  276. }
  277. // private broker management helpers
  278. // registerBroker makes sure a broker received by a Metadata or Coordinator request is registered
  279. // in the brokers map. It returns the broker that is registered, which may be the provided broker,
  280. // or a previously registered Broker instance. You must hold the write lock before calling this function.
  281. func (client *client) registerBroker(broker *Broker) {
  282. if client.brokers[broker.ID()] == nil {
  283. client.brokers[broker.ID()] = broker
  284. Logger.Printf("client/brokers Registered new broker #%d at %s", broker.ID(), broker.Addr())
  285. } else if broker.Addr() != client.brokers[broker.ID()].Addr() {
  286. safeAsyncClose(client.brokers[broker.ID()])
  287. client.brokers[broker.ID()] = broker
  288. Logger.Printf("client/brokers Replaced registered broker #%d with %s", broker.ID(), broker.Addr())
  289. }
  290. }
  291. // deregisterBroker removes a broker from the seedsBroker list, and if it's
  292. // not the seedbroker, removes it from brokers map completely.
  293. func (client *client) deregisterBroker(broker *Broker) {
  294. client.lock.Lock()
  295. defer client.lock.Unlock()
  296. if len(client.seedBrokers) > 0 && broker == client.seedBrokers[0] {
  297. client.deadSeeds = append(client.deadSeeds, broker)
  298. client.seedBrokers = client.seedBrokers[1:]
  299. } else {
  300. // we do this so that our loop in `tryRefreshMetadata` doesn't go on forever,
  301. // but we really shouldn't have to; once that loop is made better this case can be
  302. // removed, and the function generally can be renamed from `deregisterBroker` to
  303. // `nextSeedBroker` or something
  304. Logger.Printf("client/brokers Deregistered broker #%d at %s", broker.ID(), broker.Addr())
  305. delete(client.brokers, broker.ID())
  306. }
  307. }
  308. func (client *client) resurrectDeadBrokers() {
  309. client.lock.Lock()
  310. defer client.lock.Unlock()
  311. client.seedBrokers = append(client.seedBrokers, client.deadSeeds...)
  312. client.deadSeeds = nil
  313. }
  314. func (client *client) any() *Broker {
  315. client.lock.RLock()
  316. defer client.lock.RUnlock()
  317. if len(client.seedBrokers) > 0 {
  318. _ = client.seedBrokers[0].Open(client.conf)
  319. return client.seedBrokers[0]
  320. }
  321. // not guaranteed to be random *or* deterministic
  322. for _, broker := range client.brokers {
  323. _ = broker.Open(client.conf)
  324. return broker
  325. }
  326. return nil
  327. }
  328. // private caching/lazy metadata helpers
  329. type partitionType int
  330. const (
  331. allPartitions partitionType = iota
  332. writablePartitions
  333. // If you add any more types, update the partition cache in update()
  334. // Ensure this is the last partition type value
  335. maxPartitionIndex
  336. )
  337. func (client *client) cachedMetadata(topic string, partitionID int32) *PartitionMetadata {
  338. client.lock.RLock()
  339. defer client.lock.RUnlock()
  340. partitions := client.metadata[topic]
  341. if partitions != nil {
  342. return partitions[partitionID]
  343. }
  344. return nil
  345. }
  346. func (client *client) cachedPartitions(topic string, partitionSet partitionType) []int32 {
  347. client.lock.RLock()
  348. defer client.lock.RUnlock()
  349. partitions, exists := client.cachedPartitionsResults[topic]
  350. if !exists {
  351. return nil
  352. }
  353. return partitions[partitionSet]
  354. }
  355. func (client *client) setPartitionCache(topic string, partitionSet partitionType) []int32 {
  356. partitions := client.metadata[topic]
  357. if partitions == nil {
  358. return nil
  359. }
  360. ret := make([]int32, 0, len(partitions))
  361. for _, partition := range partitions {
  362. if partitionSet == writablePartitions && partition.Err == ErrLeaderNotAvailable {
  363. continue
  364. }
  365. ret = append(ret, partition.ID)
  366. }
  367. sort.Sort(int32Slice(ret))
  368. return ret
  369. }
  370. func (client *client) cachedLeader(topic string, partitionID int32) (*Broker, error) {
  371. client.lock.RLock()
  372. defer client.lock.RUnlock()
  373. partitions := client.metadata[topic]
  374. if partitions != nil {
  375. metadata, ok := partitions[partitionID]
  376. if ok {
  377. if metadata.Err == ErrLeaderNotAvailable {
  378. return nil, ErrLeaderNotAvailable
  379. }
  380. b := client.brokers[metadata.Leader]
  381. if b == nil {
  382. return nil, ErrLeaderNotAvailable
  383. }
  384. _ = b.Open(client.conf)
  385. return b, nil
  386. }
  387. }
  388. return nil, ErrUnknownTopicOrPartition
  389. }
  390. func (client *client) getOffset(topic string, partitionID int32, time int64) (int64, error) {
  391. broker, err := client.Leader(topic, partitionID)
  392. if err != nil {
  393. return -1, err
  394. }
  395. request := &OffsetRequest{}
  396. request.AddBlock(topic, partitionID, time, 1)
  397. response, err := broker.GetAvailableOffsets(request)
  398. if err != nil {
  399. _ = broker.Close()
  400. return -1, err
  401. }
  402. block := response.GetBlock(topic, partitionID)
  403. if block == nil {
  404. _ = broker.Close()
  405. return -1, ErrIncompleteResponse
  406. }
  407. if block.Err != ErrNoError {
  408. return -1, block.Err
  409. }
  410. if len(block.Offsets) != 1 {
  411. return -1, ErrOffsetOutOfRange
  412. }
  413. return block.Offsets[0], nil
  414. }
  415. // core metadata update logic
  416. func (client *client) backgroundMetadataUpdater() {
  417. if client.conf.Metadata.RefreshFrequency == time.Duration(0) {
  418. return
  419. }
  420. ticker := time.NewTicker(client.conf.Metadata.RefreshFrequency)
  421. for {
  422. select {
  423. case <-ticker.C:
  424. if err := client.RefreshMetadata(); err != nil {
  425. Logger.Println("Client background metadata update:", err)
  426. }
  427. case <-client.closer:
  428. ticker.Stop()
  429. return
  430. }
  431. }
  432. }
  433. func (client *client) tryRefreshMetadata(topics []string, retriesRemaining int) error {
  434. for broker := client.any(); broker != nil; broker = client.any() {
  435. if len(topics) > 0 {
  436. Logger.Printf("Fetching metadata for %v from broker %s\n", topics, broker.addr)
  437. } else {
  438. Logger.Printf("Fetching metadata for all topics from broker %s\n", broker.addr)
  439. }
  440. response, err := broker.GetMetadata(&MetadataRequest{Topics: topics})
  441. switch err.(type) {
  442. case nil:
  443. // valid response, use it
  444. retry, err := client.updateMetadata(response)
  445. if len(retry) > 0 {
  446. if retriesRemaining <= 0 {
  447. Logger.Println("Some partitions are leaderless, but we're out of retries")
  448. return err
  449. }
  450. Logger.Printf("Some partitions are leaderless, waiting %dms for election... (%d retries remaining)\n",
  451. client.conf.Metadata.Retry.Backoff/time.Millisecond, retriesRemaining)
  452. time.Sleep(client.conf.Metadata.Retry.Backoff) // wait for leader election
  453. return client.tryRefreshMetadata(retry, retriesRemaining-1)
  454. }
  455. return err
  456. case PacketEncodingError:
  457. // didn't even send, return the error
  458. return err
  459. default:
  460. // some other error, remove that broker and try again
  461. Logger.Println("Error from broker while fetching metadata:", err)
  462. _ = broker.Close()
  463. client.deregisterBroker(broker)
  464. }
  465. }
  466. Logger.Println("Out of available brokers.")
  467. if retriesRemaining > 0 {
  468. Logger.Printf("Resurrecting dead brokers after %dms... (%d retries remaining)\n",
  469. client.conf.Metadata.Retry.Backoff/time.Millisecond, retriesRemaining)
  470. time.Sleep(client.conf.Metadata.Retry.Backoff)
  471. client.resurrectDeadBrokers()
  472. return client.tryRefreshMetadata(topics, retriesRemaining-1)
  473. }
  474. return ErrOutOfBrokers
  475. }
  476. // if no fatal error, returns a list of topics that need retrying due to ErrLeaderNotAvailable
  477. func (client *client) updateMetadata(data *MetadataResponse) ([]string, error) {
  478. client.lock.Lock()
  479. defer client.lock.Unlock()
  480. // For all the brokers we received:
  481. // - if it is a new ID, save it
  482. // - if it is an existing ID, but the address we have is stale, discard the old one and save it
  483. // - otherwise ignore it, replacing our existing one would just bounce the connection
  484. for _, broker := range data.Brokers {
  485. client.registerBroker(broker)
  486. }
  487. toRetry := make(map[string]none)
  488. var err error
  489. for _, topic := range data.Topics {
  490. delete(client.metadata, topic.Name)
  491. delete(client.cachedPartitionsResults, topic.Name)
  492. switch topic.Err {
  493. case ErrNoError:
  494. break
  495. case ErrInvalidTopic: // don't retry, don't store partial results
  496. err = topic.Err
  497. continue
  498. case ErrUnknownTopicOrPartition: // retry, do not store partial partition results
  499. err = topic.Err
  500. toRetry[topic.Name] = none{}
  501. continue
  502. case ErrLeaderNotAvailable: // retry, but store partiial partition results
  503. toRetry[topic.Name] = none{}
  504. break
  505. default: // don't retry, don't store partial results
  506. Logger.Printf("Unexpected topic-level metadata error: %s", topic.Err)
  507. err = topic.Err
  508. continue
  509. }
  510. client.metadata[topic.Name] = make(map[int32]*PartitionMetadata, len(topic.Partitions))
  511. for _, partition := range topic.Partitions {
  512. client.metadata[topic.Name][partition.ID] = partition
  513. if partition.Err == ErrLeaderNotAvailable {
  514. toRetry[topic.Name] = none{}
  515. }
  516. }
  517. var partitionCache [maxPartitionIndex][]int32
  518. partitionCache[allPartitions] = client.setPartitionCache(topic.Name, allPartitions)
  519. partitionCache[writablePartitions] = client.setPartitionCache(topic.Name, writablePartitions)
  520. client.cachedPartitionsResults[topic.Name] = partitionCache
  521. }
  522. ret := make([]string, 0, len(toRetry))
  523. for topic := range toRetry {
  524. ret = append(ret, topic)
  525. }
  526. return ret, err
  527. }
  528. func (client *client) cachedCoordinator(consumerGroup string) *Broker {
  529. client.lock.RLock()
  530. defer client.lock.RUnlock()
  531. if coordinatorID, ok := client.coordinators[consumerGroup]; !ok {
  532. return nil
  533. } else {
  534. return client.brokers[coordinatorID]
  535. }
  536. }
  537. func (client *client) getConsumerMetadata(consumerGroup string, retriesRemaining int) (*ConsumerMetadataResponse, error) {
  538. for broker := client.any(); broker != nil; broker = client.any() {
  539. Logger.Printf("client/coordinator Requesting coordinator for consumergoup %s from %s.\n", consumerGroup, broker.Addr())
  540. request := new(ConsumerMetadataRequest)
  541. request.ConsumerGroup = consumerGroup
  542. response, err := broker.GetConsumerMetadata(request)
  543. if err != nil {
  544. Logger.Printf("client/coordinator Request to broker %s failed: %s.\n", broker.Addr(), err)
  545. switch err.(type) {
  546. case PacketEncodingError:
  547. return nil, err
  548. default:
  549. _ = broker.Close()
  550. client.deregisterBroker(broker)
  551. continue
  552. }
  553. }
  554. switch response.Err {
  555. case ErrNoError:
  556. Logger.Printf("client/coordinator Coordinator for consumergoup %s is #%d (%s:%d).\n", consumerGroup, response.CoordinatorID, response.CoordinatorHost, response.CoordinatorPort)
  557. return response, nil
  558. case ErrConsumerCoordinatorNotAvailable:
  559. Logger.Printf("client/coordinator Coordinator for consumer group %s is not available.\n", consumerGroup)
  560. // This is very ugly, but this scenario will only happen once per cluster.
  561. // The __consumer_offsets topic only has to be created one time.
  562. // The number of partitions not configurable, but partition 0 should always exist.
  563. if _, err := client.Leader("__consumer_offsets", 0); err != nil {
  564. Logger.Printf("client/coordinator The __consumer_offsets topic is not initialized completely yet. Waiting 2 seconds...\n")
  565. time.Sleep(2 * time.Second)
  566. }
  567. if retriesRemaining > 0 {
  568. Logger.Printf("Retrying after %dms... (%d retries remaining)\n", client.conf.Metadata.Retry.Backoff/time.Millisecond, retriesRemaining)
  569. time.Sleep(client.conf.Metadata.Retry.Backoff)
  570. return client.getConsumerMetadata(consumerGroup, retriesRemaining-1)
  571. }
  572. return nil, ErrConsumerCoordinatorNotAvailable
  573. default:
  574. return nil, response.Err
  575. }
  576. }
  577. Logger.Println("Out of available brokers to request consumer metadata from.")
  578. if retriesRemaining > 0 {
  579. Logger.Printf("Resurrecting dead brokers after %dms... (%d retries remaining)\n", client.conf.Metadata.Retry.Backoff/time.Millisecond, retriesRemaining)
  580. time.Sleep(client.conf.Metadata.Retry.Backoff)
  581. client.resurrectDeadBrokers()
  582. return client.getConsumerMetadata(consumerGroup, retriesRemaining-1)
  583. }
  584. return nil, ErrOutOfBrokers
  585. }