client.go 24 KB

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