client.go 30 KB

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