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