client.go 22 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676677678679680681682683684685686687688689690691692693694695696697698699700701702703704705706707708709710711712713714715716717718719720721722723724725726727728729730731732
  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. if client.Closed() {
  119. // Chances are this is being called from a defer() and the error will go unobserved
  120. // so we go ahead and log the event in this case.
  121. Logger.Printf("Close() called on already closed client")
  122. return ErrClosedClient
  123. }
  124. client.lock.Lock()
  125. defer client.lock.Unlock()
  126. Logger.Println("Closing Client")
  127. for _, broker := range client.brokers {
  128. safeAsyncClose(broker)
  129. }
  130. for _, broker := range client.seedBrokers {
  131. safeAsyncClose(broker)
  132. }
  133. client.brokers = nil
  134. client.metadata = nil
  135. close(client.closer)
  136. return nil
  137. }
  138. func (client *client) Closed() bool {
  139. return client.brokers == nil
  140. }
  141. func (client *client) Topics() ([]string, error) {
  142. if client.Closed() {
  143. return nil, ErrClosedClient
  144. }
  145. client.lock.RLock()
  146. defer client.lock.RUnlock()
  147. ret := make([]string, 0, len(client.metadata))
  148. for topic := range client.metadata {
  149. ret = append(ret, topic)
  150. }
  151. return ret, nil
  152. }
  153. func (client *client) Partitions(topic string) ([]int32, error) {
  154. if client.Closed() {
  155. return nil, ErrClosedClient
  156. }
  157. partitions := client.cachedPartitions(topic, allPartitions)
  158. if len(partitions) == 0 {
  159. err := client.RefreshMetadata(topic)
  160. if err != nil {
  161. return nil, err
  162. }
  163. partitions = client.cachedPartitions(topic, allPartitions)
  164. }
  165. if partitions == nil {
  166. return nil, ErrUnknownTopicOrPartition
  167. }
  168. return partitions, nil
  169. }
  170. func (client *client) WritablePartitions(topic string) ([]int32, error) {
  171. if client.Closed() {
  172. return nil, ErrClosedClient
  173. }
  174. partitions := client.cachedPartitions(topic, writablePartitions)
  175. // len==0 catches when it's nil (no such topic) and the odd case when every single
  176. // partition is undergoing leader election simultaneously. Callers have to be able to handle
  177. // this function returning an empty slice (which is a valid return value) but catching it
  178. // here the first time (note we *don't* catch it below where we return ErrUnknownTopicOrPartition) triggers
  179. // a metadata refresh as a nicety so callers can just try again and don't have to manually
  180. // trigger a refresh (otherwise they'd just keep getting a stale cached copy).
  181. if len(partitions) == 0 {
  182. err := client.RefreshMetadata(topic)
  183. if err != nil {
  184. return nil, err
  185. }
  186. partitions = client.cachedPartitions(topic, writablePartitions)
  187. }
  188. if partitions == nil {
  189. return nil, ErrUnknownTopicOrPartition
  190. }
  191. return partitions, nil
  192. }
  193. func (client *client) Replicas(topic string, partitionID int32) ([]int32, error) {
  194. if client.Closed() {
  195. return nil, ErrClosedClient
  196. }
  197. metadata := client.cachedMetadata(topic, partitionID)
  198. if metadata == nil {
  199. err := client.RefreshMetadata(topic)
  200. if err != nil {
  201. return nil, err
  202. }
  203. metadata = client.cachedMetadata(topic, partitionID)
  204. }
  205. if metadata == nil {
  206. return nil, ErrUnknownTopicOrPartition
  207. }
  208. if metadata.Err == ErrReplicaNotAvailable {
  209. return nil, metadata.Err
  210. }
  211. return dupeAndSort(metadata.Replicas), nil
  212. }
  213. func (client *client) Leader(topic string, partitionID int32) (*Broker, error) {
  214. if client.Closed() {
  215. return nil, ErrClosedClient
  216. }
  217. leader, err := client.cachedLeader(topic, partitionID)
  218. if leader == nil {
  219. err := client.RefreshMetadata(topic)
  220. if err != nil {
  221. return nil, err
  222. }
  223. leader, err = client.cachedLeader(topic, partitionID)
  224. }
  225. return leader, err
  226. }
  227. func (client *client) RefreshMetadata(topics ...string) error {
  228. if client.Closed() {
  229. return ErrClosedClient
  230. }
  231. // Prior to 0.8.2, Kafka will throw exceptions on an empty topic and not return a proper
  232. // error. This handles the case by returning an error instead of sending it
  233. // off to Kafka. See: https://github.com/Shopify/sarama/pull/38#issuecomment-26362310
  234. for _, topic := range topics {
  235. if len(topic) == 0 {
  236. return ErrInvalidTopic // this is the error that 0.8.2 and later correctly return
  237. }
  238. }
  239. return client.tryRefreshMetadata(topics, client.conf.Metadata.Retry.Max)
  240. }
  241. func (client *client) GetOffset(topic string, partitionID int32, time int64) (int64, error) {
  242. if client.Closed() {
  243. return -1, ErrClosedClient
  244. }
  245. offset, err := client.getOffset(topic, partitionID, time)
  246. if err != nil {
  247. if err := client.RefreshMetadata(topic); err != nil {
  248. return -1, err
  249. }
  250. return client.getOffset(topic, partitionID, time)
  251. }
  252. return offset, err
  253. }
  254. func (client *client) Coordinator(consumerGroup string) (*Broker, error) {
  255. if client.Closed() {
  256. return nil, ErrClosedClient
  257. }
  258. coordinator := client.cachedCoordinator(consumerGroup)
  259. if coordinator == nil {
  260. if err := client.RefreshCoordinator(consumerGroup); err != nil {
  261. return nil, err
  262. }
  263. coordinator = client.cachedCoordinator(consumerGroup)
  264. }
  265. if coordinator == nil {
  266. return nil, ErrConsumerCoordinatorNotAvailable
  267. }
  268. _ = coordinator.Open(client.conf)
  269. return coordinator, nil
  270. }
  271. func (client *client) RefreshCoordinator(consumerGroup string) error {
  272. if client.Closed() {
  273. return ErrClosedClient
  274. }
  275. response, err := client.getConsumerMetadata(consumerGroup, client.conf.Metadata.Retry.Max)
  276. if err != nil {
  277. return err
  278. }
  279. client.lock.Lock()
  280. defer client.lock.Unlock()
  281. client.registerBroker(response.Coordinator)
  282. client.coordinators[consumerGroup] = response.Coordinator.ID()
  283. return nil
  284. }
  285. // private broker management helpers
  286. // registerBroker makes sure a broker received by a Metadata or Coordinator request is registered
  287. // in the brokers map. It returns the broker that is registered, which may be the provided broker,
  288. // or a previously registered Broker instance. You must hold the write lock before calling this function.
  289. func (client *client) registerBroker(broker *Broker) {
  290. if client.brokers[broker.ID()] == nil {
  291. client.brokers[broker.ID()] = broker
  292. Logger.Printf("client/brokers Registered new broker #%d at %s", broker.ID(), broker.Addr())
  293. } else if broker.Addr() != client.brokers[broker.ID()].Addr() {
  294. safeAsyncClose(client.brokers[broker.ID()])
  295. client.brokers[broker.ID()] = broker
  296. Logger.Printf("client/brokers Replaced registered broker #%d with %s", broker.ID(), broker.Addr())
  297. }
  298. }
  299. // deregisterBroker removes a broker from the seedsBroker list, and if it's
  300. // not the seedbroker, removes it from brokers map completely.
  301. func (client *client) deregisterBroker(broker *Broker) {
  302. client.lock.Lock()
  303. defer client.lock.Unlock()
  304. if len(client.seedBrokers) > 0 && broker == client.seedBrokers[0] {
  305. client.deadSeeds = append(client.deadSeeds, broker)
  306. client.seedBrokers = client.seedBrokers[1:]
  307. } else {
  308. // we do this so that our loop in `tryRefreshMetadata` doesn't go on forever,
  309. // but we really shouldn't have to; once that loop is made better this case can be
  310. // removed, and the function generally can be renamed from `deregisterBroker` to
  311. // `nextSeedBroker` or something
  312. Logger.Printf("client/brokers Deregistered broker #%d at %s", broker.ID(), broker.Addr())
  313. delete(client.brokers, broker.ID())
  314. }
  315. }
  316. func (client *client) resurrectDeadBrokers() {
  317. client.lock.Lock()
  318. defer client.lock.Unlock()
  319. client.seedBrokers = append(client.seedBrokers, client.deadSeeds...)
  320. client.deadSeeds = nil
  321. }
  322. func (client *client) any() *Broker {
  323. client.lock.RLock()
  324. defer client.lock.RUnlock()
  325. if len(client.seedBrokers) > 0 {
  326. _ = client.seedBrokers[0].Open(client.conf)
  327. return client.seedBrokers[0]
  328. }
  329. // not guaranteed to be random *or* deterministic
  330. for _, broker := range client.brokers {
  331. _ = broker.Open(client.conf)
  332. return broker
  333. }
  334. return nil
  335. }
  336. // private caching/lazy metadata helpers
  337. type partitionType int
  338. const (
  339. allPartitions partitionType = iota
  340. writablePartitions
  341. // If you add any more types, update the partition cache in update()
  342. // Ensure this is the last partition type value
  343. maxPartitionIndex
  344. )
  345. func (client *client) cachedMetadata(topic string, partitionID int32) *PartitionMetadata {
  346. client.lock.RLock()
  347. defer client.lock.RUnlock()
  348. partitions := client.metadata[topic]
  349. if partitions != nil {
  350. return partitions[partitionID]
  351. }
  352. return nil
  353. }
  354. func (client *client) cachedPartitions(topic string, partitionSet partitionType) []int32 {
  355. client.lock.RLock()
  356. defer client.lock.RUnlock()
  357. partitions, exists := client.cachedPartitionsResults[topic]
  358. if !exists {
  359. return nil
  360. }
  361. return partitions[partitionSet]
  362. }
  363. func (client *client) setPartitionCache(topic string, partitionSet partitionType) []int32 {
  364. partitions := client.metadata[topic]
  365. if partitions == nil {
  366. return nil
  367. }
  368. ret := make([]int32, 0, len(partitions))
  369. for _, partition := range partitions {
  370. if partitionSet == writablePartitions && partition.Err == ErrLeaderNotAvailable {
  371. continue
  372. }
  373. ret = append(ret, partition.ID)
  374. }
  375. sort.Sort(int32Slice(ret))
  376. return ret
  377. }
  378. func (client *client) cachedLeader(topic string, partitionID int32) (*Broker, error) {
  379. client.lock.RLock()
  380. defer client.lock.RUnlock()
  381. partitions := client.metadata[topic]
  382. if partitions != nil {
  383. metadata, ok := partitions[partitionID]
  384. if ok {
  385. if metadata.Err == ErrLeaderNotAvailable {
  386. return nil, ErrLeaderNotAvailable
  387. }
  388. b := client.brokers[metadata.Leader]
  389. if b == nil {
  390. return nil, ErrLeaderNotAvailable
  391. }
  392. _ = b.Open(client.conf)
  393. return b, nil
  394. }
  395. }
  396. return nil, ErrUnknownTopicOrPartition
  397. }
  398. func (client *client) getOffset(topic string, partitionID int32, time int64) (int64, error) {
  399. broker, err := client.Leader(topic, partitionID)
  400. if err != nil {
  401. return -1, err
  402. }
  403. request := &OffsetRequest{}
  404. request.AddBlock(topic, partitionID, time, 1)
  405. response, err := broker.GetAvailableOffsets(request)
  406. if err != nil {
  407. _ = broker.Close()
  408. return -1, err
  409. }
  410. block := response.GetBlock(topic, partitionID)
  411. if block == nil {
  412. _ = broker.Close()
  413. return -1, ErrIncompleteResponse
  414. }
  415. if block.Err != ErrNoError {
  416. return -1, block.Err
  417. }
  418. if len(block.Offsets) != 1 {
  419. return -1, ErrOffsetOutOfRange
  420. }
  421. return block.Offsets[0], nil
  422. }
  423. // core metadata update logic
  424. func (client *client) backgroundMetadataUpdater() {
  425. if client.conf.Metadata.RefreshFrequency == time.Duration(0) {
  426. return
  427. }
  428. ticker := time.NewTicker(client.conf.Metadata.RefreshFrequency)
  429. for {
  430. select {
  431. case <-ticker.C:
  432. if err := client.RefreshMetadata(); err != nil {
  433. Logger.Println("Client background metadata update:", err)
  434. }
  435. case <-client.closer:
  436. ticker.Stop()
  437. return
  438. }
  439. }
  440. }
  441. func (client *client) tryRefreshMetadata(topics []string, retriesRemaining int) error {
  442. for broker := client.any(); broker != nil; broker = client.any() {
  443. if len(topics) > 0 {
  444. Logger.Printf("Fetching metadata for %v from broker %s\n", topics, broker.addr)
  445. } else {
  446. Logger.Printf("Fetching metadata for all topics from broker %s\n", broker.addr)
  447. }
  448. response, err := broker.GetMetadata(&MetadataRequest{Topics: topics})
  449. switch err.(type) {
  450. case nil:
  451. // valid response, use it
  452. retry, err := client.updateMetadata(response)
  453. if len(retry) > 0 {
  454. if retriesRemaining <= 0 {
  455. Logger.Println("Some partitions are leaderless, but we're out of retries")
  456. return err
  457. }
  458. Logger.Printf("Some partitions are leaderless, waiting %dms for election... (%d retries remaining)\n",
  459. client.conf.Metadata.Retry.Backoff/time.Millisecond, retriesRemaining)
  460. time.Sleep(client.conf.Metadata.Retry.Backoff) // wait for leader election
  461. return client.tryRefreshMetadata(retry, retriesRemaining-1)
  462. }
  463. return err
  464. case PacketEncodingError:
  465. // didn't even send, return the error
  466. return err
  467. default:
  468. // some other error, remove that broker and try again
  469. Logger.Println("Error from broker while fetching metadata:", err)
  470. _ = broker.Close()
  471. client.deregisterBroker(broker)
  472. }
  473. }
  474. Logger.Println("Out of available brokers.")
  475. if retriesRemaining > 0 {
  476. Logger.Printf("Resurrecting dead brokers after %dms... (%d retries remaining)\n",
  477. client.conf.Metadata.Retry.Backoff/time.Millisecond, retriesRemaining)
  478. time.Sleep(client.conf.Metadata.Retry.Backoff)
  479. client.resurrectDeadBrokers()
  480. return client.tryRefreshMetadata(topics, retriesRemaining-1)
  481. }
  482. return ErrOutOfBrokers
  483. }
  484. // if no fatal error, returns a list of topics that need retrying due to ErrLeaderNotAvailable
  485. func (client *client) updateMetadata(data *MetadataResponse) ([]string, error) {
  486. client.lock.Lock()
  487. defer client.lock.Unlock()
  488. // For all the brokers we received:
  489. // - if it is a new ID, save it
  490. // - if it is an existing ID, but the address we have is stale, discard the old one and save it
  491. // - otherwise ignore it, replacing our existing one would just bounce the connection
  492. for _, broker := range data.Brokers {
  493. client.registerBroker(broker)
  494. }
  495. toRetry := make(map[string]none)
  496. var err error
  497. for _, topic := range data.Topics {
  498. delete(client.metadata, topic.Name)
  499. delete(client.cachedPartitionsResults, topic.Name)
  500. switch topic.Err {
  501. case ErrNoError:
  502. break
  503. case ErrInvalidTopic: // don't retry, don't store partial results
  504. err = topic.Err
  505. continue
  506. case ErrUnknownTopicOrPartition: // retry, do not store partial partition results
  507. err = topic.Err
  508. toRetry[topic.Name] = none{}
  509. continue
  510. case ErrLeaderNotAvailable: // retry, but store partiial partition results
  511. toRetry[topic.Name] = none{}
  512. break
  513. default: // don't retry, don't store partial results
  514. Logger.Printf("Unexpected topic-level metadata error: %s", topic.Err)
  515. err = topic.Err
  516. continue
  517. }
  518. client.metadata[topic.Name] = make(map[int32]*PartitionMetadata, len(topic.Partitions))
  519. for _, partition := range topic.Partitions {
  520. client.metadata[topic.Name][partition.ID] = partition
  521. if partition.Err == ErrLeaderNotAvailable {
  522. toRetry[topic.Name] = none{}
  523. }
  524. }
  525. var partitionCache [maxPartitionIndex][]int32
  526. partitionCache[allPartitions] = client.setPartitionCache(topic.Name, allPartitions)
  527. partitionCache[writablePartitions] = client.setPartitionCache(topic.Name, writablePartitions)
  528. client.cachedPartitionsResults[topic.Name] = partitionCache
  529. }
  530. ret := make([]string, 0, len(toRetry))
  531. for topic := range toRetry {
  532. ret = append(ret, topic)
  533. }
  534. return ret, err
  535. }
  536. func (client *client) cachedCoordinator(consumerGroup string) *Broker {
  537. client.lock.RLock()
  538. defer client.lock.RUnlock()
  539. if coordinatorID, ok := client.coordinators[consumerGroup]; !ok {
  540. return nil
  541. } else {
  542. return client.brokers[coordinatorID]
  543. }
  544. }
  545. func (client *client) getConsumerMetadata(consumerGroup string, retriesRemaining int) (*ConsumerMetadataResponse, error) {
  546. for broker := client.any(); broker != nil; broker = client.any() {
  547. Logger.Printf("client/coordinator Requesting coordinator for consumergoup %s from %s.\n", consumerGroup, broker.Addr())
  548. request := new(ConsumerMetadataRequest)
  549. request.ConsumerGroup = consumerGroup
  550. response, err := broker.GetConsumerMetadata(request)
  551. if err != nil {
  552. Logger.Printf("client/coordinator Request to broker %s failed: %s.\n", broker.Addr(), err)
  553. switch err.(type) {
  554. case PacketEncodingError:
  555. return nil, err
  556. default:
  557. _ = broker.Close()
  558. client.deregisterBroker(broker)
  559. continue
  560. }
  561. }
  562. switch response.Err {
  563. case ErrNoError:
  564. Logger.Printf("client/coordinator Coordinator for consumergoup %s is #%d (%s:%d).\n", consumerGroup, response.CoordinatorID, response.CoordinatorHost, response.CoordinatorPort)
  565. return response, nil
  566. case ErrConsumerCoordinatorNotAvailable:
  567. Logger.Printf("client/coordinator Coordinator for consumer group %s is not available.\n", consumerGroup)
  568. // This is very ugly, but this scenario will only happen once per cluster.
  569. // The __consumer_offsets topic only has to be created one time.
  570. // The number of partitions not configurable, but partition 0 should always exist.
  571. if _, err := client.Leader("__consumer_offsets", 0); err != nil {
  572. Logger.Printf("client/coordinator The __consumer_offsets topic is not initialized completely yet. Waiting 2 seconds...\n")
  573. time.Sleep(2 * time.Second)
  574. }
  575. if retriesRemaining > 0 {
  576. Logger.Printf("Retrying after %dms... (%d retries remaining)\n", client.conf.Metadata.Retry.Backoff/time.Millisecond, retriesRemaining)
  577. time.Sleep(client.conf.Metadata.Retry.Backoff)
  578. return client.getConsumerMetadata(consumerGroup, retriesRemaining-1)
  579. }
  580. return nil, ErrConsumerCoordinatorNotAvailable
  581. default:
  582. return nil, response.Err
  583. }
  584. }
  585. Logger.Println("Out of available brokers to request consumer metadata from.")
  586. if retriesRemaining > 0 {
  587. Logger.Printf("Resurrecting dead brokers after %dms... (%d retries remaining)\n", client.conf.Metadata.Retry.Backoff/time.Millisecond, retriesRemaining)
  588. time.Sleep(client.conf.Metadata.Retry.Backoff)
  589. client.resurrectDeadBrokers()
  590. return client.getConsumerMetadata(consumerGroup, retriesRemaining-1)
  591. }
  592. return nil, ErrOutOfBrokers
  593. }