client.go 22 KB

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