client.go 30 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676677678679680681682683684685686687688689690691692693694695696697698699700701702703704705706707708709710711712713714715716717718719720721722723724725726727728729730731732733734735736737738739740741742743744745746747748749750751752753754755756757758759760761762763764765766767768769770771772773774775776777778779780781782783784785786787788789790791792793794795796797798799800801802803804805806807808809810811812813814815816817818819820821822823824825826827828829830831832833834835836837838839840841842843844845846847848849850851852853854855856857858859860861862863864865866867868869870871872873874875876877878879880881882883884885886887888889890891892893894895896897898899900901902903904905906907908909910911912913914915916917918919920921922923924925926927928929930931932933934935936937938939940941942943944945946947948949950951952953954955956957958959960961962963964965966967968969970971972973974975976977978979980981982983984985986987988989990991992
  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. return client.brokers == nil
  200. }
  201. func (client *client) Topics() ([]string, error) {
  202. if client.Closed() {
  203. return nil, ErrClosedClient
  204. }
  205. client.lock.RLock()
  206. defer client.lock.RUnlock()
  207. ret := make([]string, 0, len(client.metadata))
  208. for topic := range client.metadata {
  209. ret = append(ret, topic)
  210. }
  211. return ret, nil
  212. }
  213. func (client *client) MetadataTopics() ([]string, error) {
  214. if client.Closed() {
  215. return nil, ErrClosedClient
  216. }
  217. client.lock.RLock()
  218. defer client.lock.RUnlock()
  219. ret := make([]string, 0, len(client.metadataTopics))
  220. for topic := range client.metadataTopics {
  221. ret = append(ret, topic)
  222. }
  223. return ret, nil
  224. }
  225. func (client *client) Partitions(topic string) ([]int32, error) {
  226. if client.Closed() {
  227. return nil, ErrClosedClient
  228. }
  229. partitions := client.cachedPartitions(topic, allPartitions)
  230. if len(partitions) == 0 {
  231. err := client.RefreshMetadata(topic)
  232. if err != nil {
  233. return nil, err
  234. }
  235. partitions = client.cachedPartitions(topic, allPartitions)
  236. }
  237. // no partitions found after refresh metadata
  238. if len(partitions) == 0 {
  239. return nil, ErrUnknownTopicOrPartition
  240. }
  241. return partitions, nil
  242. }
  243. func (client *client) WritablePartitions(topic string) ([]int32, error) {
  244. if client.Closed() {
  245. return nil, ErrClosedClient
  246. }
  247. partitions := client.cachedPartitions(topic, writablePartitions)
  248. // len==0 catches when it's nil (no such topic) and the odd case when every single
  249. // partition is undergoing leader election simultaneously. Callers have to be able to handle
  250. // this function returning an empty slice (which is a valid return value) but catching it
  251. // here the first time (note we *don't* catch it below where we return ErrUnknownTopicOrPartition) triggers
  252. // a metadata refresh as a nicety so callers can just try again and don't have to manually
  253. // trigger a refresh (otherwise they'd just keep getting a stale cached copy).
  254. if len(partitions) == 0 {
  255. err := client.RefreshMetadata(topic)
  256. if err != nil {
  257. return nil, err
  258. }
  259. partitions = client.cachedPartitions(topic, writablePartitions)
  260. }
  261. if partitions == nil {
  262. return nil, ErrUnknownTopicOrPartition
  263. }
  264. return partitions, nil
  265. }
  266. func (client *client) Replicas(topic string, partitionID int32) ([]int32, error) {
  267. if client.Closed() {
  268. return nil, ErrClosedClient
  269. }
  270. metadata := client.cachedMetadata(topic, partitionID)
  271. if metadata == nil {
  272. err := client.RefreshMetadata(topic)
  273. if err != nil {
  274. return nil, err
  275. }
  276. metadata = client.cachedMetadata(topic, partitionID)
  277. }
  278. if metadata == nil {
  279. return nil, ErrUnknownTopicOrPartition
  280. }
  281. if metadata.Err == ErrReplicaNotAvailable {
  282. return dupInt32Slice(metadata.Replicas), metadata.Err
  283. }
  284. return dupInt32Slice(metadata.Replicas), nil
  285. }
  286. func (client *client) InSyncReplicas(topic string, partitionID int32) ([]int32, error) {
  287. if client.Closed() {
  288. return nil, ErrClosedClient
  289. }
  290. metadata := client.cachedMetadata(topic, partitionID)
  291. if metadata == nil {
  292. err := client.RefreshMetadata(topic)
  293. if err != nil {
  294. return nil, err
  295. }
  296. metadata = client.cachedMetadata(topic, partitionID)
  297. }
  298. if metadata == nil {
  299. return nil, ErrUnknownTopicOrPartition
  300. }
  301. if metadata.Err == ErrReplicaNotAvailable {
  302. return dupInt32Slice(metadata.Isr), metadata.Err
  303. }
  304. return dupInt32Slice(metadata.Isr), nil
  305. }
  306. func (client *client) OfflineReplicas(topic string, partitionID int32) ([]int32, error) {
  307. if client.Closed() {
  308. return nil, ErrClosedClient
  309. }
  310. metadata := client.cachedMetadata(topic, partitionID)
  311. if metadata == nil {
  312. err := client.RefreshMetadata(topic)
  313. if err != nil {
  314. return nil, err
  315. }
  316. metadata = client.cachedMetadata(topic, partitionID)
  317. }
  318. if metadata == nil {
  319. return nil, ErrUnknownTopicOrPartition
  320. }
  321. if metadata.Err == ErrReplicaNotAvailable {
  322. return dupInt32Slice(metadata.OfflineReplicas), metadata.Err
  323. }
  324. return dupInt32Slice(metadata.OfflineReplicas), nil
  325. }
  326. func (client *client) Leader(topic string, partitionID int32) (*Broker, error) {
  327. if client.Closed() {
  328. return nil, ErrClosedClient
  329. }
  330. leader, err := client.cachedLeader(topic, partitionID)
  331. if leader == nil {
  332. err = client.RefreshMetadata(topic)
  333. if err != nil {
  334. return nil, err
  335. }
  336. leader, err = client.cachedLeader(topic, partitionID)
  337. }
  338. return leader, err
  339. }
  340. func (client *client) RefreshMetadata(topics ...string) error {
  341. if client.Closed() {
  342. return ErrClosedClient
  343. }
  344. // Prior to 0.8.2, Kafka will throw exceptions on an empty topic and not return a proper
  345. // error. This handles the case by returning an error instead of sending it
  346. // off to Kafka. See: https://github.com/Shopify/sarama/pull/38#issuecomment-26362310
  347. for _, topic := range topics {
  348. if len(topic) == 0 {
  349. return ErrInvalidTopic // this is the error that 0.8.2 and later correctly return
  350. }
  351. }
  352. deadline := time.Time{}
  353. if client.conf.Metadata.Timeout > 0 {
  354. deadline = time.Now().Add(client.conf.Metadata.Timeout)
  355. }
  356. return client.tryRefreshMetadata(topics, client.conf.Metadata.Retry.Max, deadline)
  357. }
  358. func (client *client) GetOffset(topic string, partitionID int32, time int64) (int64, error) {
  359. if client.Closed() {
  360. return -1, ErrClosedClient
  361. }
  362. offset, err := client.getOffset(topic, partitionID, time)
  363. if err != nil {
  364. if err := client.RefreshMetadata(topic); err != nil {
  365. return -1, err
  366. }
  367. return client.getOffset(topic, partitionID, time)
  368. }
  369. return offset, err
  370. }
  371. func (client *client) Controller() (*Broker, error) {
  372. if client.Closed() {
  373. return nil, ErrClosedClient
  374. }
  375. if !client.conf.Version.IsAtLeast(V0_10_0_0) {
  376. return nil, ErrUnsupportedVersion
  377. }
  378. controller := client.cachedController()
  379. if controller == nil {
  380. if err := client.refreshMetadata(); err != nil {
  381. return nil, err
  382. }
  383. controller = client.cachedController()
  384. }
  385. if controller == nil {
  386. return nil, ErrControllerNotAvailable
  387. }
  388. _ = controller.Open(client.conf)
  389. return controller, nil
  390. }
  391. func (client *client) Coordinator(consumerGroup string) (*Broker, error) {
  392. if client.Closed() {
  393. return nil, ErrClosedClient
  394. }
  395. coordinator := client.cachedCoordinator(consumerGroup)
  396. if coordinator == nil {
  397. if err := client.RefreshCoordinator(consumerGroup); err != nil {
  398. return nil, err
  399. }
  400. coordinator = client.cachedCoordinator(consumerGroup)
  401. }
  402. if coordinator == nil {
  403. return nil, ErrConsumerCoordinatorNotAvailable
  404. }
  405. _ = coordinator.Open(client.conf)
  406. return coordinator, nil
  407. }
  408. func (client *client) RefreshCoordinator(consumerGroup string) error {
  409. if client.Closed() {
  410. return ErrClosedClient
  411. }
  412. response, err := client.getConsumerMetadata(consumerGroup, client.conf.Metadata.Retry.Max)
  413. if err != nil {
  414. return err
  415. }
  416. client.lock.Lock()
  417. defer client.lock.Unlock()
  418. client.registerBroker(response.Coordinator)
  419. client.coordinators[consumerGroup] = response.Coordinator.ID()
  420. return nil
  421. }
  422. // private broker management helpers
  423. // registerBroker makes sure a broker received by a Metadata or Coordinator request is registered
  424. // in the brokers map. It returns the broker that is registered, which may be the provided broker,
  425. // or a previously registered Broker instance. You must hold the write lock before calling this function.
  426. func (client *client) registerBroker(broker *Broker) {
  427. if client.brokers[broker.ID()] == nil {
  428. client.brokers[broker.ID()] = broker
  429. Logger.Printf("client/brokers registered new broker #%d at %s", broker.ID(), broker.Addr())
  430. } else if broker.Addr() != client.brokers[broker.ID()].Addr() {
  431. safeAsyncClose(client.brokers[broker.ID()])
  432. client.brokers[broker.ID()] = broker
  433. Logger.Printf("client/brokers replaced registered broker #%d with %s", broker.ID(), broker.Addr())
  434. }
  435. }
  436. // deregisterBroker removes a broker from the seedsBroker list, and if it's
  437. // not the seedbroker, removes it from brokers map completely.
  438. func (client *client) deregisterBroker(broker *Broker) {
  439. client.lock.Lock()
  440. defer client.lock.Unlock()
  441. if len(client.seedBrokers) > 0 && broker == client.seedBrokers[0] {
  442. client.deadSeeds = append(client.deadSeeds, broker)
  443. client.seedBrokers = client.seedBrokers[1:]
  444. } else {
  445. // we do this so that our loop in `tryRefreshMetadata` doesn't go on forever,
  446. // but we really shouldn't have to; once that loop is made better this case can be
  447. // removed, and the function generally can be renamed from `deregisterBroker` to
  448. // `nextSeedBroker` or something
  449. Logger.Printf("client/brokers deregistered broker #%d at %s", broker.ID(), broker.Addr())
  450. delete(client.brokers, broker.ID())
  451. }
  452. }
  453. func (client *client) resurrectDeadBrokers() {
  454. client.lock.Lock()
  455. defer client.lock.Unlock()
  456. Logger.Printf("client/brokers resurrecting %d dead seed brokers", len(client.deadSeeds))
  457. client.seedBrokers = append(client.seedBrokers, client.deadSeeds...)
  458. client.deadSeeds = nil
  459. }
  460. func (client *client) any() *Broker {
  461. client.lock.RLock()
  462. defer client.lock.RUnlock()
  463. if len(client.seedBrokers) > 0 {
  464. _ = client.seedBrokers[0].Open(client.conf)
  465. return client.seedBrokers[0]
  466. }
  467. // not guaranteed to be random *or* deterministic
  468. for _, broker := range client.brokers {
  469. _ = broker.Open(client.conf)
  470. return broker
  471. }
  472. return nil
  473. }
  474. // private caching/lazy metadata helpers
  475. type partitionType int
  476. const (
  477. allPartitions partitionType = iota
  478. writablePartitions
  479. // If you add any more types, update the partition cache in update()
  480. // Ensure this is the last partition type value
  481. maxPartitionIndex
  482. )
  483. func (client *client) cachedMetadata(topic string, partitionID int32) *PartitionMetadata {
  484. client.lock.RLock()
  485. defer client.lock.RUnlock()
  486. partitions := client.metadata[topic]
  487. if partitions != nil {
  488. return partitions[partitionID]
  489. }
  490. return nil
  491. }
  492. func (client *client) cachedPartitions(topic string, partitionSet partitionType) []int32 {
  493. client.lock.RLock()
  494. defer client.lock.RUnlock()
  495. partitions, exists := client.cachedPartitionsResults[topic]
  496. if !exists {
  497. return nil
  498. }
  499. return partitions[partitionSet]
  500. }
  501. func (client *client) setPartitionCache(topic string, partitionSet partitionType) []int32 {
  502. partitions := client.metadata[topic]
  503. if partitions == nil {
  504. return nil
  505. }
  506. ret := make([]int32, 0, len(partitions))
  507. for _, partition := range partitions {
  508. if partitionSet == writablePartitions && partition.Err == ErrLeaderNotAvailable {
  509. continue
  510. }
  511. ret = append(ret, partition.ID)
  512. }
  513. sort.Sort(int32Slice(ret))
  514. return ret
  515. }
  516. func (client *client) cachedLeader(topic string, partitionID int32) (*Broker, error) {
  517. client.lock.RLock()
  518. defer client.lock.RUnlock()
  519. partitions := client.metadata[topic]
  520. if partitions != nil {
  521. metadata, ok := partitions[partitionID]
  522. if ok {
  523. if metadata.Err == ErrLeaderNotAvailable {
  524. return nil, ErrLeaderNotAvailable
  525. }
  526. b := client.brokers[metadata.Leader]
  527. if b == nil {
  528. return nil, ErrLeaderNotAvailable
  529. }
  530. _ = b.Open(client.conf)
  531. return b, nil
  532. }
  533. }
  534. return nil, ErrUnknownTopicOrPartition
  535. }
  536. func (client *client) getOffset(topic string, partitionID int32, time int64) (int64, error) {
  537. broker, err := client.Leader(topic, partitionID)
  538. if err != nil {
  539. return -1, err
  540. }
  541. request := &OffsetRequest{}
  542. if client.conf.Version.IsAtLeast(V0_10_1_0) {
  543. request.Version = 1
  544. }
  545. request.AddBlock(topic, partitionID, time, 1)
  546. response, err := broker.GetAvailableOffsets(request)
  547. if err != nil {
  548. _ = broker.Close()
  549. return -1, err
  550. }
  551. block := response.GetBlock(topic, partitionID)
  552. if block == nil {
  553. _ = broker.Close()
  554. return -1, ErrIncompleteResponse
  555. }
  556. if block.Err != ErrNoError {
  557. return -1, block.Err
  558. }
  559. if len(block.Offsets) != 1 {
  560. return -1, ErrOffsetOutOfRange
  561. }
  562. return block.Offsets[0], nil
  563. }
  564. // core metadata update logic
  565. func (client *client) backgroundMetadataUpdater() {
  566. defer close(client.closed)
  567. if client.conf.Metadata.RefreshFrequency == time.Duration(0) {
  568. return
  569. }
  570. ticker := time.NewTicker(client.conf.Metadata.RefreshFrequency)
  571. defer ticker.Stop()
  572. for {
  573. select {
  574. case <-ticker.C:
  575. if err := client.refreshMetadata(); err != nil {
  576. Logger.Println("Client background metadata update:", err)
  577. }
  578. case <-client.closer:
  579. return
  580. }
  581. }
  582. }
  583. func (client *client) refreshMetadata() error {
  584. topics := []string{}
  585. if !client.conf.Metadata.Full {
  586. if specificTopics, err := client.MetadataTopics(); err != nil {
  587. return err
  588. } else if len(specificTopics) == 0 {
  589. return ErrNoTopicsToUpdateMetadata
  590. } else {
  591. topics = specificTopics
  592. }
  593. }
  594. if err := client.RefreshMetadata(topics...); err != nil {
  595. return err
  596. }
  597. return nil
  598. }
  599. func (client *client) tryRefreshMetadata(topics []string, attemptsRemaining int, deadline time.Time) error {
  600. pastDeadline := func(backoff time.Duration) bool {
  601. if !deadline.IsZero() && time.Now().Add(backoff).After(deadline) {
  602. // we are past the deadline
  603. return true
  604. }
  605. return false
  606. }
  607. retry := func(err error) error {
  608. if attemptsRemaining > 0 {
  609. backoff := client.computeBackoff(attemptsRemaining)
  610. if pastDeadline(backoff) {
  611. Logger.Println("client/metadata skipping last retries as we would go past the metadata timeout")
  612. return err
  613. }
  614. Logger.Printf("client/metadata retrying after %dms... (%d attempts remaining)\n", client.conf.Metadata.Retry.Backoff/time.Millisecond, attemptsRemaining)
  615. if backoff > 0 {
  616. time.Sleep(backoff)
  617. }
  618. return client.tryRefreshMetadata(topics, attemptsRemaining-1, deadline)
  619. }
  620. return err
  621. }
  622. broker := client.any()
  623. for ; broker != nil && !pastDeadline(0); broker = client.any() {
  624. allowAutoTopicCreation := true
  625. if len(topics) > 0 {
  626. Logger.Printf("client/metadata fetching metadata for %v from broker %s\n", topics, broker.addr)
  627. } else {
  628. allowAutoTopicCreation = false
  629. Logger.Printf("client/metadata fetching metadata for all topics from broker %s\n", broker.addr)
  630. }
  631. req := &MetadataRequest{Topics: topics, AllowAutoTopicCreation: allowAutoTopicCreation}
  632. if client.conf.Version.IsAtLeast(V1_0_0_0) {
  633. req.Version = 5
  634. } else if client.conf.Version.IsAtLeast(V0_10_0_0) {
  635. req.Version = 1
  636. }
  637. response, err := broker.GetMetadata(req)
  638. switch err.(type) {
  639. case nil:
  640. allKnownMetaData := len(topics) == 0
  641. // valid response, use it
  642. shouldRetry, err := client.updateMetadata(response, allKnownMetaData)
  643. if shouldRetry {
  644. Logger.Println("client/metadata found some partitions to be leaderless")
  645. return retry(err) // note: err can be nil
  646. }
  647. return err
  648. case PacketEncodingError:
  649. // didn't even send, return the error
  650. return err
  651. case KError:
  652. // if SASL auth error return as this _should_ be a non retryable err for all brokers
  653. if err.(KError) == ErrSASLAuthenticationFailed {
  654. Logger.Println("client/metadata failed SASL authentication")
  655. return err
  656. }
  657. // else remove that broker and try again
  658. Logger.Printf("client/metadata got error from broker %d while fetching metadata: %v\n", broker.ID(), err)
  659. _ = broker.Close()
  660. client.deregisterBroker(broker)
  661. default:
  662. // some other error, remove that broker and try again
  663. Logger.Printf("client/metadata got error from broker %d while fetching metadata: %v\n", broker.ID(), err)
  664. _ = broker.Close()
  665. client.deregisterBroker(broker)
  666. }
  667. }
  668. if broker != nil {
  669. Logger.Println("client/metadata not fetching metadata from broker %s as we would go past the metadata timeout\n", broker.addr)
  670. return retry(ErrOutOfBrokers)
  671. }
  672. Logger.Println("client/metadata no available broker to send metadata request to")
  673. client.resurrectDeadBrokers()
  674. return retry(ErrOutOfBrokers)
  675. }
  676. // if no fatal error, returns a list of topics that need retrying due to ErrLeaderNotAvailable
  677. func (client *client) updateMetadata(data *MetadataResponse, allKnownMetaData bool) (retry bool, err error) {
  678. client.lock.Lock()
  679. defer client.lock.Unlock()
  680. // For all the brokers we received:
  681. // - if it is a new ID, save it
  682. // - if it is an existing ID, but the address we have is stale, discard the old one and save it
  683. // - otherwise ignore it, replacing our existing one would just bounce the connection
  684. for _, broker := range data.Brokers {
  685. client.registerBroker(broker)
  686. }
  687. client.controllerID = data.ControllerID
  688. if allKnownMetaData {
  689. client.metadata = make(map[string]map[int32]*PartitionMetadata)
  690. client.metadataTopics = make(map[string]none)
  691. client.cachedPartitionsResults = make(map[string][maxPartitionIndex][]int32)
  692. }
  693. for _, topic := range data.Topics {
  694. // topics must be added firstly to `metadataTopics` to guarantee that all
  695. // requested topics must be recorded to keep them trackable for periodically
  696. // metadata refresh.
  697. if _, exists := client.metadataTopics[topic.Name]; !exists {
  698. client.metadataTopics[topic.Name] = none{}
  699. }
  700. delete(client.metadata, topic.Name)
  701. delete(client.cachedPartitionsResults, topic.Name)
  702. switch topic.Err {
  703. case ErrNoError:
  704. // no-op
  705. case ErrInvalidTopic, ErrTopicAuthorizationFailed: // don't retry, don't store partial results
  706. err = topic.Err
  707. continue
  708. case ErrUnknownTopicOrPartition: // retry, do not store partial partition results
  709. err = topic.Err
  710. retry = true
  711. continue
  712. case ErrLeaderNotAvailable: // retry, but store partial partition results
  713. retry = true
  714. default: // don't retry, don't store partial results
  715. Logger.Printf("Unexpected topic-level metadata error: %s", topic.Err)
  716. err = topic.Err
  717. continue
  718. }
  719. client.metadata[topic.Name] = make(map[int32]*PartitionMetadata, len(topic.Partitions))
  720. for _, partition := range topic.Partitions {
  721. client.metadata[topic.Name][partition.ID] = partition
  722. if partition.Err == ErrLeaderNotAvailable {
  723. retry = true
  724. }
  725. }
  726. var partitionCache [maxPartitionIndex][]int32
  727. partitionCache[allPartitions] = client.setPartitionCache(topic.Name, allPartitions)
  728. partitionCache[writablePartitions] = client.setPartitionCache(topic.Name, writablePartitions)
  729. client.cachedPartitionsResults[topic.Name] = partitionCache
  730. }
  731. return
  732. }
  733. func (client *client) cachedCoordinator(consumerGroup string) *Broker {
  734. client.lock.RLock()
  735. defer client.lock.RUnlock()
  736. if coordinatorID, ok := client.coordinators[consumerGroup]; ok {
  737. return client.brokers[coordinatorID]
  738. }
  739. return nil
  740. }
  741. func (client *client) cachedController() *Broker {
  742. client.lock.RLock()
  743. defer client.lock.RUnlock()
  744. return client.brokers[client.controllerID]
  745. }
  746. func (client *client) computeBackoff(attemptsRemaining int) time.Duration {
  747. if client.conf.Metadata.Retry.BackoffFunc != nil {
  748. maxRetries := client.conf.Metadata.Retry.Max
  749. retries := maxRetries - attemptsRemaining
  750. return client.conf.Metadata.Retry.BackoffFunc(retries, maxRetries)
  751. }
  752. return client.conf.Metadata.Retry.Backoff
  753. }
  754. func (client *client) getConsumerMetadata(consumerGroup string, attemptsRemaining int) (*FindCoordinatorResponse, error) {
  755. retry := func(err error) (*FindCoordinatorResponse, error) {
  756. if attemptsRemaining > 0 {
  757. backoff := client.computeBackoff(attemptsRemaining)
  758. Logger.Printf("client/coordinator retrying after %dms... (%d attempts remaining)\n", backoff/time.Millisecond, attemptsRemaining)
  759. time.Sleep(backoff)
  760. return client.getConsumerMetadata(consumerGroup, attemptsRemaining-1)
  761. }
  762. return nil, err
  763. }
  764. for broker := client.any(); broker != nil; broker = client.any() {
  765. Logger.Printf("client/coordinator requesting coordinator for consumergroup %s from %s\n", consumerGroup, broker.Addr())
  766. request := new(FindCoordinatorRequest)
  767. request.CoordinatorKey = consumerGroup
  768. request.CoordinatorType = CoordinatorGroup
  769. response, err := broker.FindCoordinator(request)
  770. if err != nil {
  771. Logger.Printf("client/coordinator request to broker %s failed: %s\n", broker.Addr(), err)
  772. switch err.(type) {
  773. case PacketEncodingError:
  774. return nil, err
  775. default:
  776. _ = broker.Close()
  777. client.deregisterBroker(broker)
  778. continue
  779. }
  780. }
  781. switch response.Err {
  782. case ErrNoError:
  783. Logger.Printf("client/coordinator coordinator for consumergroup %s is #%d (%s)\n", consumerGroup, response.Coordinator.ID(), response.Coordinator.Addr())
  784. return response, nil
  785. case ErrConsumerCoordinatorNotAvailable:
  786. Logger.Printf("client/coordinator coordinator for consumer group %s is not available\n", consumerGroup)
  787. // This is very ugly, but this scenario will only happen once per cluster.
  788. // The __consumer_offsets topic only has to be created one time.
  789. // The number of partitions not configurable, but partition 0 should always exist.
  790. if _, err := client.Leader("__consumer_offsets", 0); err != nil {
  791. Logger.Printf("client/coordinator the __consumer_offsets topic is not initialized completely yet. Waiting 2 seconds...\n")
  792. time.Sleep(2 * time.Second)
  793. }
  794. return retry(ErrConsumerCoordinatorNotAvailable)
  795. default:
  796. return nil, response.Err
  797. }
  798. }
  799. Logger.Println("client/coordinator no available broker to send consumer metadata request to")
  800. client.resurrectDeadBrokers()
  801. return retry(ErrOutOfBrokers)
  802. }
  803. // nopCloserClient embeds an existing Client, but disables
  804. // the Close method (yet all other methods pass
  805. // through unchanged). This is for use in larger structs
  806. // where it is undesirable to close the client that was
  807. // passed in by the caller.
  808. type nopCloserClient struct {
  809. Client
  810. }
  811. // Close intercepts and purposely does not call the underlying
  812. // client's Close() method.
  813. func (ncc *nopCloserClient) Close() error {
  814. return nil
  815. }