client.go 32 KB

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