client.go 8.2 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285
  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. brokers map[int32]*Broker // maps broker ids to brokers
  20. leaders map[string]map[int32]int32 // maps topics to partition ids to broker ids
  21. lock sync.RWMutex // protects access to the maps, only one since they're always written together
  22. }
  23. // NewClient creates a new Client with the given client ID. It connects to the broker at the given
  24. // host:port address, and uses that broker to automatically fetch metadata on the rest of the kafka cluster.
  25. // If metadata cannot be retrieved (even if the connection otherwise succeeds) then the client is not created.
  26. func NewClient(id string, host string, port int32, config *ClientConfig) (client *Client, err error) {
  27. if config == nil {
  28. config = new(ClientConfig)
  29. }
  30. if config.MetadataRetries < 0 {
  31. return nil, ConfigurationError("Invalid MetadataRetries")
  32. }
  33. tmp := NewBroker(host, port)
  34. err = tmp.Open()
  35. if err != nil {
  36. return nil, err
  37. }
  38. _, err = tmp.Connected()
  39. if err != nil {
  40. return nil, err
  41. }
  42. client = new(Client)
  43. client.id = id
  44. client.config = *config
  45. client.brokers = make(map[int32]*Broker)
  46. client.leaders = make(map[string]map[int32]int32)
  47. // add it to the set so that refreshTopics can find it
  48. // brokers created through NewBroker() have an ID of -1, which won't conflict with
  49. // whatever the metadata request returns
  50. client.brokers[tmp.ID()] = tmp
  51. // do an initial fetch of all cluster metadata by specifing an empty list of topics
  52. err = client.refreshTopics(make([]string, 0), client.config.MetadataRetries)
  53. if err != nil {
  54. client.Close() // this closes tmp, since it's still in the brokers hash
  55. return nil, err
  56. }
  57. // So apparently a kafka broker is not required to return its own address in response
  58. // to a 'give me *all* the metadata request'... I'm not sure if that's because you're
  59. // assumed to have it already or what. Regardless, this means that we can't assume we can
  60. // disconnect our tmp broker here, since if it didn't return itself to us we want to keep
  61. // it around anyways. The worst that happens is we end up with two connections to the same
  62. // broker, one with ID -1 and one with the real ID.
  63. return client, nil
  64. }
  65. // Close shuts down all broker connections managed by this client. It is required to call this function before
  66. // a client object passes out of scope, as it will otherwise leak memory. You must close any Producers or Consumers
  67. // using a client before you close the client.
  68. func (client *Client) Close() error {
  69. client.lock.Lock()
  70. defer client.lock.Unlock()
  71. for _, broker := range client.brokers {
  72. go broker.Close()
  73. }
  74. client.brokers = nil
  75. client.leaders = nil
  76. return nil
  77. }
  78. // functions for use by producers and consumers
  79. // if Go had the concept they would be marked 'protected'
  80. func (client *Client) leader(topic string, partition_id int32) (*Broker, error) {
  81. leader := client.cachedLeader(topic, partition_id)
  82. if leader == nil {
  83. err := client.refreshTopic(topic)
  84. if err != nil {
  85. return nil, err
  86. }
  87. leader = client.cachedLeader(topic, partition_id)
  88. }
  89. if leader == nil {
  90. return nil, UNKNOWN_TOPIC_OR_PARTITION
  91. }
  92. return leader, nil
  93. }
  94. func (client *Client) partitions(topic string) ([]int32, error) {
  95. partitions := client.cachedPartitions(topic)
  96. if partitions == nil {
  97. err := client.refreshTopic(topic)
  98. if err != nil {
  99. return nil, err
  100. }
  101. partitions = client.cachedPartitions(topic)
  102. }
  103. if partitions == nil {
  104. return nil, NoSuchTopic
  105. }
  106. return partitions, nil
  107. }
  108. func (client *Client) disconnectBroker(broker *Broker) {
  109. client.lock.Lock()
  110. defer client.lock.Unlock()
  111. // we don't need to update the leaders hash, it will automatically get refreshed next time because
  112. // the broker lookup will return nil
  113. delete(client.brokers, broker.ID())
  114. go broker.Close()
  115. }
  116. func (client *Client) refreshTopic(topic string) error {
  117. tmp := make([]string, 1)
  118. tmp[0] = topic
  119. // we permit three retries by default, 'cause that seemed like a nice number
  120. return client.refreshTopics(tmp, client.config.MetadataRetries)
  121. }
  122. // truly private helper functions
  123. func (client *Client) refreshTopics(topics []string, retries int) error {
  124. for broker := client.any(); broker != nil; broker = client.any() {
  125. response, err := broker.GetMetadata(client.id, &MetadataRequest{Topics: topics})
  126. switch err {
  127. case nil:
  128. // valid response, use it
  129. retry, err := client.update(response)
  130. switch {
  131. case err != nil:
  132. return err
  133. case len(retry) == 0:
  134. return nil
  135. default:
  136. if retries <= 0 {
  137. return LEADER_NOT_AVAILABLE
  138. }
  139. time.Sleep(client.config.WaitForElection) // wait for leader election
  140. return client.refreshTopics(retry, retries-1)
  141. }
  142. case EncodingError:
  143. // didn't even send, return the error
  144. return err
  145. }
  146. // some other error, remove that broker and try again
  147. client.disconnectBroker(broker)
  148. }
  149. return OutOfBrokers
  150. }
  151. func (client *Client) any() *Broker {
  152. client.lock.RLock()
  153. defer client.lock.RUnlock()
  154. for _, broker := range client.brokers {
  155. return broker
  156. }
  157. return nil
  158. }
  159. func (client *Client) cachedLeader(topic string, partition_id int32) *Broker {
  160. client.lock.RLock()
  161. defer client.lock.RUnlock()
  162. partitions := client.leaders[topic]
  163. if partitions != nil {
  164. leader, ok := partitions[partition_id]
  165. if ok {
  166. return client.brokers[leader]
  167. }
  168. }
  169. return nil
  170. }
  171. func (client *Client) cachedPartitions(topic string) []int32 {
  172. client.lock.RLock()
  173. defer client.lock.RUnlock()
  174. partitions := client.leaders[topic]
  175. if partitions == nil {
  176. return nil
  177. }
  178. ret := make([]int32, 0, len(partitions))
  179. for id, _ := range partitions {
  180. ret = append(ret, id)
  181. }
  182. sort.Sort(int32Slice(ret))
  183. return ret
  184. }
  185. // if no fatal error, returns a list of topics that need retrying due to LEADER_NOT_AVAILABLE
  186. func (client *Client) update(data *MetadataResponse) ([]string, error) {
  187. client.lock.Lock()
  188. defer client.lock.Unlock()
  189. // First discard brokers that we already know about. This avoids bouncing TCP connections,
  190. // and especially avoids closing valid connections out from under other code which may be trying
  191. // to use them.
  192. var newBrokers []*Broker
  193. for _, broker := range data.Brokers {
  194. if !broker.Equals(client.brokers[broker.ID()]) {
  195. newBrokers = append(newBrokers, broker)
  196. }
  197. }
  198. // Now asynchronously try to open connections to the new brokers. We don't care if they
  199. // fail, since maybe that broker is unreachable but doesn't have a topic we care about.
  200. // If it fails and we do care, whoever tries to use it will get the connection error.
  201. // If we have an old broker with that ID (but a different host/port, since they didn't
  202. // compare as equals above) then close and remove that broker before saving the new one.
  203. for _, broker := range newBrokers {
  204. if client.brokers[broker.ID()] != nil {
  205. go client.brokers[broker.ID()].Close()
  206. }
  207. broker.Open()
  208. client.brokers[broker.ID()] = broker
  209. }
  210. toRetry := make(map[string]bool)
  211. for _, topic := range data.Topics {
  212. switch topic.Err {
  213. case NO_ERROR:
  214. break
  215. case LEADER_NOT_AVAILABLE:
  216. toRetry[topic.Name] = true
  217. default:
  218. return nil, topic.Err
  219. }
  220. client.leaders[topic.Name] = make(map[int32]int32, len(topic.Partitions))
  221. for _, partition := range topic.Partitions {
  222. switch partition.Err {
  223. case LEADER_NOT_AVAILABLE:
  224. toRetry[topic.Name] = true
  225. delete(client.leaders[topic.Name], partition.Id)
  226. case NO_ERROR:
  227. client.leaders[topic.Name][partition.Id] = partition.Leader
  228. default:
  229. return nil, partition.Err
  230. }
  231. }
  232. }
  233. ret := make([]string, 0, len(toRetry))
  234. for topic, _ := range toRetry {
  235. ret = append(ret, topic)
  236. }
  237. return ret, nil
  238. }