client.go 28 KB

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