client.go 19 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634
  1. package sarama
  2. import (
  3. "sort"
  4. "sync"
  5. "time"
  6. )
  7. // ClientConfig is used to pass multiple configuration options to NewClient.
  8. type ClientConfig struct {
  9. MetadataRetries int // How many times to retry a metadata request when a partition is in the middle of leader election.
  10. WaitForElection time.Duration // How long to wait for leader election to finish between retries.
  11. DefaultBrokerConf *BrokerConfig // Default configuration for broker connections created by this client.
  12. BackgroundRefreshFrequency time.Duration // How frequently the client will refresh the cluster metadata in the background. Defaults to 10 minutes. Set to 0 to disable.
  13. }
  14. // NewClientConfig creates a new ClientConfig instance with sensible defaults
  15. func NewClientConfig() *ClientConfig {
  16. return &ClientConfig{
  17. MetadataRetries: 3,
  18. WaitForElection: 250 * time.Millisecond,
  19. BackgroundRefreshFrequency: 10 * time.Minute,
  20. }
  21. }
  22. // Validate checks a ClientConfig instance. This will return a
  23. // ConfigurationError if the specified values don't make sense.
  24. func (config *ClientConfig) Validate() error {
  25. if config.MetadataRetries < 0 {
  26. return ConfigurationError("Invalid MetadataRetries, must be >= 0")
  27. }
  28. if config.WaitForElection <= time.Duration(0) {
  29. return ConfigurationError("Invalid WaitForElection, must be > 0")
  30. }
  31. if config.DefaultBrokerConf != nil {
  32. if err := config.DefaultBrokerConf.Validate(); err != nil {
  33. return err
  34. }
  35. }
  36. if config.BackgroundRefreshFrequency < time.Duration(0) {
  37. return ConfigurationError("Invalid BackgroundRefreshFrequency, must be >= 0")
  38. }
  39. return nil
  40. }
  41. // Client is a generic Kafka client. It manages connections to one or more Kafka brokers.
  42. // You MUST call Close() on a client to avoid leaks, it will not be garbage-collected
  43. // automatically when it passes out of scope. A single client can be safely shared by
  44. // multiple concurrent Producers and Consumers.
  45. type Client struct {
  46. id string
  47. config ClientConfig
  48. closer chan none
  49. // the broker addresses given to us through the constructor are not guaranteed to be returned in
  50. // the cluster metadata (I *think* it only returns brokers who are currently leading partitions?)
  51. // so we store them separately
  52. seedBrokerAddrs []string
  53. seedBroker *Broker
  54. deadBrokerAddrs map[string]none
  55. brokers map[int32]*Broker // maps broker ids to brokers
  56. metadata map[string]map[int32]*PartitionMetadata // maps topics to partition ids to metadata
  57. // If the number of partitions is large, we can get some churn calling cachedPartitions,
  58. // so the result is cached. It is important to update this value whenever metadata is changed
  59. cachedPartitionsResults map[string][maxPartitionIndex][]int32
  60. lock sync.RWMutex // protects access to the maps, only one since they're always written together
  61. }
  62. // NewClient creates a new Client with the given client ID. It connects to one of the given broker addresses
  63. // and uses that broker to automatically fetch metadata on the rest of the kafka cluster. If metadata cannot
  64. // be retrieved from any of the given broker addresses, the client is not created.
  65. func NewClient(id string, addrs []string, config *ClientConfig) (*Client, error) {
  66. Logger.Println("Initializing new client")
  67. if config == nil {
  68. config = NewClientConfig()
  69. }
  70. if err := config.Validate(); err != nil {
  71. return nil, err
  72. }
  73. if len(addrs) < 1 {
  74. return nil, ConfigurationError("You must provide at least one broker address")
  75. }
  76. client := &Client{
  77. id: id,
  78. config: *config,
  79. closer: make(chan none),
  80. seedBrokerAddrs: addrs,
  81. seedBroker: NewBroker(addrs[0]),
  82. deadBrokerAddrs: make(map[string]none),
  83. brokers: make(map[int32]*Broker),
  84. metadata: make(map[string]map[int32]*PartitionMetadata),
  85. cachedPartitionsResults: make(map[string][maxPartitionIndex][]int32),
  86. }
  87. _ = client.seedBroker.Open(config.DefaultBrokerConf)
  88. // do an initial fetch of all cluster metadata by specifing an empty list of topics
  89. err := client.RefreshAllMetadata()
  90. switch err {
  91. case nil:
  92. break
  93. case ErrLeaderNotAvailable, ErrReplicaNotAvailable:
  94. // indicates that maybe part of the cluster is down, but is not fatal to creating the client
  95. Logger.Println(err)
  96. default:
  97. _ = client.Close()
  98. return nil, err
  99. }
  100. go withRecover(client.backgroundMetadataUpdater)
  101. Logger.Println("Successfully initialized new client")
  102. return client, nil
  103. }
  104. // Close shuts down all broker connections managed by this client. It is required to call this function before
  105. // a client object passes out of scope, as it will otherwise leak memory. You must close any Producers or Consumers
  106. // using a client before you close the client.
  107. func (client *Client) Close() error {
  108. // Check to see whether the client is closed
  109. if client.Closed() {
  110. // Chances are this is being called from a defer() and the error will go unobserved
  111. // so we go ahead and log the event in this case.
  112. Logger.Printf("Close() called on already closed client")
  113. return ErrClosedClient
  114. }
  115. client.lock.Lock()
  116. defer client.lock.Unlock()
  117. Logger.Println("Closing Client")
  118. for _, broker := range client.brokers {
  119. safeAsyncClose(broker)
  120. }
  121. client.brokers = nil
  122. client.metadata = nil
  123. if client.seedBroker != nil {
  124. safeAsyncClose(client.seedBroker)
  125. }
  126. close(client.closer)
  127. return nil
  128. }
  129. // Closed returns true if the client has already had Close called on it
  130. func (client *Client) Closed() bool {
  131. return client.brokers == nil
  132. }
  133. // Topics returns the set of available topics as retrieved from the cluster metadata.
  134. func (client *Client) Topics() ([]string, error) {
  135. // Check to see whether the client is closed
  136. if client.Closed() {
  137. return nil, ErrClosedClient
  138. }
  139. client.lock.RLock()
  140. defer client.lock.RUnlock()
  141. ret := make([]string, 0, len(client.metadata))
  142. for topic := range client.metadata {
  143. ret = append(ret, topic)
  144. }
  145. return ret, nil
  146. }
  147. // Partitions returns the sorted list of all partition IDs for the given topic.
  148. func (client *Client) Partitions(topic string) ([]int32, error) {
  149. // Check to see whether the client is closed
  150. if client.Closed() {
  151. return nil, ErrClosedClient
  152. }
  153. partitions := client.cachedPartitions(topic, allPartitions)
  154. if len(partitions) == 0 {
  155. err := client.RefreshTopicMetadata(topic)
  156. if err != nil {
  157. return nil, err
  158. }
  159. partitions = client.cachedPartitions(topic, allPartitions)
  160. }
  161. if partitions == nil {
  162. return nil, ErrUnknownTopicOrPartition
  163. }
  164. return partitions, nil
  165. }
  166. // WritablePartitions returns the sorted list of all writable partition IDs for the given topic,
  167. // where "writable" means "having a valid leader accepting writes".
  168. func (client *Client) WritablePartitions(topic string) ([]int32, error) {
  169. // Check to see whether the client is closed
  170. if client.Closed() {
  171. return nil, ErrClosedClient
  172. }
  173. partitions := client.cachedPartitions(topic, writablePartitions)
  174. // len==0 catches when it's nil (no such topic) and the odd case when every single
  175. // partition is undergoing leader election simultaneously. Callers have to be able to handle
  176. // this function returning an empty slice (which is a valid return value) but catching it
  177. // here the first time (note we *don't* catch it below where we return ErrUnknownTopicOrPartition) triggers
  178. // a metadata refresh as a nicety so callers can just try again and don't have to manually
  179. // trigger a refresh (otherwise they'd just keep getting a stale cached copy).
  180. if len(partitions) == 0 {
  181. err := client.RefreshTopicMetadata(topic)
  182. if err != nil {
  183. return nil, err
  184. }
  185. partitions = client.cachedPartitions(topic, writablePartitions)
  186. }
  187. if partitions == nil {
  188. return nil, ErrUnknownTopicOrPartition
  189. }
  190. return partitions, nil
  191. }
  192. // Replicas returns the set of all replica IDs for the given partition.
  193. func (client *Client) Replicas(topic string, partitionID int32) ([]int32, error) {
  194. if client.Closed() {
  195. return nil, ErrClosedClient
  196. }
  197. metadata, err := client.getMetadata(topic, partitionID)
  198. if err != nil {
  199. return nil, err
  200. }
  201. if metadata.Err == ErrReplicaNotAvailable {
  202. return nil, metadata.Err
  203. }
  204. return dupeAndSort(metadata.Replicas), nil
  205. }
  206. // ReplicasInSync returns the set of all in-sync replica IDs for the given partition.
  207. // Note: kafka's metadata here is known to be stale in many cases, and should not generally be trusted.
  208. // This method should be considered effectively deprecated.
  209. func (client *Client) ReplicasInSync(topic string, partitionID int32) ([]int32, error) {
  210. if client.Closed() {
  211. return nil, ErrClosedClient
  212. }
  213. metadata, err := client.getMetadata(topic, partitionID)
  214. if err != nil {
  215. return nil, err
  216. }
  217. if metadata.Err == ErrReplicaNotAvailable {
  218. return nil, metadata.Err
  219. }
  220. return dupeAndSort(metadata.Isr), nil
  221. }
  222. // Leader returns the broker object that is the leader of the current topic/partition, as
  223. // determined by querying the cluster metadata.
  224. func (client *Client) Leader(topic string, partitionID int32) (*Broker, error) {
  225. leader, err := client.cachedLeader(topic, partitionID)
  226. if leader == nil {
  227. err := client.RefreshTopicMetadata(topic)
  228. if err != nil {
  229. return nil, err
  230. }
  231. leader, err = client.cachedLeader(topic, partitionID)
  232. }
  233. return leader, err
  234. }
  235. // RefreshTopicMetadata takes a list of topics and queries the cluster to refresh the
  236. // available metadata for those topics.
  237. func (client *Client) RefreshTopicMetadata(topics ...string) error {
  238. return client.refreshMetadata(topics, client.config.MetadataRetries)
  239. }
  240. // RefreshAllMetadata queries the cluster to refresh the available metadata for all topics.
  241. func (client *Client) RefreshAllMetadata() error {
  242. // Kafka refreshes all when you encode it an empty array...
  243. return client.refreshMetadata(make([]string, 0), client.config.MetadataRetries)
  244. }
  245. // GetOffset queries the cluster to get the most recent available offset at the given
  246. // time on the topic/partition combination.
  247. func (client *Client) GetOffset(topic string, partitionID int32, where OffsetTime) (int64, error) {
  248. broker, err := client.Leader(topic, partitionID)
  249. if err != nil {
  250. return -1, err
  251. }
  252. request := &OffsetRequest{}
  253. request.AddBlock(topic, partitionID, where, 1)
  254. response, err := broker.GetAvailableOffsets(client.id, request)
  255. if err != nil {
  256. return -1, err
  257. }
  258. block := response.GetBlock(topic, partitionID)
  259. if block == nil {
  260. return -1, ErrIncompleteResponse
  261. }
  262. if block.Err != ErrNoError {
  263. return -1, block.Err
  264. }
  265. if len(block.Offsets) != 1 {
  266. return -1, ErrOffsetOutOfRange
  267. }
  268. return block.Offsets[0], nil
  269. }
  270. // private broker management helpers
  271. // XXX: see https://github.com/Shopify/sarama/issues/15
  272. // and https://github.com/Shopify/sarama/issues/23
  273. // disconnectBroker is a bad hacky way to accomplish broker management. It should be replaced with
  274. // something sane and the replacement should be made part of the public Client API
  275. func (client *Client) disconnectBroker(broker *Broker) {
  276. client.lock.Lock()
  277. defer client.lock.Unlock()
  278. Logger.Printf("Disconnecting Broker %d\n", broker.ID())
  279. client.deadBrokerAddrs[broker.addr] = none{}
  280. if broker == client.seedBroker {
  281. client.seedBrokerAddrs = client.seedBrokerAddrs[1:]
  282. if len(client.seedBrokerAddrs) > 0 {
  283. client.seedBroker = NewBroker(client.seedBrokerAddrs[0])
  284. _ = client.seedBroker.Open(client.config.DefaultBrokerConf)
  285. } else {
  286. client.seedBroker = nil
  287. }
  288. } else {
  289. // we don't need to update the leaders hash, it will automatically get refreshed next time because
  290. // the broker lookup will return nil
  291. delete(client.brokers, broker.ID())
  292. }
  293. safeAsyncClose(broker)
  294. }
  295. func (client *Client) resurrectDeadBrokers() {
  296. client.lock.Lock()
  297. defer client.lock.Unlock()
  298. for _, addr := range client.seedBrokerAddrs {
  299. client.deadBrokerAddrs[addr] = none{}
  300. }
  301. client.seedBrokerAddrs = []string{}
  302. for addr := range client.deadBrokerAddrs {
  303. client.seedBrokerAddrs = append(client.seedBrokerAddrs, addr)
  304. }
  305. client.deadBrokerAddrs = make(map[string]none)
  306. client.seedBroker = NewBroker(client.seedBrokerAddrs[0])
  307. _ = client.seedBroker.Open(client.config.DefaultBrokerConf)
  308. }
  309. func (client *Client) any() *Broker {
  310. client.lock.RLock()
  311. defer client.lock.RUnlock()
  312. if client.seedBroker != nil {
  313. return client.seedBroker
  314. }
  315. for _, broker := range client.brokers {
  316. return broker
  317. }
  318. return nil
  319. }
  320. // private caching/lazy metadata helpers
  321. type partitionType int
  322. const (
  323. allPartitions partitionType = iota
  324. writablePartitions
  325. // If you add any more types, update the partition cache in update()
  326. // Ensure this is the last partition type value
  327. maxPartitionIndex
  328. )
  329. func (client *Client) getMetadata(topic string, partitionID int32) (*PartitionMetadata, error) {
  330. metadata := client.cachedMetadata(topic, partitionID)
  331. if metadata == nil {
  332. err := client.RefreshTopicMetadata(topic)
  333. if err != nil {
  334. return nil, err
  335. }
  336. metadata = client.cachedMetadata(topic, partitionID)
  337. }
  338. if metadata == nil {
  339. return nil, ErrUnknownTopicOrPartition
  340. }
  341. return metadata, nil
  342. }
  343. func (client *Client) cachedMetadata(topic string, partitionID int32) *PartitionMetadata {
  344. client.lock.RLock()
  345. defer client.lock.RUnlock()
  346. partitions := client.metadata[topic]
  347. if partitions != nil {
  348. return partitions[partitionID]
  349. }
  350. return nil
  351. }
  352. func (client *Client) cachedPartitions(topic string, partitionSet partitionType) []int32 {
  353. client.lock.RLock()
  354. defer client.lock.RUnlock()
  355. partitions, exists := client.cachedPartitionsResults[topic]
  356. if !exists {
  357. return nil
  358. }
  359. return partitions[partitionSet]
  360. }
  361. func (client *Client) setPartitionCache(topic string, partitionSet partitionType) []int32 {
  362. partitions := client.metadata[topic]
  363. if partitions == nil {
  364. return nil
  365. }
  366. ret := make([]int32, 0, len(partitions))
  367. for _, partition := range partitions {
  368. if partitionSet == writablePartitions && partition.Err == ErrLeaderNotAvailable {
  369. continue
  370. }
  371. ret = append(ret, partition.ID)
  372. }
  373. sort.Sort(int32Slice(ret))
  374. return ret
  375. }
  376. func (client *Client) cachedLeader(topic string, partitionID int32) (*Broker, error) {
  377. client.lock.RLock()
  378. defer client.lock.RUnlock()
  379. partitions := client.metadata[topic]
  380. if partitions != nil {
  381. metadata, ok := partitions[partitionID]
  382. if ok {
  383. if metadata.Err == ErrLeaderNotAvailable {
  384. return nil, ErrLeaderNotAvailable
  385. }
  386. b := client.brokers[metadata.Leader]
  387. if b == nil {
  388. return nil, ErrLeaderNotAvailable
  389. }
  390. return b, nil
  391. }
  392. }
  393. return nil, ErrUnknownTopicOrPartition
  394. }
  395. // core metadata update logic
  396. func (client *Client) backgroundMetadataUpdater() {
  397. if client.config.BackgroundRefreshFrequency == time.Duration(0) {
  398. return
  399. }
  400. ticker := time.NewTicker(client.config.BackgroundRefreshFrequency)
  401. for {
  402. select {
  403. case <-ticker.C:
  404. if err := client.RefreshAllMetadata(); err != nil {
  405. Logger.Println("Client background metadata update:", err)
  406. }
  407. case <-client.closer:
  408. ticker.Stop()
  409. return
  410. }
  411. }
  412. }
  413. func (client *Client) refreshMetadata(topics []string, retriesRemaining int) error {
  414. // This function is a sort of central point for most functions that create new
  415. // resources. Check to see if we're dealing with a closed Client and error
  416. // out immediately if so.
  417. if client.Closed() {
  418. return ErrClosedClient
  419. }
  420. // Kafka will throw exceptions on an empty topic and not return a proper
  421. // error. This handles the case by returning an error instead of sending it
  422. // off to Kafka. See: https://github.com/Shopify/sarama/pull/38#issuecomment-26362310
  423. for _, topic := range topics {
  424. if len(topic) == 0 {
  425. return ErrUnknownTopicOrPartition
  426. }
  427. }
  428. for broker := client.any(); broker != nil; broker = client.any() {
  429. if len(topics) > 0 {
  430. Logger.Printf("Fetching metadata for %v from broker %s\n", topics, broker.addr)
  431. } else {
  432. Logger.Printf("Fetching metadata for all topics from broker %s\n", broker.addr)
  433. }
  434. response, err := broker.GetMetadata(client.id, &MetadataRequest{Topics: topics})
  435. switch err.(type) {
  436. case nil:
  437. // valid response, use it
  438. retry, err := client.update(response)
  439. if len(retry) > 0 {
  440. if retriesRemaining <= 0 {
  441. Logger.Println("Some partitions are leaderless, but we're out of retries")
  442. return nil
  443. }
  444. Logger.Printf("Some partitions are leaderless, waiting %dms for election... (%d retries remaining)\n", client.config.WaitForElection/time.Millisecond, retriesRemaining)
  445. time.Sleep(client.config.WaitForElection) // wait for leader election
  446. return client.refreshMetadata(retry, retriesRemaining-1)
  447. }
  448. return err
  449. case PacketEncodingError:
  450. // didn't even send, return the error
  451. return err
  452. default:
  453. // some other error, remove that broker and try again
  454. Logger.Println("Error from broker while fetching metadata:", err)
  455. client.disconnectBroker(broker)
  456. }
  457. }
  458. Logger.Println("Out of available brokers.")
  459. if retriesRemaining > 0 {
  460. Logger.Printf("Resurrecting dead brokers after %dms... (%d retries remaining)\n", client.config.WaitForElection/time.Millisecond, retriesRemaining)
  461. time.Sleep(client.config.WaitForElection)
  462. client.resurrectDeadBrokers()
  463. return client.refreshMetadata(topics, retriesRemaining-1)
  464. }
  465. return ErrOutOfBrokers
  466. }
  467. // if no fatal error, returns a list of topics that need retrying due to ErrLeaderNotAvailable
  468. func (client *Client) update(data *MetadataResponse) ([]string, error) {
  469. client.lock.Lock()
  470. defer client.lock.Unlock()
  471. // For all the brokers we received:
  472. // - if it is a new ID, save it
  473. // - if it is an existing ID, but the address we have is stale, discard the old one and save it
  474. // - otherwise ignore it, replacing our existing one would just bounce the connection
  475. // We asynchronously try to open connections to the new brokers. We don't care if they
  476. // fail, since maybe that broker is unreachable but doesn't have a topic we care about.
  477. // If it fails and we do care, whoever tries to use it will get the connection error.
  478. for _, broker := range data.Brokers {
  479. if client.brokers[broker.ID()] == nil {
  480. _ = broker.Open(client.config.DefaultBrokerConf)
  481. client.brokers[broker.ID()] = broker
  482. Logger.Printf("Registered new broker #%d at %s", broker.ID(), broker.Addr())
  483. } else if broker.Addr() != client.brokers[broker.ID()].Addr() {
  484. safeAsyncClose(client.brokers[broker.ID()])
  485. _ = broker.Open(client.config.DefaultBrokerConf)
  486. client.brokers[broker.ID()] = broker
  487. Logger.Printf("Replaced registered broker #%d with %s", broker.ID(), broker.Addr())
  488. }
  489. }
  490. toRetry := make(map[string]bool)
  491. var err error
  492. for _, topic := range data.Topics {
  493. switch topic.Err {
  494. case ErrNoError:
  495. break
  496. case ErrLeaderNotAvailable:
  497. toRetry[topic.Name] = true
  498. default:
  499. err = topic.Err
  500. }
  501. client.metadata[topic.Name] = make(map[int32]*PartitionMetadata, len(topic.Partitions))
  502. delete(client.cachedPartitionsResults, topic.Name)
  503. for _, partition := range topic.Partitions {
  504. client.metadata[topic.Name][partition.ID] = partition
  505. if partition.Err == ErrLeaderNotAvailable {
  506. toRetry[topic.Name] = true
  507. }
  508. }
  509. var partitionCache [maxPartitionIndex][]int32
  510. partitionCache[allPartitions] = client.setPartitionCache(topic.Name, allPartitions)
  511. partitionCache[writablePartitions] = client.setPartitionCache(topic.Name, writablePartitions)
  512. client.cachedPartitionsResults[topic.Name] = partitionCache
  513. }
  514. if err != nil {
  515. return nil, err
  516. }
  517. ret := make([]string, 0, len(toRetry))
  518. for topic := range toRetry {
  519. ret = append(ret, topic)
  520. }
  521. return ret, nil
  522. }