client.go 11 KB

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