client.go 9.6 KB

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