client.go 13 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432
  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. DefaultBrokerConf *BrokerConfig // Default configuration for broker connections created by this client.
  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 = NewClientConfig()
  37. }
  38. if err := config.Validate(); err != nil {
  39. return nil, err
  40. }
  41. if len(addrs) < 1 {
  42. return nil, ConfigurationError("You must provide at least one broker address")
  43. }
  44. client := &Client{
  45. id: id,
  46. config: *config,
  47. extraBrokerAddrs: addrs,
  48. extraBroker: NewBroker(addrs[0]),
  49. brokers: make(map[int32]*Broker),
  50. leaders: make(map[string]map[int32]int32),
  51. }
  52. client.extraBroker.Open(config.DefaultBrokerConf)
  53. // do an initial fetch of all cluster metadata by specifing an empty list of topics
  54. err := client.RefreshAllMetadata()
  55. if err != nil {
  56. client.Close()
  57. return nil, err
  58. }
  59. Logger.Println("Successfully initialized new client")
  60. return client, nil
  61. }
  62. // Close shuts down all broker connections managed by this client. It is required to call this function before
  63. // a client object passes out of scope, as it will otherwise leak memory. You must close any Producers or Consumers
  64. // using a client before you close the client.
  65. func (client *Client) Close() error {
  66. client.lock.Lock()
  67. defer client.lock.Unlock()
  68. Logger.Println("Closing Client")
  69. for _, broker := range client.brokers {
  70. myBroker := broker // NB: block-local prevents clobbering
  71. go withRecover(func() { myBroker.Close() })
  72. }
  73. client.brokers = nil
  74. client.leaders = nil
  75. if client.extraBroker != nil {
  76. go withRecover(func() { client.extraBroker.Close() })
  77. }
  78. return nil
  79. }
  80. // Partitions returns the sorted list of available partition IDs for the given topic.
  81. func (client *Client) Partitions(topic string) ([]int32, error) {
  82. partitions := client.cachedPartitions(topic)
  83. // len==0 catches when it's nil (no such topic) and the odd case when every single
  84. // partition is undergoing leader election simultaneously. Callers have to be able to handle
  85. // this function returning an empty slice (which is a valid return value) but catching it
  86. // here the first time (note we *don't* catch it below where we return NoSuchTopic) triggers
  87. // a metadata refresh as a nicety so callers can just try again and don't have to manually
  88. // trigger a refresh (otherwise they'd just keep getting a stale cached copy).
  89. if len(partitions) == 0 {
  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. // GetOffset queries the cluster to get the most recent available offset at the given
  138. // time on the topic/partition combination.
  139. func (client *Client) GetOffset(topic string, partitionID int32, where OffsetTime) (int64, error) {
  140. broker, err := client.Leader(topic, partitionID)
  141. if err != nil {
  142. return -1, err
  143. }
  144. request := &OffsetRequest{}
  145. request.AddBlock(topic, partitionID, where, 1)
  146. response, err := broker.GetAvailableOffsets(client.id, request)
  147. if err != nil {
  148. return -1, err
  149. }
  150. block := response.GetBlock(topic, partitionID)
  151. if block == nil {
  152. return -1, IncompleteResponse
  153. }
  154. if block.Err != NoError {
  155. return -1, block.Err
  156. }
  157. if len(block.Offsets) != 1 {
  158. return -1, IncompleteResponse
  159. }
  160. return block.Offsets[0], nil
  161. }
  162. // misc private helper functions
  163. // XXX: see https://github.com/Shopify/sarama/issues/15
  164. // and https://github.com/Shopify/sarama/issues/23
  165. // disconnectBroker is a bad hacky way to accomplish broker management. It should be replaced with
  166. // something sane and the replacement should be made part of the public Client API
  167. func (client *Client) disconnectBroker(broker *Broker) {
  168. client.lock.Lock()
  169. defer client.lock.Unlock()
  170. Logger.Printf("Disconnecting Broker %d\n", broker.ID())
  171. client.deadBrokerAddrs = append(client.deadBrokerAddrs, broker.addr)
  172. if broker == client.extraBroker {
  173. client.extraBrokerAddrs = client.extraBrokerAddrs[1:]
  174. if len(client.extraBrokerAddrs) > 0 {
  175. client.extraBroker = NewBroker(client.extraBrokerAddrs[0])
  176. client.extraBroker.Open(client.config.DefaultBrokerConf)
  177. } else {
  178. client.extraBroker = nil
  179. }
  180. } else {
  181. // we don't need to update the leaders hash, it will automatically get refreshed next time because
  182. // the broker lookup will return nil
  183. delete(client.brokers, broker.ID())
  184. }
  185. myBroker := broker // NB: block-local prevents clobbering
  186. go withRecover(func() { myBroker.Close() })
  187. }
  188. func (client *Client) refreshMetadata(topics []string, retries int) error {
  189. // Kafka will throw exceptions on an empty topic and not return a proper
  190. // error. This handles the case by returning an error instead of sending it
  191. // off to Kafka. See: https://github.com/Shopify/sarama/pull/38#issuecomment-26362310
  192. for _, topic := range topics {
  193. if len(topic) == 0 {
  194. return NoSuchTopic
  195. }
  196. }
  197. for broker := client.any(); broker != nil; broker = client.any() {
  198. Logger.Printf("Fetching metadata from broker %s\n", broker.addr)
  199. response, err := broker.GetMetadata(client.id, &MetadataRequest{Topics: topics})
  200. switch err {
  201. case nil:
  202. // valid response, use it
  203. retry, err := client.update(response)
  204. switch {
  205. case err != nil:
  206. return err
  207. case len(retry) == 0:
  208. return nil
  209. default:
  210. if retries <= 0 {
  211. return LeaderNotAvailable
  212. }
  213. Logger.Printf("Failed to fetch metadata from broker %s, waiting %dms... (%d retries remaining)\n", broker.addr, client.config.WaitForElection/time.Millisecond, retries)
  214. time.Sleep(client.config.WaitForElection) // wait for leader election
  215. return client.refreshMetadata(retry, retries-1)
  216. }
  217. case EncodingError:
  218. // didn't even send, return the error
  219. return err
  220. }
  221. // some other error, remove that broker and try again
  222. Logger.Println("Unexpected error from GetMetadata, closing broker:", err)
  223. client.disconnectBroker(broker)
  224. }
  225. if retries > 0 {
  226. Logger.Printf("Out of available brokers. Resurrecting dead brokers after %dms... (%d retries remaining)\n", client.config.WaitForElection/time.Millisecond, retries)
  227. time.Sleep(client.config.WaitForElection)
  228. client.resurrectDeadBrokers()
  229. return client.refreshMetadata(topics, retries-1)
  230. } else {
  231. Logger.Printf("Out of available brokers.\n")
  232. }
  233. return OutOfBrokers
  234. }
  235. func (client *Client) resurrectDeadBrokers() {
  236. client.lock.Lock()
  237. defer client.lock.Unlock()
  238. brokers := make(map[string]struct{})
  239. for _, addr := range client.deadBrokerAddrs {
  240. brokers[addr] = struct{}{}
  241. }
  242. for _, addr := range client.extraBrokerAddrs {
  243. brokers[addr] = struct{}{}
  244. }
  245. client.deadBrokerAddrs = []string{}
  246. client.extraBrokerAddrs = []string{}
  247. for addr := range brokers {
  248. client.extraBrokerAddrs = append(client.extraBrokerAddrs, addr)
  249. }
  250. client.extraBroker = NewBroker(client.extraBrokerAddrs[0])
  251. client.extraBroker.Open(client.config.DefaultBrokerConf)
  252. }
  253. func (client *Client) any() *Broker {
  254. client.lock.RLock()
  255. defer client.lock.RUnlock()
  256. for _, broker := range client.brokers {
  257. return broker
  258. }
  259. return client.extraBroker
  260. }
  261. func (client *Client) cachedLeader(topic string, partitionID int32) *Broker {
  262. client.lock.RLock()
  263. defer client.lock.RUnlock()
  264. partitions := client.leaders[topic]
  265. if partitions != nil {
  266. leader, ok := partitions[partitionID]
  267. if ok {
  268. return client.brokers[leader]
  269. }
  270. }
  271. return nil
  272. }
  273. func (client *Client) cachedPartitions(topic string) []int32 {
  274. client.lock.RLock()
  275. defer client.lock.RUnlock()
  276. partitions := client.leaders[topic]
  277. if partitions == nil {
  278. return nil
  279. }
  280. ret := make([]int32, 0, len(partitions))
  281. for id := range partitions {
  282. ret = append(ret, id)
  283. }
  284. sort.Sort(int32Slice(ret))
  285. return ret
  286. }
  287. // if no fatal error, returns a list of topics that need retrying due to LeaderNotAvailable
  288. func (client *Client) update(data *MetadataResponse) ([]string, error) {
  289. client.lock.Lock()
  290. defer client.lock.Unlock()
  291. // For all the brokers we received:
  292. // - if it is a new ID, save it
  293. // - if it is an existing ID, but the address we have is stale, discard the old one and save it
  294. // - otherwise ignore it, replacing our existing one would just bounce the connection
  295. // We asynchronously try to open connections to the new brokers. We don't care if they
  296. // fail, since maybe that broker is unreachable but doesn't have a topic we care about.
  297. // If it fails and we do care, whoever tries to use it will get the connection error.
  298. for _, broker := range data.Brokers {
  299. if client.brokers[broker.ID()] == nil {
  300. broker.Open(client.config.DefaultBrokerConf)
  301. client.brokers[broker.ID()] = broker
  302. Logger.Printf("Registered new broker #%d at %s", broker.ID(), broker.Addr())
  303. } else if broker.Addr() != client.brokers[broker.ID()].Addr() {
  304. myBroker := client.brokers[broker.ID()] // use block-local to prevent clobbering `broker` for Gs
  305. go withRecover(func() { myBroker.Close() })
  306. broker.Open(client.config.DefaultBrokerConf)
  307. client.brokers[broker.ID()] = broker
  308. Logger.Printf("Replaced registered broker #%d with %s", broker.ID(), broker.Addr())
  309. }
  310. }
  311. toRetry := make(map[string]bool)
  312. for _, topic := range data.Topics {
  313. switch topic.Err {
  314. case NoError:
  315. break
  316. case LeaderNotAvailable:
  317. toRetry[topic.Name] = true
  318. default:
  319. return nil, topic.Err
  320. }
  321. client.leaders[topic.Name] = make(map[int32]int32, len(topic.Partitions))
  322. for _, partition := range topic.Partitions {
  323. switch partition.Err {
  324. case LeaderNotAvailable:
  325. toRetry[topic.Name] = true
  326. delete(client.leaders[topic.Name], partition.ID)
  327. case NoError:
  328. client.leaders[topic.Name][partition.ID] = partition.Leader
  329. default:
  330. return nil, partition.Err
  331. }
  332. }
  333. }
  334. ret := make([]string, 0, len(toRetry))
  335. for topic := range toRetry {
  336. ret = append(ret, topic)
  337. }
  338. return ret, nil
  339. }
  340. // NewClientConfig creates a new ClientConfig instance with sensible defaults
  341. func NewClientConfig() *ClientConfig {
  342. return &ClientConfig{
  343. MetadataRetries: 3,
  344. WaitForElection: 250 * time.Millisecond,
  345. }
  346. }
  347. // Validate checks a ClientConfig instance. This will return a
  348. // ConfigurationError if the specified values don't make sense.
  349. func (config *ClientConfig) Validate() error {
  350. if config.MetadataRetries <= 0 {
  351. return ConfigurationError("Invalid MetadataRetries. Try 10")
  352. }
  353. if config.WaitForElection <= time.Duration(0) {
  354. return ConfigurationError("Invalid WaitForElection. Try 250*time.Millisecond")
  355. }
  356. if config.DefaultBrokerConf != nil {
  357. if err := config.DefaultBrokerConf.Validate(); err != nil {
  358. return err
  359. }
  360. }
  361. return nil
  362. }