client.go 22 KB

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