client.go 24 KB

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