client.go 22 KB

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