client.go 9.0 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310
  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. }
  12. // Client is a generic Kafka client. It manages connections to one or more Kafka brokers.
  13. // You MUST call Close() on a client to avoid leaks, it will not be garbage-collected
  14. // automatically when it passes out of scope. A single client can be safely shared by
  15. // multiple concurrent Producers and Consumers.
  16. type Client struct {
  17. id string
  18. config ClientConfig
  19. // the broker addresses given to us through the constructor are not guaranteed to be returned in
  20. // the cluster metadata (I *think* it only returns brokers who are currently leading partitions?)
  21. // so we store them separately
  22. extraBrokerAddrs []string
  23. extraBroker *Broker
  24. brokers map[int32]*Broker // maps broker ids to brokers
  25. leaders map[string]map[int32]int32 // maps topics to partition ids to broker ids
  26. lock sync.RWMutex // protects access to the maps, only one since they're always written together
  27. }
  28. // NewClient creates a new Client with the given client ID. It connects to one of the given broker addresses
  29. // and uses that broker to automatically fetch metadata on the rest of the kafka cluster. If metadata cannot
  30. // be retrieved from any of the given broker addresses, the client is not created.
  31. func NewClient(id string, addrs []string, config *ClientConfig) (*Client, error) {
  32. if config == nil {
  33. config = new(ClientConfig)
  34. }
  35. if config.MetadataRetries < 0 {
  36. return nil, ConfigurationError("Invalid MetadataRetries")
  37. }
  38. if len(addrs) < 1 {
  39. return nil, ConfigurationError("You must provide at least one broker address")
  40. }
  41. client := &Client{
  42. id: id,
  43. config: *config,
  44. extraBrokerAddrs: addrs,
  45. extraBroker: NewBroker(addrs[0]),
  46. brokers: make(map[int32]*Broker),
  47. leaders: make(map[string]map[int32]int32),
  48. }
  49. client.extraBroker.Open()
  50. // do an initial fetch of all cluster metadata by specifing an empty list of topics
  51. err := client.RefreshAllMetadata()
  52. if err != nil {
  53. client.Close() // this closes tmp, since it's still in the brokers hash
  54. return nil, err
  55. }
  56. return client, nil
  57. }
  58. // Close shuts down all broker connections managed by this client. It is required to call this function before
  59. // a client object passes out of scope, as it will otherwise leak memory. You must close any Producers or Consumers
  60. // using a client before you close the client.
  61. func (client *Client) Close() error {
  62. client.lock.Lock()
  63. defer client.lock.Unlock()
  64. for _, broker := range client.brokers {
  65. go broker.Close()
  66. }
  67. client.brokers = nil
  68. client.leaders = nil
  69. if client.extraBroker != nil {
  70. go client.extraBroker.Close()
  71. }
  72. return nil
  73. }
  74. // Partitions returns the sorted list of available partition IDs for the given topic.
  75. func (client *Client) Partitions(topic string) ([]int32, error) {
  76. partitions := client.cachedPartitions(topic)
  77. if partitions == nil {
  78. err := client.RefreshTopicMetadata(topic)
  79. if err != nil {
  80. return nil, err
  81. }
  82. partitions = client.cachedPartitions(topic)
  83. }
  84. if partitions == nil {
  85. return nil, NoSuchTopic
  86. }
  87. return partitions, nil
  88. }
  89. // Topics returns the set of available topics as retrieved from the cluster metadata.
  90. func (client *Client) Topics() ([]string, error) {
  91. client.lock.RLock()
  92. defer client.lock.RUnlock()
  93. ret := make([]string, 0, len(client.leaders))
  94. for topic := range client.leaders {
  95. ret = append(ret, topic)
  96. }
  97. return ret, nil
  98. }
  99. // Leader returns the broker object that is the leader of the current topic/partition, as
  100. // determined by querying the cluster metadata.
  101. func (client *Client) Leader(topic string, partitionID int32) (*Broker, error) {
  102. leader := client.cachedLeader(topic, partitionID)
  103. if leader == nil {
  104. err := client.RefreshTopicMetadata(topic)
  105. if err != nil {
  106. return nil, err
  107. }
  108. leader = client.cachedLeader(topic, partitionID)
  109. }
  110. if leader == nil {
  111. return nil, UnknownTopicOrPartition
  112. }
  113. return leader, nil
  114. }
  115. // RefreshTopicMetadata takes a list of topics and queries the cluster to refresh the
  116. // available metadata for those topics.
  117. func (client *Client) RefreshTopicMetadata(topics ...string) error {
  118. return client.refreshMetadata(topics, client.config.MetadataRetries)
  119. }
  120. // RefreshAllMetadata queries the cluster to refresh the available metadata for all topics.
  121. func (client *Client) RefreshAllMetadata() error {
  122. // Kafka refreshes all when you encode it an empty array...
  123. return client.refreshMetadata(make([]string, 0), client.config.MetadataRetries)
  124. }
  125. // misc private helper functions
  126. // XXX: see https://github.com/Shopify/sarama/issues/15
  127. // and https://github.com/Shopify/sarama/issues/23
  128. // disconnectBroker is a bad hacky way to accomplish broker management. It should be replaced with
  129. // something sane and the replacement should be made part of the public Client API
  130. func (client *Client) disconnectBroker(broker *Broker) {
  131. client.lock.Lock()
  132. defer client.lock.Unlock()
  133. if broker == client.extraBroker {
  134. client.extraBrokerAddrs = client.extraBrokerAddrs[1:]
  135. if len(client.extraBrokerAddrs) > 0 {
  136. client.extraBroker = NewBroker(client.extraBrokerAddrs[0])
  137. client.extraBroker.Open()
  138. } else {
  139. client.extraBroker = nil
  140. }
  141. } else {
  142. // we don't need to update the leaders hash, it will automatically get refreshed next time because
  143. // the broker lookup will return nil
  144. delete(client.brokers, broker.ID())
  145. }
  146. go broker.Close()
  147. }
  148. func (client *Client) refreshMetadata(topics []string, retries int) error {
  149. for broker := client.any(); broker != nil; broker = client.any() {
  150. response, err := broker.GetMetadata(client.id, &MetadataRequest{Topics: topics})
  151. switch err {
  152. case nil:
  153. // valid response, use it
  154. retry, err := client.update(response)
  155. switch {
  156. case err != nil:
  157. return err
  158. case len(retry) == 0:
  159. return nil
  160. default:
  161. if retries <= 0 {
  162. return LeaderNotAvailable
  163. }
  164. time.Sleep(client.config.WaitForElection) // wait for leader election
  165. return client.refreshMetadata(retry, retries-1)
  166. }
  167. case EncodingError:
  168. // didn't even send, return the error
  169. return err
  170. }
  171. // some other error, remove that broker and try again
  172. client.disconnectBroker(broker)
  173. }
  174. return OutOfBrokers
  175. }
  176. func (client *Client) any() *Broker {
  177. client.lock.RLock()
  178. defer client.lock.RUnlock()
  179. for _, broker := range client.brokers {
  180. return broker
  181. }
  182. return client.extraBroker
  183. }
  184. func (client *Client) cachedLeader(topic string, partitionID int32) *Broker {
  185. client.lock.RLock()
  186. defer client.lock.RUnlock()
  187. partitions := client.leaders[topic]
  188. if partitions != nil {
  189. leader, ok := partitions[partitionID]
  190. if ok {
  191. return client.brokers[leader]
  192. }
  193. }
  194. return nil
  195. }
  196. func (client *Client) cachedPartitions(topic string) []int32 {
  197. client.lock.RLock()
  198. defer client.lock.RUnlock()
  199. partitions := client.leaders[topic]
  200. if partitions == nil {
  201. return nil
  202. }
  203. ret := make([]int32, 0, len(partitions))
  204. for id := range partitions {
  205. ret = append(ret, id)
  206. }
  207. sort.Sort(int32Slice(ret))
  208. return ret
  209. }
  210. // if no fatal error, returns a list of topics that need retrying due to LeaderNotAvailable
  211. func (client *Client) update(data *MetadataResponse) ([]string, error) {
  212. client.lock.Lock()
  213. defer client.lock.Unlock()
  214. // For all the brokers we received:
  215. // - if it is a new ID, save it
  216. // - if it is an existing ID, but the address we have is stale, discard the old one and save it
  217. // - otherwise ignore it, replacing our existing one would just bounce the connection
  218. // We asynchronously try to open connections to the new brokers. We don't care if they
  219. // fail, since maybe that broker is unreachable but doesn't have a topic we care about.
  220. // If it fails and we do care, whoever tries to use it will get the connection error.
  221. for _, broker := range data.Brokers {
  222. if client.brokers[broker.ID()] == nil {
  223. broker.Open()
  224. client.brokers[broker.ID()] = broker
  225. } else if broker.Addr() != client.brokers[broker.ID()].Addr() {
  226. go client.brokers[broker.ID()].Close()
  227. broker.Open()
  228. client.brokers[broker.ID()] = broker
  229. }
  230. }
  231. toRetry := make(map[string]bool)
  232. for _, topic := range data.Topics {
  233. switch topic.Err {
  234. case NoError:
  235. break
  236. case LeaderNotAvailable:
  237. toRetry[topic.Name] = true
  238. default:
  239. return nil, topic.Err
  240. }
  241. client.leaders[topic.Name] = make(map[int32]int32, len(topic.Partitions))
  242. for _, partition := range topic.Partitions {
  243. switch partition.Err {
  244. case LeaderNotAvailable:
  245. toRetry[topic.Name] = true
  246. delete(client.leaders[topic.Name], partition.ID)
  247. case NoError:
  248. client.leaders[topic.Name][partition.ID] = partition.Leader
  249. default:
  250. return nil, partition.Err
  251. }
  252. }
  253. }
  254. ret := make([]string, 0, len(toRetry))
  255. for topic := range toRetry {
  256. ret = append(ret, topic)
  257. }
  258. return ret, nil
  259. }