client.go 22 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676677678679680681682683684685686687688689690691692693694695696697698699700701702703704705706707708709710711712713714715716717718719720721722
  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, closed chan none // for shutting down background metadata updater
  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. closed: make(chan none),
  91. brokers: make(map[int32]*Broker),
  92. metadata: make(map[string]map[int32]*PartitionMetadata),
  93. cachedPartitionsResults: make(map[string][maxPartitionIndex][]int32),
  94. coordinators: make(map[string]int32),
  95. }
  96. for _, addr := range addrs {
  97. client.seedBrokers = append(client.seedBrokers, NewBroker(addr))
  98. }
  99. // do an initial fetch of all cluster metadata by specifing an empty list of topics
  100. err := client.RefreshMetadata()
  101. switch err {
  102. case nil:
  103. break
  104. case ErrLeaderNotAvailable, ErrReplicaNotAvailable:
  105. // indicates that maybe part of the cluster is down, but is not fatal to creating the client
  106. Logger.Println(err)
  107. default:
  108. close(client.closed) // we haven't started the background updater yet, so we have to do this manually
  109. _ = client.Close()
  110. return nil, err
  111. }
  112. go withRecover(client.backgroundMetadataUpdater)
  113. Logger.Println("Successfully initialized new client")
  114. return client, nil
  115. }
  116. func (client *client) Config() *Config {
  117. return client.conf
  118. }
  119. func (client *client) Close() error {
  120. if client.Closed() {
  121. // Chances are this is being called from a defer() and the error will go unobserved
  122. // so we go ahead and log the event in this case.
  123. Logger.Printf("Close() called on already closed client")
  124. return ErrClosedClient
  125. }
  126. // shutdown and wait for the background thread before we take the lock, to avoid races
  127. close(client.closer)
  128. <-client.closed
  129. client.lock.Lock()
  130. defer client.lock.Unlock()
  131. Logger.Println("Closing Client")
  132. for _, broker := range client.brokers {
  133. safeAsyncClose(broker)
  134. }
  135. for _, broker := range client.seedBrokers {
  136. safeAsyncClose(broker)
  137. }
  138. client.brokers = nil
  139. client.metadata = nil
  140. return nil
  141. }
  142. func (client *client) Closed() bool {
  143. return client.brokers == nil
  144. }
  145. func (client *client) Topics() ([]string, error) {
  146. if client.Closed() {
  147. return nil, ErrClosedClient
  148. }
  149. client.lock.RLock()
  150. defer client.lock.RUnlock()
  151. ret := make([]string, 0, len(client.metadata))
  152. for topic := range client.metadata {
  153. ret = append(ret, topic)
  154. }
  155. return ret, nil
  156. }
  157. func (client *client) Partitions(topic string) ([]int32, error) {
  158. if client.Closed() {
  159. return nil, ErrClosedClient
  160. }
  161. partitions := client.cachedPartitions(topic, allPartitions)
  162. if len(partitions) == 0 {
  163. err := client.RefreshMetadata(topic)
  164. if err != nil {
  165. return nil, err
  166. }
  167. partitions = client.cachedPartitions(topic, allPartitions)
  168. }
  169. if partitions == nil {
  170. return nil, ErrUnknownTopicOrPartition
  171. }
  172. return partitions, nil
  173. }
  174. func (client *client) WritablePartitions(topic string) ([]int32, error) {
  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. if client.Closed() {
  219. return nil, ErrClosedClient
  220. }
  221. leader, err := client.cachedLeader(topic, partitionID)
  222. if leader == nil {
  223. err := client.RefreshMetadata(topic)
  224. if err != nil {
  225. return nil, err
  226. }
  227. leader, err = client.cachedLeader(topic, partitionID)
  228. }
  229. return leader, err
  230. }
  231. func (client *client) RefreshMetadata(topics ...string) error {
  232. if client.Closed() {
  233. return ErrClosedClient
  234. }
  235. // Prior to 0.8.2, Kafka will throw exceptions on an empty topic and not return a proper
  236. // error. This handles the case by returning an error instead of sending it
  237. // off to Kafka. See: https://github.com/Shopify/sarama/pull/38#issuecomment-26362310
  238. for _, topic := range topics {
  239. if len(topic) == 0 {
  240. return ErrInvalidTopic // this is the error that 0.8.2 and later correctly return
  241. }
  242. }
  243. return client.tryRefreshMetadata(topics, client.conf.Metadata.Retry.Max)
  244. }
  245. func (client *client) GetOffset(topic string, partitionID int32, time int64) (int64, error) {
  246. if client.Closed() {
  247. return -1, ErrClosedClient
  248. }
  249. offset, err := client.getOffset(topic, partitionID, time)
  250. if err != nil {
  251. if err := client.RefreshMetadata(topic); err != nil {
  252. return -1, err
  253. }
  254. return client.getOffset(topic, partitionID, time)
  255. }
  256. return offset, err
  257. }
  258. func (client *client) Coordinator(consumerGroup string) (*Broker, error) {
  259. if client.Closed() {
  260. return nil, ErrClosedClient
  261. }
  262. coordinator := client.cachedCoordinator(consumerGroup)
  263. if coordinator == nil {
  264. if err := client.RefreshCoordinator(consumerGroup); err != nil {
  265. return nil, err
  266. }
  267. coordinator = client.cachedCoordinator(consumerGroup)
  268. }
  269. if coordinator == nil {
  270. return nil, ErrConsumerCoordinatorNotAvailable
  271. }
  272. _ = coordinator.Open(client.conf)
  273. return coordinator, nil
  274. }
  275. func (client *client) RefreshCoordinator(consumerGroup string) error {
  276. if client.Closed() {
  277. return ErrClosedClient
  278. }
  279. response, err := client.getConsumerMetadata(consumerGroup, client.conf.Metadata.Retry.Max)
  280. if err != nil {
  281. return err
  282. }
  283. client.lock.Lock()
  284. defer client.lock.Unlock()
  285. client.registerBroker(response.Coordinator)
  286. client.coordinators[consumerGroup] = response.Coordinator.ID()
  287. return nil
  288. }
  289. // private broker management helpers
  290. // registerBroker makes sure a broker received by a Metadata or Coordinator request is registered
  291. // in the brokers map. It returns the broker that is registered, which may be the provided broker,
  292. // or a previously registered Broker instance. You must hold the write lock before calling this function.
  293. func (client *client) registerBroker(broker *Broker) {
  294. if client.brokers[broker.ID()] == nil {
  295. client.brokers[broker.ID()] = broker
  296. Logger.Printf("client/brokers registered new broker #%d at %s", broker.ID(), broker.Addr())
  297. } else if broker.Addr() != client.brokers[broker.ID()].Addr() {
  298. safeAsyncClose(client.brokers[broker.ID()])
  299. client.brokers[broker.ID()] = broker
  300. Logger.Printf("client/brokers replaced registered broker #%d with %s", broker.ID(), broker.Addr())
  301. }
  302. }
  303. // deregisterBroker removes a broker from the seedsBroker list, and if it's
  304. // not the seedbroker, removes it from brokers map completely.
  305. func (client *client) deregisterBroker(broker *Broker) {
  306. client.lock.Lock()
  307. defer client.lock.Unlock()
  308. if len(client.seedBrokers) > 0 && broker == client.seedBrokers[0] {
  309. client.deadSeeds = append(client.deadSeeds, broker)
  310. client.seedBrokers = client.seedBrokers[1:]
  311. } else {
  312. // we do this so that our loop in `tryRefreshMetadata` doesn't go on forever,
  313. // but we really shouldn't have to; once that loop is made better this case can be
  314. // removed, and the function generally can be renamed from `deregisterBroker` to
  315. // `nextSeedBroker` or something
  316. Logger.Printf("client/brokers deregistered broker #%d at %s", broker.ID(), broker.Addr())
  317. delete(client.brokers, broker.ID())
  318. }
  319. }
  320. func (client *client) resurrectDeadBrokers() {
  321. client.lock.Lock()
  322. defer client.lock.Unlock()
  323. Logger.Printf("client/brokers resurrecting %d dead seed brokers", len(client.deadSeeds))
  324. client.seedBrokers = append(client.seedBrokers, client.deadSeeds...)
  325. client.deadSeeds = nil
  326. }
  327. func (client *client) any() *Broker {
  328. client.lock.RLock()
  329. defer client.lock.RUnlock()
  330. if len(client.seedBrokers) > 0 {
  331. _ = client.seedBrokers[0].Open(client.conf)
  332. return client.seedBrokers[0]
  333. }
  334. // not guaranteed to be random *or* deterministic
  335. for _, broker := range client.brokers {
  336. _ = broker.Open(client.conf)
  337. return broker
  338. }
  339. return nil
  340. }
  341. // private caching/lazy metadata helpers
  342. type partitionType int
  343. const (
  344. allPartitions partitionType = iota
  345. writablePartitions
  346. // If you add any more types, update the partition cache in update()
  347. // Ensure this is the last partition type value
  348. maxPartitionIndex
  349. )
  350. func (client *client) cachedMetadata(topic string, partitionID int32) *PartitionMetadata {
  351. client.lock.RLock()
  352. defer client.lock.RUnlock()
  353. partitions := client.metadata[topic]
  354. if partitions != nil {
  355. return partitions[partitionID]
  356. }
  357. return nil
  358. }
  359. func (client *client) cachedPartitions(topic string, partitionSet partitionType) []int32 {
  360. client.lock.RLock()
  361. defer client.lock.RUnlock()
  362. partitions, exists := client.cachedPartitionsResults[topic]
  363. if !exists {
  364. return nil
  365. }
  366. return partitions[partitionSet]
  367. }
  368. func (client *client) setPartitionCache(topic string, partitionSet partitionType) []int32 {
  369. partitions := client.metadata[topic]
  370. if partitions == nil {
  371. return nil
  372. }
  373. ret := make([]int32, 0, len(partitions))
  374. for _, partition := range partitions {
  375. if partitionSet == writablePartitions && partition.Err == ErrLeaderNotAvailable {
  376. continue
  377. }
  378. ret = append(ret, partition.ID)
  379. }
  380. sort.Sort(int32Slice(ret))
  381. return ret
  382. }
  383. func (client *client) cachedLeader(topic string, partitionID int32) (*Broker, error) {
  384. client.lock.RLock()
  385. defer client.lock.RUnlock()
  386. partitions := client.metadata[topic]
  387. if partitions != nil {
  388. metadata, ok := partitions[partitionID]
  389. if ok {
  390. if metadata.Err == ErrLeaderNotAvailable {
  391. return nil, ErrLeaderNotAvailable
  392. }
  393. b := client.brokers[metadata.Leader]
  394. if b == nil {
  395. return nil, ErrLeaderNotAvailable
  396. }
  397. _ = b.Open(client.conf)
  398. return b, nil
  399. }
  400. }
  401. return nil, ErrUnknownTopicOrPartition
  402. }
  403. func (client *client) getOffset(topic string, partitionID int32, time int64) (int64, error) {
  404. broker, err := client.Leader(topic, partitionID)
  405. if err != nil {
  406. return -1, err
  407. }
  408. request := &OffsetRequest{}
  409. request.AddBlock(topic, partitionID, time, 1)
  410. response, err := broker.GetAvailableOffsets(request)
  411. if err != nil {
  412. _ = broker.Close()
  413. return -1, err
  414. }
  415. block := response.GetBlock(topic, partitionID)
  416. if block == nil {
  417. _ = broker.Close()
  418. return -1, ErrIncompleteResponse
  419. }
  420. if block.Err != ErrNoError {
  421. return -1, block.Err
  422. }
  423. if len(block.Offsets) != 1 {
  424. return -1, ErrOffsetOutOfRange
  425. }
  426. return block.Offsets[0], nil
  427. }
  428. // core metadata update logic
  429. func (client *client) backgroundMetadataUpdater() {
  430. defer close(client.closed)
  431. if client.conf.Metadata.RefreshFrequency == time.Duration(0) {
  432. return
  433. }
  434. ticker := time.NewTicker(client.conf.Metadata.RefreshFrequency)
  435. defer ticker.Stop()
  436. for {
  437. select {
  438. case <-ticker.C:
  439. if err := client.RefreshMetadata(); err != nil {
  440. Logger.Println("Client background metadata update:", err)
  441. }
  442. case <-client.closer:
  443. return
  444. }
  445. }
  446. }
  447. func (client *client) tryRefreshMetadata(topics []string, attemptsRemaining int) error {
  448. retry := func(err error) error {
  449. if attemptsRemaining > 0 {
  450. Logger.Printf("client/metadata retrying after %dms... (%d attempts remaining)\n", client.conf.Metadata.Retry.Backoff/time.Millisecond, attemptsRemaining)
  451. time.Sleep(client.conf.Metadata.Retry.Backoff)
  452. return client.tryRefreshMetadata(topics, attemptsRemaining-1)
  453. }
  454. return err
  455. }
  456. for broker := client.any(); broker != nil; broker = client.any() {
  457. if len(topics) > 0 {
  458. Logger.Printf("client/metadata fetching metadata for %v from broker %s\n", topics, broker.addr)
  459. } else {
  460. Logger.Printf("client/metadata fetching metadata for all topics from broker %s\n", broker.addr)
  461. }
  462. response, err := broker.GetMetadata(&MetadataRequest{Topics: topics})
  463. switch err.(type) {
  464. case nil:
  465. // valid response, use it
  466. if shouldRetry, err := client.updateMetadata(response); shouldRetry {
  467. Logger.Println("client/metadata found some partitions to be leaderless")
  468. return retry(err) // note: err can be nil
  469. } else {
  470. return err
  471. }
  472. case PacketEncodingError:
  473. // didn't even send, return the error
  474. return err
  475. default:
  476. // some other error, remove that broker and try again
  477. Logger.Println("client/metadata got error from broker while fetching metadata:", err)
  478. _ = broker.Close()
  479. client.deregisterBroker(broker)
  480. }
  481. }
  482. Logger.Println("client/metadata no available broker to send metadata request to")
  483. client.resurrectDeadBrokers()
  484. return retry(ErrOutOfBrokers)
  485. }
  486. // if no fatal error, returns a list of topics that need retrying due to ErrLeaderNotAvailable
  487. func (client *client) updateMetadata(data *MetadataResponse) (retry bool, err error) {
  488. client.lock.Lock()
  489. defer client.lock.Unlock()
  490. // For all the brokers we received:
  491. // - if it is a new ID, save it
  492. // - if it is an existing ID, but the address we have is stale, discard the old one and save it
  493. // - otherwise ignore it, replacing our existing one would just bounce the connection
  494. for _, broker := range data.Brokers {
  495. client.registerBroker(broker)
  496. }
  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. retry = true
  509. continue
  510. case ErrLeaderNotAvailable: // retry, but store partial partition results
  511. retry = true
  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. retry = true
  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. return
  531. }
  532. func (client *client) cachedCoordinator(consumerGroup string) *Broker {
  533. client.lock.RLock()
  534. defer client.lock.RUnlock()
  535. if coordinatorID, ok := client.coordinators[consumerGroup]; !ok {
  536. return nil
  537. } else {
  538. return client.brokers[coordinatorID]
  539. }
  540. }
  541. func (client *client) getConsumerMetadata(consumerGroup string, attemptsRemaining int) (*ConsumerMetadataResponse, error) {
  542. retry := func(err error) (*ConsumerMetadataResponse, error) {
  543. if attemptsRemaining > 0 {
  544. Logger.Printf("client/coordinator retrying after %dms... (%d attempts remaining)\n", client.conf.Metadata.Retry.Backoff/time.Millisecond, attemptsRemaining)
  545. time.Sleep(client.conf.Metadata.Retry.Backoff)
  546. return client.getConsumerMetadata(consumerGroup, attemptsRemaining-1)
  547. }
  548. return nil, err
  549. }
  550. for broker := client.any(); broker != nil; broker = client.any() {
  551. Logger.Printf("client/coordinator requesting coordinator for consumergoup %s from %s\n", consumerGroup, broker.Addr())
  552. request := new(ConsumerMetadataRequest)
  553. request.ConsumerGroup = consumerGroup
  554. response, err := broker.GetConsumerMetadata(request)
  555. if err != nil {
  556. Logger.Printf("client/coordinator request to broker %s failed: %s\n", broker.Addr(), err)
  557. switch err.(type) {
  558. case PacketEncodingError:
  559. return nil, err
  560. default:
  561. _ = broker.Close()
  562. client.deregisterBroker(broker)
  563. continue
  564. }
  565. }
  566. switch response.Err {
  567. case ErrNoError:
  568. Logger.Printf("client/coordinator coordinator for consumergoup %s is #%d (%s)\n", consumerGroup, response.Coordinator.ID(), response.Coordinator.Addr())
  569. return response, nil
  570. case ErrConsumerCoordinatorNotAvailable:
  571. Logger.Printf("client/coordinator coordinator for consumer group %s is not available\n", consumerGroup)
  572. // This is very ugly, but this scenario will only happen once per cluster.
  573. // The __consumer_offsets topic only has to be created one time.
  574. // The number of partitions not configurable, but partition 0 should always exist.
  575. if _, err := client.Leader("__consumer_offsets", 0); err != nil {
  576. Logger.Printf("client/coordinator the __consumer_offsets topic is not initialized completely yet. Waiting 2 seconds...\n")
  577. time.Sleep(2 * time.Second)
  578. }
  579. return retry(ErrConsumerCoordinatorNotAvailable)
  580. default:
  581. return nil, response.Err
  582. }
  583. }
  584. Logger.Println("client/coordinator no available broker to send consumer metadata request to")
  585. client.resurrectDeadBrokers()
  586. return retry(ErrOutOfBrokers)
  587. }