client.go 11 KB

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