client.go 33 KB

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