client.go 7.7 KB

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