client.go 31 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676677678679680681682683684685686687688689690691692693694695696697698699700701702703704705706707708709710711712713714715716717718719720721722723724725726727728729730731732733734735736737738739740741742743744745746747748749750751752753754755756757758759760761762763764765766767768769770771772773774775776777778779780781782783784785786787788789790791792793794795796797798799800801802803804805806807808809810811812813814815816817818819820821822823824825826827828829830831832833834835836837838839840841842843844845846847848849850851852853854855856857858859860861862863864865866867868869870871872873874875876877878879880881882883884885886887888889890891892893894895896897898899900901902903904905906907908909910911912913914915916917918919920921922923924925926927928929930931932933934935936937938939940941942943944945946947948949950951952953954955956957958959960961962963964965966967968969970971972973974975976977978979980981982983984985986987988989990991992993994995996997998999100010011002100310041005100610071008100910101011101210131014101510161017101810191020102110221023102410251026102710281029103010311032103310341035103610371038103910401041104210431044104510461047
  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. // registerBroker makes sure a broker received by a Metadata or Coordinator request is registered
  454. // in the brokers map. It returns the broker that is registered, which may be the provided broker,
  455. // or a previously registered Broker instance. You must hold the write lock before calling this function.
  456. func (client *client) registerBroker(broker *Broker) {
  457. if client.brokers == nil {
  458. Logger.Printf("cannot register broker #%d at %s, client already closed", broker.ID(), broker.Addr())
  459. return
  460. }
  461. if client.brokers[broker.ID()] == nil {
  462. client.brokers[broker.ID()] = broker
  463. Logger.Printf("client/brokers registered new broker #%d at %s", broker.ID(), broker.Addr())
  464. } else if broker.Addr() != client.brokers[broker.ID()].Addr() {
  465. safeAsyncClose(client.brokers[broker.ID()])
  466. client.brokers[broker.ID()] = broker
  467. Logger.Printf("client/brokers replaced registered broker #%d with %s", broker.ID(), broker.Addr())
  468. }
  469. }
  470. // deregisterBroker removes a broker from the seedsBroker list, and if it's
  471. // not the seedbroker, removes it from brokers map completely.
  472. func (client *client) deregisterBroker(broker *Broker) {
  473. client.lock.Lock()
  474. defer client.lock.Unlock()
  475. if len(client.seedBrokers) > 0 && broker == client.seedBrokers[0] {
  476. client.deadSeeds = append(client.deadSeeds, broker)
  477. client.seedBrokers = client.seedBrokers[1:]
  478. } else {
  479. // we do this so that our loop in `tryRefreshMetadata` doesn't go on forever,
  480. // but we really shouldn't have to; once that loop is made better this case can be
  481. // removed, and the function generally can be renamed from `deregisterBroker` to
  482. // `nextSeedBroker` or something
  483. Logger.Printf("client/brokers deregistered broker #%d at %s", broker.ID(), broker.Addr())
  484. delete(client.brokers, broker.ID())
  485. }
  486. }
  487. func (client *client) resurrectDeadBrokers() {
  488. client.lock.Lock()
  489. defer client.lock.Unlock()
  490. Logger.Printf("client/brokers resurrecting %d dead seed brokers", len(client.deadSeeds))
  491. client.seedBrokers = append(client.seedBrokers, client.deadSeeds...)
  492. client.deadSeeds = nil
  493. }
  494. func (client *client) any() *Broker {
  495. client.lock.RLock()
  496. defer client.lock.RUnlock()
  497. if len(client.seedBrokers) > 0 {
  498. _ = client.seedBrokers[0].Open(client.conf)
  499. return client.seedBrokers[0]
  500. }
  501. // not guaranteed to be random *or* deterministic
  502. for _, broker := range client.brokers {
  503. _ = broker.Open(client.conf)
  504. return broker
  505. }
  506. return nil
  507. }
  508. // private caching/lazy metadata helpers
  509. type partitionType int
  510. const (
  511. allPartitions partitionType = iota
  512. writablePartitions
  513. // If you add any more types, update the partition cache in update()
  514. // Ensure this is the last partition type value
  515. maxPartitionIndex
  516. )
  517. func (client *client) cachedMetadata(topic string, partitionID int32) *PartitionMetadata {
  518. client.lock.RLock()
  519. defer client.lock.RUnlock()
  520. partitions := client.metadata[topic]
  521. if partitions != nil {
  522. return partitions[partitionID]
  523. }
  524. return nil
  525. }
  526. func (client *client) cachedPartitions(topic string, partitionSet partitionType) []int32 {
  527. client.lock.RLock()
  528. defer client.lock.RUnlock()
  529. partitions, exists := client.cachedPartitionsResults[topic]
  530. if !exists {
  531. return nil
  532. }
  533. return partitions[partitionSet]
  534. }
  535. func (client *client) setPartitionCache(topic string, partitionSet partitionType) []int32 {
  536. partitions := client.metadata[topic]
  537. if partitions == nil {
  538. return nil
  539. }
  540. ret := make([]int32, 0, len(partitions))
  541. for _, partition := range partitions {
  542. if partitionSet == writablePartitions && partition.Err == ErrLeaderNotAvailable {
  543. continue
  544. }
  545. ret = append(ret, partition.ID)
  546. }
  547. sort.Sort(int32Slice(ret))
  548. return ret
  549. }
  550. func (client *client) cachedLeader(topic string, partitionID int32) (*Broker, error) {
  551. client.lock.RLock()
  552. defer client.lock.RUnlock()
  553. partitions := client.metadata[topic]
  554. if partitions != nil {
  555. metadata, ok := partitions[partitionID]
  556. if ok {
  557. if metadata.Err == ErrLeaderNotAvailable {
  558. return nil, ErrLeaderNotAvailable
  559. }
  560. b := client.brokers[metadata.Leader]
  561. if b == nil {
  562. return nil, ErrLeaderNotAvailable
  563. }
  564. _ = b.Open(client.conf)
  565. return b, nil
  566. }
  567. }
  568. return nil, ErrUnknownTopicOrPartition
  569. }
  570. func (client *client) getOffset(topic string, partitionID int32, time int64) (int64, error) {
  571. broker, err := client.Leader(topic, partitionID)
  572. if err != nil {
  573. return -1, err
  574. }
  575. request := &OffsetRequest{}
  576. if client.conf.Version.IsAtLeast(V0_10_1_0) {
  577. request.Version = 1
  578. }
  579. request.AddBlock(topic, partitionID, time, 1)
  580. response, err := broker.GetAvailableOffsets(request)
  581. if err != nil {
  582. _ = broker.Close()
  583. return -1, err
  584. }
  585. block := response.GetBlock(topic, partitionID)
  586. if block == nil {
  587. _ = broker.Close()
  588. return -1, ErrIncompleteResponse
  589. }
  590. if block.Err != ErrNoError {
  591. return -1, block.Err
  592. }
  593. if len(block.Offsets) != 1 {
  594. return -1, ErrOffsetOutOfRange
  595. }
  596. return block.Offsets[0], nil
  597. }
  598. // core metadata update logic
  599. func (client *client) backgroundMetadataUpdater() {
  600. defer close(client.closed)
  601. if client.conf.Metadata.RefreshFrequency == time.Duration(0) {
  602. return
  603. }
  604. ticker := time.NewTicker(client.conf.Metadata.RefreshFrequency)
  605. defer ticker.Stop()
  606. for {
  607. select {
  608. case <-ticker.C:
  609. if err := client.refreshMetadata(); err != nil {
  610. Logger.Println("Client background metadata update:", err)
  611. }
  612. case <-client.closer:
  613. return
  614. }
  615. }
  616. }
  617. func (client *client) refreshMetadata() error {
  618. topics := []string{}
  619. if !client.conf.Metadata.Full {
  620. if specificTopics, err := client.MetadataTopics(); err != nil {
  621. return err
  622. } else if len(specificTopics) == 0 {
  623. return ErrNoTopicsToUpdateMetadata
  624. } else {
  625. topics = specificTopics
  626. }
  627. }
  628. if err := client.RefreshMetadata(topics...); err != nil {
  629. return err
  630. }
  631. return nil
  632. }
  633. func (client *client) tryRefreshMetadata(topics []string, attemptsRemaining int, deadline time.Time) error {
  634. pastDeadline := func(backoff time.Duration) bool {
  635. if !deadline.IsZero() && time.Now().Add(backoff).After(deadline) {
  636. // we are past the deadline
  637. return true
  638. }
  639. return false
  640. }
  641. retry := func(err error) error {
  642. if attemptsRemaining > 0 {
  643. backoff := client.computeBackoff(attemptsRemaining)
  644. if pastDeadline(backoff) {
  645. Logger.Println("client/metadata skipping last retries as we would go past the metadata timeout")
  646. return err
  647. }
  648. Logger.Printf("client/metadata retrying after %dms... (%d attempts remaining)\n", client.conf.Metadata.Retry.Backoff/time.Millisecond, attemptsRemaining)
  649. if backoff > 0 {
  650. time.Sleep(backoff)
  651. }
  652. return client.tryRefreshMetadata(topics, attemptsRemaining-1, deadline)
  653. }
  654. return err
  655. }
  656. broker := client.any()
  657. for ; broker != nil && !pastDeadline(0); broker = client.any() {
  658. allowAutoTopicCreation := true
  659. if len(topics) > 0 {
  660. Logger.Printf("client/metadata fetching metadata for %v from broker %s\n", topics, broker.addr)
  661. } else {
  662. allowAutoTopicCreation = false
  663. Logger.Printf("client/metadata fetching metadata for all topics from broker %s\n", broker.addr)
  664. }
  665. req := &MetadataRequest{Topics: topics, AllowAutoTopicCreation: allowAutoTopicCreation}
  666. if client.conf.Version.IsAtLeast(V1_0_0_0) {
  667. req.Version = 5
  668. } else if client.conf.Version.IsAtLeast(V0_10_0_0) {
  669. req.Version = 1
  670. }
  671. response, err := broker.GetMetadata(req)
  672. switch err.(type) {
  673. case nil:
  674. allKnownMetaData := len(topics) == 0
  675. // valid response, use it
  676. shouldRetry, err := client.updateMetadata(response, allKnownMetaData)
  677. if shouldRetry {
  678. Logger.Println("client/metadata found some partitions to be leaderless")
  679. return retry(err) // note: err can be nil
  680. }
  681. return err
  682. case PacketEncodingError:
  683. // didn't even send, return the error
  684. return err
  685. case KError:
  686. // if SASL auth error return as this _should_ be a non retryable err for all brokers
  687. if err.(KError) == ErrSASLAuthenticationFailed {
  688. Logger.Println("client/metadata failed SASL authentication")
  689. return err
  690. }
  691. if err.(KError) == ErrTopicAuthorizationFailed {
  692. Logger.Println("client is not authorized to access this topic. The topics were: ", topics)
  693. return err
  694. }
  695. // else remove that broker and try again
  696. Logger.Printf("client/metadata got error from broker %d while fetching metadata: %v\n", broker.ID(), err)
  697. _ = broker.Close()
  698. client.deregisterBroker(broker)
  699. default:
  700. // some other error, remove that broker and try again
  701. Logger.Printf("client/metadata got error from broker %d while fetching metadata: %v\n", broker.ID(), err)
  702. _ = broker.Close()
  703. client.deregisterBroker(broker)
  704. }
  705. }
  706. if broker != nil {
  707. Logger.Printf("client/metadata not fetching metadata from broker %s as we would go past the metadata timeout\n", broker.addr)
  708. return retry(ErrOutOfBrokers)
  709. }
  710. Logger.Println("client/metadata no available broker to send metadata request to")
  711. client.resurrectDeadBrokers()
  712. return retry(ErrOutOfBrokers)
  713. }
  714. // if no fatal error, returns a list of topics that need retrying due to ErrLeaderNotAvailable
  715. func (client *client) updateMetadata(data *MetadataResponse, allKnownMetaData bool) (retry bool, err error) {
  716. if client.Closed() {
  717. return
  718. }
  719. client.lock.Lock()
  720. defer client.lock.Unlock()
  721. // For all the brokers we received:
  722. // - if it is a new ID, save it
  723. // - if it is an existing ID, but the address we have is stale, discard the old one and save it
  724. // - otherwise ignore it, replacing our existing one would just bounce the connection
  725. for _, broker := range data.Brokers {
  726. client.registerBroker(broker)
  727. }
  728. client.controllerID = data.ControllerID
  729. if allKnownMetaData {
  730. client.metadata = make(map[string]map[int32]*PartitionMetadata)
  731. client.metadataTopics = make(map[string]none)
  732. client.cachedPartitionsResults = make(map[string][maxPartitionIndex][]int32)
  733. }
  734. for _, topic := range data.Topics {
  735. // topics must be added firstly to `metadataTopics` to guarantee that all
  736. // requested topics must be recorded to keep them trackable for periodically
  737. // metadata refresh.
  738. if _, exists := client.metadataTopics[topic.Name]; !exists {
  739. client.metadataTopics[topic.Name] = none{}
  740. }
  741. delete(client.metadata, topic.Name)
  742. delete(client.cachedPartitionsResults, topic.Name)
  743. switch topic.Err {
  744. case ErrNoError:
  745. // no-op
  746. case ErrInvalidTopic, ErrTopicAuthorizationFailed: // don't retry, don't store partial results
  747. err = topic.Err
  748. continue
  749. case ErrUnknownTopicOrPartition: // retry, do not store partial partition results
  750. err = topic.Err
  751. retry = true
  752. continue
  753. case ErrLeaderNotAvailable: // retry, but store partial partition results
  754. retry = true
  755. default: // don't retry, don't store partial results
  756. Logger.Printf("Unexpected topic-level metadata error: %s", topic.Err)
  757. err = topic.Err
  758. continue
  759. }
  760. client.metadata[topic.Name] = make(map[int32]*PartitionMetadata, len(topic.Partitions))
  761. for _, partition := range topic.Partitions {
  762. client.metadata[topic.Name][partition.ID] = partition
  763. if partition.Err == ErrLeaderNotAvailable {
  764. retry = true
  765. }
  766. }
  767. var partitionCache [maxPartitionIndex][]int32
  768. partitionCache[allPartitions] = client.setPartitionCache(topic.Name, allPartitions)
  769. partitionCache[writablePartitions] = client.setPartitionCache(topic.Name, writablePartitions)
  770. client.cachedPartitionsResults[topic.Name] = partitionCache
  771. }
  772. return
  773. }
  774. func (client *client) cachedCoordinator(consumerGroup string) *Broker {
  775. client.lock.RLock()
  776. defer client.lock.RUnlock()
  777. if coordinatorID, ok := client.coordinators[consumerGroup]; ok {
  778. return client.brokers[coordinatorID]
  779. }
  780. return nil
  781. }
  782. func (client *client) cachedController() *Broker {
  783. client.lock.RLock()
  784. defer client.lock.RUnlock()
  785. return client.brokers[client.controllerID]
  786. }
  787. func (client *client) computeBackoff(attemptsRemaining int) time.Duration {
  788. if client.conf.Metadata.Retry.BackoffFunc != nil {
  789. maxRetries := client.conf.Metadata.Retry.Max
  790. retries := maxRetries - attemptsRemaining
  791. return client.conf.Metadata.Retry.BackoffFunc(retries, maxRetries)
  792. }
  793. return client.conf.Metadata.Retry.Backoff
  794. }
  795. func (client *client) getConsumerMetadata(consumerGroup string, attemptsRemaining int) (*FindCoordinatorResponse, error) {
  796. retry := func(err error) (*FindCoordinatorResponse, error) {
  797. if attemptsRemaining > 0 {
  798. backoff := client.computeBackoff(attemptsRemaining)
  799. Logger.Printf("client/coordinator retrying after %dms... (%d attempts remaining)\n", backoff/time.Millisecond, attemptsRemaining)
  800. time.Sleep(backoff)
  801. return client.getConsumerMetadata(consumerGroup, attemptsRemaining-1)
  802. }
  803. return nil, err
  804. }
  805. for broker := client.any(); broker != nil; broker = client.any() {
  806. Logger.Printf("client/coordinator requesting coordinator for consumergroup %s from %s\n", consumerGroup, broker.Addr())
  807. request := new(FindCoordinatorRequest)
  808. request.CoordinatorKey = consumerGroup
  809. request.CoordinatorType = CoordinatorGroup
  810. response, err := broker.FindCoordinator(request)
  811. if err != nil {
  812. Logger.Printf("client/coordinator request to broker %s failed: %s\n", broker.Addr(), err)
  813. switch err.(type) {
  814. case PacketEncodingError:
  815. return nil, err
  816. default:
  817. _ = broker.Close()
  818. client.deregisterBroker(broker)
  819. continue
  820. }
  821. }
  822. switch response.Err {
  823. case ErrNoError:
  824. Logger.Printf("client/coordinator coordinator for consumergroup %s is #%d (%s)\n", consumerGroup, response.Coordinator.ID(), response.Coordinator.Addr())
  825. return response, nil
  826. case ErrConsumerCoordinatorNotAvailable:
  827. Logger.Printf("client/coordinator coordinator for consumer group %s is not available\n", consumerGroup)
  828. // This is very ugly, but this scenario will only happen once per cluster.
  829. // The __consumer_offsets topic only has to be created one time.
  830. // The number of partitions not configurable, but partition 0 should always exist.
  831. if _, err := client.Leader("__consumer_offsets", 0); err != nil {
  832. Logger.Printf("client/coordinator the __consumer_offsets topic is not initialized completely yet. Waiting 2 seconds...\n")
  833. time.Sleep(2 * time.Second)
  834. }
  835. return retry(ErrConsumerCoordinatorNotAvailable)
  836. case ErrGroupAuthorizationFailed:
  837. Logger.Printf("client was not authorized to access group %s while attempting to find coordinator", consumerGroup)
  838. return retry(ErrGroupAuthorizationFailed)
  839. default:
  840. return nil, response.Err
  841. }
  842. }
  843. Logger.Println("client/coordinator no available broker to send consumer metadata request to")
  844. client.resurrectDeadBrokers()
  845. return retry(ErrOutOfBrokers)
  846. }
  847. // nopCloserClient embeds an existing Client, but disables
  848. // the Close method (yet all other methods pass
  849. // through unchanged). This is for use in larger structs
  850. // where it is undesirable to close the client that was
  851. // passed in by the caller.
  852. type nopCloserClient struct {
  853. Client
  854. }
  855. // Close intercepts and purposely does not call the underlying
  856. // client's Close() method.
  857. func (ncc *nopCloserClient) Close() error {
  858. return nil
  859. }