client.go 23 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676677678679680681682683684685686687688689690691692693694695696697698699700701702703704705706707708709710711712713714715716717718719720721722723724725726727728729730731732733734735736737738739740741742743744745746747748749750751752753754755756757758759760761762763764765766767768769770771772773774
  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 specifying 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. client.lock.RLock()
  134. defer client.lock.RUnlock()
  135. brokers := make([]*Broker, 0)
  136. for _, broker := range client.brokers {
  137. brokers = append(brokers, broker)
  138. }
  139. return brokers
  140. }
  141. func (client *client) Close() error {
  142. if client.Closed() {
  143. // Chances are this is being called from a defer() and the error will go unobserved
  144. // so we go ahead and log the event in this case.
  145. Logger.Printf("Close() called on already closed client")
  146. return ErrClosedClient
  147. }
  148. // shutdown and wait for the background thread before we take the lock, to avoid races
  149. close(client.closer)
  150. <-client.closed
  151. client.lock.Lock()
  152. defer client.lock.Unlock()
  153. Logger.Println("Closing Client")
  154. for _, broker := range client.brokers {
  155. safeAsyncClose(broker)
  156. }
  157. for _, broker := range client.seedBrokers {
  158. safeAsyncClose(broker)
  159. }
  160. client.brokers = nil
  161. client.metadata = nil
  162. return nil
  163. }
  164. func (client *client) Closed() bool {
  165. return client.brokers == nil
  166. }
  167. func (client *client) Topics() ([]string, error) {
  168. if client.Closed() {
  169. return nil, ErrClosedClient
  170. }
  171. client.lock.RLock()
  172. defer client.lock.RUnlock()
  173. ret := make([]string, 0, len(client.metadata))
  174. for topic := range client.metadata {
  175. ret = append(ret, topic)
  176. }
  177. return ret, nil
  178. }
  179. func (client *client) Partitions(topic string) ([]int32, error) {
  180. if client.Closed() {
  181. return nil, ErrClosedClient
  182. }
  183. partitions := client.cachedPartitions(topic, allPartitions)
  184. if len(partitions) == 0 {
  185. err := client.RefreshMetadata(topic)
  186. if err != nil {
  187. return nil, err
  188. }
  189. partitions = client.cachedPartitions(topic, allPartitions)
  190. }
  191. if partitions == nil {
  192. return nil, ErrUnknownTopicOrPartition
  193. }
  194. return partitions, nil
  195. }
  196. func (client *client) WritablePartitions(topic string) ([]int32, error) {
  197. if client.Closed() {
  198. return nil, ErrClosedClient
  199. }
  200. partitions := client.cachedPartitions(topic, writablePartitions)
  201. // len==0 catches when it's nil (no such topic) and the odd case when every single
  202. // partition is undergoing leader election simultaneously. Callers have to be able to handle
  203. // this function returning an empty slice (which is a valid return value) but catching it
  204. // here the first time (note we *don't* catch it below where we return ErrUnknownTopicOrPartition) triggers
  205. // a metadata refresh as a nicety so callers can just try again and don't have to manually
  206. // trigger a refresh (otherwise they'd just keep getting a stale cached copy).
  207. if len(partitions) == 0 {
  208. err := client.RefreshMetadata(topic)
  209. if err != nil {
  210. return nil, err
  211. }
  212. partitions = client.cachedPartitions(topic, writablePartitions)
  213. }
  214. if partitions == nil {
  215. return nil, ErrUnknownTopicOrPartition
  216. }
  217. return partitions, nil
  218. }
  219. func (client *client) Replicas(topic string, partitionID int32) ([]int32, error) {
  220. if client.Closed() {
  221. return nil, ErrClosedClient
  222. }
  223. metadata := client.cachedMetadata(topic, partitionID)
  224. if metadata == nil {
  225. err := client.RefreshMetadata(topic)
  226. if err != nil {
  227. return nil, err
  228. }
  229. metadata = client.cachedMetadata(topic, partitionID)
  230. }
  231. if metadata == nil {
  232. return nil, ErrUnknownTopicOrPartition
  233. }
  234. if metadata.Err == ErrReplicaNotAvailable {
  235. return nil, metadata.Err
  236. }
  237. return dupeAndSort(metadata.Replicas), nil
  238. }
  239. func (client *client) Isr(topic string, partitionID int32) ([]int32, error) {
  240. if client.Closed() {
  241. return nil, ErrClosedClient
  242. }
  243. metadata := client.cachedMetadata(topic, partitionID)
  244. if metadata == nil {
  245. err := client.RefreshMetadata(topic)
  246. if err != nil {
  247. return nil, err
  248. }
  249. metadata = client.cachedMetadata(topic, partitionID)
  250. }
  251. if metadata == nil {
  252. return nil, ErrUnknownTopicOrPartition
  253. }
  254. if metadata.Err == ErrReplicaNotAvailable {
  255. return nil, metadata.Err
  256. }
  257. return dupeAndSort(metadata.Isr), nil
  258. }
  259. func (client *client) Leader(topic string, partitionID int32) (*Broker, error) {
  260. if client.Closed() {
  261. return nil, ErrClosedClient
  262. }
  263. leader, err := client.cachedLeader(topic, partitionID)
  264. if leader == nil {
  265. err = client.RefreshMetadata(topic)
  266. if err != nil {
  267. return nil, err
  268. }
  269. leader, err = client.cachedLeader(topic, partitionID)
  270. }
  271. return leader, err
  272. }
  273. func (client *client) RefreshMetadata(topics ...string) error {
  274. if client.Closed() {
  275. return ErrClosedClient
  276. }
  277. // Prior to 0.8.2, Kafka will throw exceptions on an empty topic and not return a proper
  278. // error. This handles the case by returning an error instead of sending it
  279. // off to Kafka. See: https://github.com/Shopify/sarama/pull/38#issuecomment-26362310
  280. for _, topic := range topics {
  281. if len(topic) == 0 {
  282. return ErrInvalidTopic // this is the error that 0.8.2 and later correctly return
  283. }
  284. }
  285. return client.tryRefreshMetadata(topics, client.conf.Metadata.Retry.Max)
  286. }
  287. func (client *client) GetOffset(topic string, partitionID int32, time int64) (int64, error) {
  288. if client.Closed() {
  289. return -1, ErrClosedClient
  290. }
  291. offset, err := client.getOffset(topic, partitionID, time)
  292. if err != nil {
  293. if err := client.RefreshMetadata(topic); err != nil {
  294. return -1, err
  295. }
  296. return client.getOffset(topic, partitionID, time)
  297. }
  298. return offset, err
  299. }
  300. func (client *client) Coordinator(consumerGroup string) (*Broker, error) {
  301. if client.Closed() {
  302. return nil, ErrClosedClient
  303. }
  304. coordinator := client.cachedCoordinator(consumerGroup)
  305. if coordinator == nil {
  306. if err := client.RefreshCoordinator(consumerGroup); err != nil {
  307. return nil, err
  308. }
  309. coordinator = client.cachedCoordinator(consumerGroup)
  310. }
  311. if coordinator == nil {
  312. return nil, ErrConsumerCoordinatorNotAvailable
  313. }
  314. _ = coordinator.Open(client.conf)
  315. return coordinator, nil
  316. }
  317. func (client *client) RefreshCoordinator(consumerGroup string) error {
  318. if client.Closed() {
  319. return ErrClosedClient
  320. }
  321. response, err := client.getConsumerMetadata(consumerGroup, client.conf.Metadata.Retry.Max)
  322. if err != nil {
  323. return err
  324. }
  325. client.lock.Lock()
  326. defer client.lock.Unlock()
  327. client.registerBroker(response.Coordinator)
  328. client.coordinators[consumerGroup] = response.Coordinator.ID()
  329. return nil
  330. }
  331. // private broker management helpers
  332. // registerBroker makes sure a broker received by a Metadata or Coordinator request is registered
  333. // in the brokers map. It returns the broker that is registered, which may be the provided broker,
  334. // or a previously registered Broker instance. You must hold the write lock before calling this function.
  335. func (client *client) registerBroker(broker *Broker) {
  336. if client.brokers[broker.ID()] == nil {
  337. client.brokers[broker.ID()] = broker
  338. Logger.Printf("client/brokers registered new broker #%d at %s", broker.ID(), broker.Addr())
  339. } else if broker.Addr() != client.brokers[broker.ID()].Addr() {
  340. safeAsyncClose(client.brokers[broker.ID()])
  341. client.brokers[broker.ID()] = broker
  342. Logger.Printf("client/brokers replaced registered broker #%d with %s", broker.ID(), broker.Addr())
  343. }
  344. }
  345. // deregisterBroker removes a broker from the seedsBroker list, and if it's
  346. // not the seedbroker, removes it from brokers map completely.
  347. func (client *client) deregisterBroker(broker *Broker) {
  348. client.lock.Lock()
  349. defer client.lock.Unlock()
  350. if len(client.seedBrokers) > 0 && broker == client.seedBrokers[0] {
  351. client.deadSeeds = append(client.deadSeeds, broker)
  352. client.seedBrokers = client.seedBrokers[1:]
  353. } else {
  354. // we do this so that our loop in `tryRefreshMetadata` doesn't go on forever,
  355. // but we really shouldn't have to; once that loop is made better this case can be
  356. // removed, and the function generally can be renamed from `deregisterBroker` to
  357. // `nextSeedBroker` or something
  358. Logger.Printf("client/brokers deregistered broker #%d at %s", broker.ID(), broker.Addr())
  359. delete(client.brokers, broker.ID())
  360. }
  361. }
  362. func (client *client) resurrectDeadBrokers() {
  363. client.lock.Lock()
  364. defer client.lock.Unlock()
  365. Logger.Printf("client/brokers resurrecting %d dead seed brokers", len(client.deadSeeds))
  366. client.seedBrokers = append(client.seedBrokers, client.deadSeeds...)
  367. client.deadSeeds = nil
  368. }
  369. func (client *client) any() *Broker {
  370. client.lock.RLock()
  371. defer client.lock.RUnlock()
  372. if len(client.seedBrokers) > 0 {
  373. _ = client.seedBrokers[0].Open(client.conf)
  374. return client.seedBrokers[0]
  375. }
  376. // not guaranteed to be random *or* deterministic
  377. for _, broker := range client.brokers {
  378. _ = broker.Open(client.conf)
  379. return broker
  380. }
  381. return nil
  382. }
  383. // private caching/lazy metadata helpers
  384. type partitionType int
  385. const (
  386. allPartitions partitionType = iota
  387. writablePartitions
  388. // If you add any more types, update the partition cache in update()
  389. // Ensure this is the last partition type value
  390. maxPartitionIndex
  391. )
  392. func (client *client) cachedMetadata(topic string, partitionID int32) *PartitionMetadata {
  393. client.lock.RLock()
  394. defer client.lock.RUnlock()
  395. partitions := client.metadata[topic]
  396. if partitions != nil {
  397. return partitions[partitionID]
  398. }
  399. return nil
  400. }
  401. func (client *client) cachedPartitions(topic string, partitionSet partitionType) []int32 {
  402. client.lock.RLock()
  403. defer client.lock.RUnlock()
  404. partitions, exists := client.cachedPartitionsResults[topic]
  405. if !exists {
  406. return nil
  407. }
  408. return partitions[partitionSet]
  409. }
  410. func (client *client) setPartitionCache(topic string, partitionSet partitionType) []int32 {
  411. partitions := client.metadata[topic]
  412. if partitions == nil {
  413. return nil
  414. }
  415. ret := make([]int32, 0, len(partitions))
  416. for _, partition := range partitions {
  417. if partitionSet == writablePartitions && partition.Err == ErrLeaderNotAvailable {
  418. continue
  419. }
  420. ret = append(ret, partition.ID)
  421. }
  422. sort.Sort(int32Slice(ret))
  423. return ret
  424. }
  425. func (client *client) cachedLeader(topic string, partitionID int32) (*Broker, error) {
  426. client.lock.RLock()
  427. defer client.lock.RUnlock()
  428. partitions := client.metadata[topic]
  429. if partitions != nil {
  430. metadata, ok := partitions[partitionID]
  431. if ok {
  432. if metadata.Err == ErrLeaderNotAvailable {
  433. return nil, ErrLeaderNotAvailable
  434. }
  435. b := client.brokers[metadata.Leader]
  436. if b == nil {
  437. return nil, ErrLeaderNotAvailable
  438. }
  439. _ = b.Open(client.conf)
  440. return b, nil
  441. }
  442. }
  443. return nil, ErrUnknownTopicOrPartition
  444. }
  445. func (client *client) getOffset(topic string, partitionID int32, time int64) (int64, error) {
  446. broker, err := client.Leader(topic, partitionID)
  447. if err != nil {
  448. return -1, err
  449. }
  450. request := &OffsetRequest{}
  451. if client.conf.Version.IsAtLeast(V0_10_1_0) {
  452. request.Version = 1
  453. }
  454. request.AddBlock(topic, partitionID, time, 1)
  455. response, err := broker.GetAvailableOffsets(request)
  456. if err != nil {
  457. _ = broker.Close()
  458. return -1, err
  459. }
  460. block := response.GetBlock(topic, partitionID)
  461. if block == nil {
  462. _ = broker.Close()
  463. return -1, ErrIncompleteResponse
  464. }
  465. if block.Err != ErrNoError {
  466. return -1, block.Err
  467. }
  468. if len(block.Offsets) != 1 {
  469. return -1, ErrOffsetOutOfRange
  470. }
  471. return block.Offsets[0], nil
  472. }
  473. // core metadata update logic
  474. func (client *client) backgroundMetadataUpdater() {
  475. defer close(client.closed)
  476. if client.conf.Metadata.RefreshFrequency == time.Duration(0) {
  477. return
  478. }
  479. ticker := time.NewTicker(client.conf.Metadata.RefreshFrequency)
  480. defer ticker.Stop()
  481. for {
  482. select {
  483. case <-ticker.C:
  484. if err := client.RefreshMetadata(); err != nil {
  485. Logger.Println("Client background metadata update:", err)
  486. }
  487. case <-client.closer:
  488. return
  489. }
  490. }
  491. }
  492. func (client *client) tryRefreshMetadata(topics []string, attemptsRemaining int) error {
  493. retry := func(err error) error {
  494. if attemptsRemaining > 0 {
  495. Logger.Printf("client/metadata retrying after %dms... (%d attempts remaining)\n", client.conf.Metadata.Retry.Backoff/time.Millisecond, attemptsRemaining)
  496. time.Sleep(client.conf.Metadata.Retry.Backoff)
  497. return client.tryRefreshMetadata(topics, attemptsRemaining-1)
  498. }
  499. return err
  500. }
  501. for broker := client.any(); broker != nil; broker = client.any() {
  502. if len(topics) > 0 {
  503. Logger.Printf("client/metadata fetching metadata for %v from broker %s\n", topics, broker.addr)
  504. } else {
  505. Logger.Printf("client/metadata fetching metadata for all topics from broker %s\n", broker.addr)
  506. }
  507. response, err := broker.GetMetadata(&MetadataRequest{Topics: topics})
  508. switch err.(type) {
  509. case nil:
  510. // valid response, use it
  511. shouldRetry, err := client.updateMetadata(response)
  512. if shouldRetry {
  513. Logger.Println("client/metadata found some partitions to be leaderless")
  514. return retry(err) // note: err can be nil
  515. }
  516. return err
  517. case PacketEncodingError:
  518. // didn't even send, return the error
  519. return err
  520. default:
  521. // some other error, remove that broker and try again
  522. Logger.Println("client/metadata got error from broker while fetching metadata:", err)
  523. _ = broker.Close()
  524. client.deregisterBroker(broker)
  525. }
  526. }
  527. Logger.Println("client/metadata no available broker to send metadata request to")
  528. client.resurrectDeadBrokers()
  529. return retry(ErrOutOfBrokers)
  530. }
  531. // if no fatal error, returns a list of topics that need retrying due to ErrLeaderNotAvailable
  532. func (client *client) updateMetadata(data *MetadataResponse) (retry bool, err error) {
  533. client.lock.Lock()
  534. defer client.lock.Unlock()
  535. // For all the brokers we received:
  536. // - if it is a new ID, save it
  537. // - if it is an existing ID, but the address we have is stale, discard the old one and save it
  538. // - otherwise ignore it, replacing our existing one would just bounce the connection
  539. for _, broker := range data.Brokers {
  540. client.registerBroker(broker)
  541. }
  542. for _, topic := range data.Topics {
  543. delete(client.metadata, topic.Name)
  544. delete(client.cachedPartitionsResults, topic.Name)
  545. switch topic.Err {
  546. case ErrNoError:
  547. break
  548. case ErrInvalidTopic, ErrTopicAuthorizationFailed: // don't retry, don't store partial results
  549. err = topic.Err
  550. continue
  551. case ErrUnknownTopicOrPartition: // retry, do not store partial partition results
  552. err = topic.Err
  553. retry = true
  554. continue
  555. case ErrLeaderNotAvailable: // retry, but store partial partition results
  556. retry = true
  557. break
  558. default: // don't retry, don't store partial results
  559. Logger.Printf("Unexpected topic-level metadata error: %s", topic.Err)
  560. err = topic.Err
  561. continue
  562. }
  563. client.metadata[topic.Name] = make(map[int32]*PartitionMetadata, len(topic.Partitions))
  564. for _, partition := range topic.Partitions {
  565. client.metadata[topic.Name][partition.ID] = partition
  566. if partition.Err == ErrLeaderNotAvailable {
  567. retry = true
  568. }
  569. }
  570. var partitionCache [maxPartitionIndex][]int32
  571. partitionCache[allPartitions] = client.setPartitionCache(topic.Name, allPartitions)
  572. partitionCache[writablePartitions] = client.setPartitionCache(topic.Name, writablePartitions)
  573. client.cachedPartitionsResults[topic.Name] = partitionCache
  574. }
  575. return
  576. }
  577. func (client *client) cachedCoordinator(consumerGroup string) *Broker {
  578. client.lock.RLock()
  579. defer client.lock.RUnlock()
  580. if coordinatorID, ok := client.coordinators[consumerGroup]; ok {
  581. return client.brokers[coordinatorID]
  582. }
  583. return nil
  584. }
  585. func (client *client) getConsumerMetadata(consumerGroup string, attemptsRemaining int) (*ConsumerMetadataResponse, error) {
  586. retry := func(err error) (*ConsumerMetadataResponse, error) {
  587. if attemptsRemaining > 0 {
  588. Logger.Printf("client/coordinator retrying after %dms... (%d attempts remaining)\n", client.conf.Metadata.Retry.Backoff/time.Millisecond, attemptsRemaining)
  589. time.Sleep(client.conf.Metadata.Retry.Backoff)
  590. return client.getConsumerMetadata(consumerGroup, attemptsRemaining-1)
  591. }
  592. return nil, err
  593. }
  594. for broker := client.any(); broker != nil; broker = client.any() {
  595. Logger.Printf("client/coordinator requesting coordinator for consumergroup %s from %s\n", consumerGroup, broker.Addr())
  596. request := new(ConsumerMetadataRequest)
  597. request.ConsumerGroup = consumerGroup
  598. response, err := broker.GetConsumerMetadata(request)
  599. if err != nil {
  600. Logger.Printf("client/coordinator request to broker %s failed: %s\n", broker.Addr(), err)
  601. switch err.(type) {
  602. case PacketEncodingError:
  603. return nil, err
  604. default:
  605. _ = broker.Close()
  606. client.deregisterBroker(broker)
  607. continue
  608. }
  609. }
  610. switch response.Err {
  611. case ErrNoError:
  612. Logger.Printf("client/coordinator coordinator for consumergroup %s is #%d (%s)\n", consumerGroup, response.Coordinator.ID(), response.Coordinator.Addr())
  613. return response, nil
  614. case ErrConsumerCoordinatorNotAvailable:
  615. Logger.Printf("client/coordinator coordinator for consumer group %s is not available\n", consumerGroup)
  616. // This is very ugly, but this scenario will only happen once per cluster.
  617. // The __consumer_offsets topic only has to be created one time.
  618. // The number of partitions not configurable, but partition 0 should always exist.
  619. if _, err := client.Leader("__consumer_offsets", 0); err != nil {
  620. Logger.Printf("client/coordinator the __consumer_offsets topic is not initialized completely yet. Waiting 2 seconds...\n")
  621. time.Sleep(2 * time.Second)
  622. }
  623. return retry(ErrConsumerCoordinatorNotAvailable)
  624. default:
  625. return nil, response.Err
  626. }
  627. }
  628. Logger.Println("client/coordinator no available broker to send consumer metadata request to")
  629. client.resurrectDeadBrokers()
  630. return retry(ErrOutOfBrokers)
  631. }