client.go 7.6 KB

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