client.go 8.2 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293
  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.refreshTopics(make([]string, 0), client.config.MetadataRetries)
  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. // functions for use by producers and consumers
  74. // if Go had the concept they would be marked 'protected'
  75. func (client *Client) leader(topic string, partition_id int32) (*Broker, error) {
  76. leader := client.cachedLeader(topic, partition_id)
  77. if leader == nil {
  78. err := client.refreshTopic(topic)
  79. if err != nil {
  80. return nil, err
  81. }
  82. leader = client.cachedLeader(topic, partition_id)
  83. }
  84. if leader == nil {
  85. return nil, UNKNOWN_TOPIC_OR_PARTITION
  86. }
  87. return leader, nil
  88. }
  89. func (client *Client) partitions(topic string) ([]int32, error) {
  90. partitions := client.cachedPartitions(topic)
  91. if partitions == nil {
  92. err := client.refreshTopic(topic)
  93. if err != nil {
  94. return nil, err
  95. }
  96. partitions = client.cachedPartitions(topic)
  97. }
  98. if partitions == nil {
  99. return nil, NoSuchTopic
  100. }
  101. return partitions, nil
  102. }
  103. func (client *Client) disconnectBroker(broker *Broker) {
  104. client.lock.Lock()
  105. defer client.lock.Unlock()
  106. if broker == client.extraBroker {
  107. client.extraBrokerAddrs = client.extraBrokerAddrs[1:]
  108. if len(client.extraBrokerAddrs) > 0 {
  109. client.extraBroker = NewBroker(client.extraBrokerAddrs[0])
  110. client.extraBroker.Open()
  111. } else {
  112. client.extraBroker = nil
  113. }
  114. } else {
  115. // we don't need to update the leaders hash, it will automatically get refreshed next time because
  116. // the broker lookup will return nil
  117. delete(client.brokers, broker.ID())
  118. }
  119. go broker.Close()
  120. }
  121. func (client *Client) refreshTopic(topic string) error {
  122. tmp := make([]string, 1)
  123. tmp[0] = topic
  124. // we permit three retries by default, 'cause that seemed like a nice number
  125. return client.refreshTopics(tmp, client.config.MetadataRetries)
  126. }
  127. // truly private helper functions
  128. func (client *Client) refreshTopics(topics []string, retries int) error {
  129. for broker := client.any(); broker != nil; broker = client.any() {
  130. response, err := broker.GetMetadata(client.id, &MetadataRequest{Topics: topics})
  131. switch err {
  132. case nil:
  133. // valid response, use it
  134. retry, err := client.update(response)
  135. switch {
  136. case err != nil:
  137. return err
  138. case len(retry) == 0:
  139. return nil
  140. default:
  141. if retries <= 0 {
  142. return LEADER_NOT_AVAILABLE
  143. }
  144. time.Sleep(client.config.WaitForElection) // wait for leader election
  145. return client.refreshTopics(retry, retries-1)
  146. }
  147. case EncodingError:
  148. // didn't even send, return the error
  149. return err
  150. }
  151. // some other error, remove that broker and try again
  152. client.disconnectBroker(broker)
  153. }
  154. return OutOfBrokers
  155. }
  156. func (client *Client) any() *Broker {
  157. client.lock.RLock()
  158. defer client.lock.RUnlock()
  159. for _, broker := range client.brokers {
  160. return broker
  161. }
  162. return client.extraBroker
  163. }
  164. func (client *Client) cachedLeader(topic string, partition_id int32) *Broker {
  165. client.lock.RLock()
  166. defer client.lock.RUnlock()
  167. partitions := client.leaders[topic]
  168. if partitions != nil {
  169. leader, ok := partitions[partition_id]
  170. if ok {
  171. return client.brokers[leader]
  172. }
  173. }
  174. return nil
  175. }
  176. func (client *Client) cachedPartitions(topic string) []int32 {
  177. client.lock.RLock()
  178. defer client.lock.RUnlock()
  179. partitions := client.leaders[topic]
  180. if partitions == nil {
  181. return nil
  182. }
  183. ret := make([]int32, 0, len(partitions))
  184. for id, _ := range partitions {
  185. ret = append(ret, id)
  186. }
  187. sort.Sort(int32Slice(ret))
  188. return ret
  189. }
  190. // if no fatal error, returns a list of topics that need retrying due to LEADER_NOT_AVAILABLE
  191. func (client *Client) update(data *MetadataResponse) ([]string, error) {
  192. client.lock.Lock()
  193. defer client.lock.Unlock()
  194. // First discard brokers that we already know about. This avoids bouncing TCP connections,
  195. // and especially avoids closing valid connections out from under other code which may be trying
  196. // to use them.
  197. var newBrokers []*Broker
  198. for _, broker := range data.Brokers {
  199. if !broker.Equals(client.brokers[broker.ID()]) {
  200. newBrokers = append(newBrokers, broker)
  201. }
  202. }
  203. // Now asynchronously try to open connections to the new brokers. We don't care if they
  204. // fail, since maybe that broker is unreachable but doesn't have a topic we care about.
  205. // If it fails and we do care, whoever tries to use it will get the connection error.
  206. // If we have an old broker with that ID (but a different host/port, since they didn't
  207. // compare as equals above) then close and remove that broker before saving the new one.
  208. for _, broker := range newBrokers {
  209. if client.brokers[broker.ID()] != nil {
  210. go client.brokers[broker.ID()].Close()
  211. }
  212. broker.Open()
  213. client.brokers[broker.ID()] = broker
  214. }
  215. toRetry := make(map[string]bool)
  216. for _, topic := range data.Topics {
  217. switch topic.Err {
  218. case NO_ERROR:
  219. break
  220. case LEADER_NOT_AVAILABLE:
  221. toRetry[topic.Name] = true
  222. default:
  223. return nil, topic.Err
  224. }
  225. client.leaders[topic.Name] = make(map[int32]int32, len(topic.Partitions))
  226. for _, partition := range topic.Partitions {
  227. switch partition.Err {
  228. case LEADER_NOT_AVAILABLE:
  229. toRetry[topic.Name] = true
  230. delete(client.leaders[topic.Name], partition.Id)
  231. case NO_ERROR:
  232. client.leaders[topic.Name][partition.Id] = partition.Leader
  233. default:
  234. return nil, partition.Err
  235. }
  236. }
  237. }
  238. ret := make([]string, 0, len(toRetry))
  239. for topic, _ := range toRetry {
  240. ret = append(ret, topic)
  241. }
  242. return ret, nil
  243. }