client.go 9.9 KB

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