client.go 23 KB

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