client.go 14 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461
  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. // Check to see whether the client is closed
  67. if client.Closed() {
  68. // Chances are this is being called from a defer() and the error will go unobserved
  69. // so we go ahead and log the event in this case.
  70. Logger.Printf("Close() called on already closed client")
  71. return ClosedClient
  72. }
  73. client.lock.Lock()
  74. defer client.lock.Unlock()
  75. Logger.Println("Closing Client")
  76. for _, broker := range client.brokers {
  77. myBroker := broker // NB: block-local prevents clobbering
  78. go withRecover(func() { myBroker.Close() })
  79. }
  80. client.brokers = nil
  81. client.leaders = nil
  82. if client.extraBroker != nil {
  83. go withRecover(func() { client.extraBroker.Close() })
  84. }
  85. return nil
  86. }
  87. // Partitions returns the sorted list of available partition IDs for the given topic.
  88. func (client *Client) Partitions(topic string) ([]int32, error) {
  89. // Check to see whether the client is closed
  90. if client.Closed() {
  91. return nil, ClosedClient
  92. }
  93. partitions := client.cachedPartitions(topic)
  94. // len==0 catches when it's nil (no such topic) and the odd case when every single
  95. // partition is undergoing leader election simultaneously. Callers have to be able to handle
  96. // this function returning an empty slice (which is a valid return value) but catching it
  97. // here the first time (note we *don't* catch it below where we return NoSuchTopic) triggers
  98. // a metadata refresh as a nicety so callers can just try again and don't have to manually
  99. // trigger a refresh (otherwise they'd just keep getting a stale cached copy).
  100. if len(partitions) == 0 {
  101. err := client.RefreshTopicMetadata(topic)
  102. if err != nil {
  103. return nil, err
  104. }
  105. partitions = client.cachedPartitions(topic)
  106. }
  107. if partitions == nil {
  108. return nil, NoSuchTopic
  109. }
  110. return partitions, nil
  111. }
  112. // Topics returns the set of available topics as retrieved from the cluster metadata.
  113. func (client *Client) Topics() ([]string, error) {
  114. // Check to see whether the client is closed
  115. if client.Closed() {
  116. return nil, ClosedClient
  117. }
  118. client.lock.RLock()
  119. defer client.lock.RUnlock()
  120. ret := make([]string, 0, len(client.leaders))
  121. for topic := range client.leaders {
  122. ret = append(ret, topic)
  123. }
  124. return ret, nil
  125. }
  126. // Leader returns the broker object that is the leader of the current topic/partition, as
  127. // determined by querying the cluster metadata.
  128. func (client *Client) Leader(topic string, partitionID int32) (*Broker, error) {
  129. leader := client.cachedLeader(topic, partitionID)
  130. if leader == nil {
  131. err := client.RefreshTopicMetadata(topic)
  132. if err != nil {
  133. return nil, err
  134. }
  135. leader = client.cachedLeader(topic, partitionID)
  136. }
  137. if leader == nil {
  138. return nil, UnknownTopicOrPartition
  139. }
  140. return leader, nil
  141. }
  142. // RefreshTopicMetadata takes a list of topics and queries the cluster to refresh the
  143. // available metadata for those topics.
  144. func (client *Client) RefreshTopicMetadata(topics ...string) error {
  145. return client.refreshMetadata(topics, client.config.MetadataRetries)
  146. }
  147. // RefreshAllMetadata queries the cluster to refresh the available metadata for all topics.
  148. func (client *Client) RefreshAllMetadata() error {
  149. // Kafka refreshes all when you encode it an empty array...
  150. return client.refreshMetadata(make([]string, 0), client.config.MetadataRetries)
  151. }
  152. // GetOffset queries the cluster to get the most recent available offset at the given
  153. // time on the topic/partition combination.
  154. func (client *Client) GetOffset(topic string, partitionID int32, where OffsetTime) (int64, error) {
  155. broker, err := client.Leader(topic, partitionID)
  156. if err != nil {
  157. return -1, err
  158. }
  159. request := &OffsetRequest{}
  160. request.AddBlock(topic, partitionID, where, 1)
  161. response, err := broker.GetAvailableOffsets(client.id, request)
  162. if err != nil {
  163. return -1, err
  164. }
  165. block := response.GetBlock(topic, partitionID)
  166. if block == nil {
  167. return -1, IncompleteResponse
  168. }
  169. if block.Err != NoError {
  170. return -1, block.Err
  171. }
  172. if len(block.Offsets) != 1 {
  173. return -1, IncompleteResponse
  174. }
  175. return block.Offsets[0], nil
  176. }
  177. // misc private helper functions
  178. // XXX: see https://github.com/Shopify/sarama/issues/15
  179. // and https://github.com/Shopify/sarama/issues/23
  180. // disconnectBroker is a bad hacky way to accomplish broker management. It should be replaced with
  181. // something sane and the replacement should be made part of the public Client API
  182. func (client *Client) disconnectBroker(broker *Broker) {
  183. client.lock.Lock()
  184. defer client.lock.Unlock()
  185. Logger.Printf("Disconnecting Broker %d\n", broker.ID())
  186. client.deadBrokerAddrs = append(client.deadBrokerAddrs, broker.addr)
  187. if broker == client.extraBroker {
  188. client.extraBrokerAddrs = client.extraBrokerAddrs[1:]
  189. if len(client.extraBrokerAddrs) > 0 {
  190. client.extraBroker = NewBroker(client.extraBrokerAddrs[0])
  191. client.extraBroker.Open(client.config.DefaultBrokerConf)
  192. } else {
  193. client.extraBroker = nil
  194. }
  195. } else {
  196. // we don't need to update the leaders hash, it will automatically get refreshed next time because
  197. // the broker lookup will return nil
  198. delete(client.brokers, broker.ID())
  199. }
  200. myBroker := broker // NB: block-local prevents clobbering
  201. go withRecover(func() { myBroker.Close() })
  202. }
  203. func (client *Client) Closed() bool {
  204. return client.brokers == nil
  205. }
  206. func (client *Client) refreshMetadata(topics []string, retries int) error {
  207. // This function is a sort of central point for most functions that create new
  208. // resources. Check to see if we're dealing with a closed Client and error
  209. // out immediately if so.
  210. if client.Closed() {
  211. return ClosedClient
  212. }
  213. // Kafka will throw exceptions on an empty topic and not return a proper
  214. // error. This handles the case by returning an error instead of sending it
  215. // off to Kafka. See: https://github.com/Shopify/sarama/pull/38#issuecomment-26362310
  216. for _, topic := range topics {
  217. if len(topic) == 0 {
  218. return NoSuchTopic
  219. }
  220. }
  221. for broker := client.any(); broker != nil; broker = client.any() {
  222. Logger.Printf("Fetching metadata from broker %s\n", broker.addr)
  223. response, err := broker.GetMetadata(client.id, &MetadataRequest{Topics: topics})
  224. switch err {
  225. case nil:
  226. // valid response, use it
  227. retry, err := client.update(response)
  228. switch {
  229. case err != nil:
  230. return err
  231. case len(retry) == 0:
  232. return nil
  233. default:
  234. if retries <= 0 {
  235. return LeaderNotAvailable
  236. }
  237. Logger.Printf("Failed to fetch metadata from broker %s, waiting %dms... (%d retries remaining)\n", broker.addr, client.config.WaitForElection/time.Millisecond, retries)
  238. time.Sleep(client.config.WaitForElection) // wait for leader election
  239. return client.refreshMetadata(retry, retries-1)
  240. }
  241. case EncodingError:
  242. // didn't even send, return the error
  243. return err
  244. }
  245. // some other error, remove that broker and try again
  246. Logger.Println("Unexpected error from GetMetadata, closing broker:", err)
  247. client.disconnectBroker(broker)
  248. }
  249. if retries > 0 {
  250. Logger.Printf("Out of available brokers. Resurrecting dead brokers after %dms... (%d retries remaining)\n", client.config.WaitForElection/time.Millisecond, retries)
  251. time.Sleep(client.config.WaitForElection)
  252. client.resurrectDeadBrokers()
  253. return client.refreshMetadata(topics, retries-1)
  254. } else {
  255. Logger.Printf("Out of available brokers.\n")
  256. }
  257. return OutOfBrokers
  258. }
  259. func (client *Client) resurrectDeadBrokers() {
  260. client.lock.Lock()
  261. defer client.lock.Unlock()
  262. brokers := make(map[string]struct{})
  263. for _, addr := range client.deadBrokerAddrs {
  264. brokers[addr] = struct{}{}
  265. }
  266. for _, addr := range client.extraBrokerAddrs {
  267. brokers[addr] = struct{}{}
  268. }
  269. client.deadBrokerAddrs = []string{}
  270. client.extraBrokerAddrs = []string{}
  271. for addr := range brokers {
  272. client.extraBrokerAddrs = append(client.extraBrokerAddrs, addr)
  273. }
  274. client.extraBroker = NewBroker(client.extraBrokerAddrs[0])
  275. client.extraBroker.Open(client.config.DefaultBrokerConf)
  276. }
  277. func (client *Client) any() *Broker {
  278. client.lock.RLock()
  279. defer client.lock.RUnlock()
  280. for _, broker := range client.brokers {
  281. return broker
  282. }
  283. return client.extraBroker
  284. }
  285. func (client *Client) cachedLeader(topic string, partitionID int32) *Broker {
  286. client.lock.RLock()
  287. defer client.lock.RUnlock()
  288. partitions := client.leaders[topic]
  289. if partitions != nil {
  290. leader, ok := partitions[partitionID]
  291. if ok {
  292. return client.brokers[leader]
  293. }
  294. }
  295. return nil
  296. }
  297. func (client *Client) cachedPartitions(topic string) []int32 {
  298. client.lock.RLock()
  299. defer client.lock.RUnlock()
  300. partitions := client.leaders[topic]
  301. if partitions == nil {
  302. return nil
  303. }
  304. ret := make([]int32, 0, len(partitions))
  305. for id := range partitions {
  306. ret = append(ret, id)
  307. }
  308. sort.Sort(int32Slice(ret))
  309. return ret
  310. }
  311. // if no fatal error, returns a list of topics that need retrying due to LeaderNotAvailable
  312. func (client *Client) update(data *MetadataResponse) ([]string, error) {
  313. client.lock.Lock()
  314. defer client.lock.Unlock()
  315. // For all the brokers we received:
  316. // - if it is a new ID, save it
  317. // - if it is an existing ID, but the address we have is stale, discard the old one and save it
  318. // - otherwise ignore it, replacing our existing one would just bounce the connection
  319. // We asynchronously try to open connections to the new brokers. We don't care if they
  320. // fail, since maybe that broker is unreachable but doesn't have a topic we care about.
  321. // If it fails and we do care, whoever tries to use it will get the connection error.
  322. for _, broker := range data.Brokers {
  323. if client.brokers[broker.ID()] == nil {
  324. broker.Open(client.config.DefaultBrokerConf)
  325. client.brokers[broker.ID()] = broker
  326. Logger.Printf("Registered new broker #%d at %s", broker.ID(), broker.Addr())
  327. } else if broker.Addr() != client.brokers[broker.ID()].Addr() {
  328. myBroker := client.brokers[broker.ID()] // use block-local to prevent clobbering `broker` for Gs
  329. go withRecover(func() { myBroker.Close() })
  330. broker.Open(client.config.DefaultBrokerConf)
  331. client.brokers[broker.ID()] = broker
  332. Logger.Printf("Replaced registered broker #%d with %s", broker.ID(), broker.Addr())
  333. }
  334. }
  335. toRetry := make(map[string]bool)
  336. for _, topic := range data.Topics {
  337. switch topic.Err {
  338. case NoError:
  339. break
  340. case LeaderNotAvailable:
  341. toRetry[topic.Name] = true
  342. default:
  343. return nil, topic.Err
  344. }
  345. client.leaders[topic.Name] = make(map[int32]int32, len(topic.Partitions))
  346. for _, partition := range topic.Partitions {
  347. switch partition.Err {
  348. case LeaderNotAvailable:
  349. toRetry[topic.Name] = true
  350. delete(client.leaders[topic.Name], partition.ID)
  351. case NoError:
  352. client.leaders[topic.Name][partition.ID] = partition.Leader
  353. default:
  354. return nil, partition.Err
  355. }
  356. }
  357. }
  358. ret := make([]string, 0, len(toRetry))
  359. for topic := range toRetry {
  360. ret = append(ret, topic)
  361. }
  362. return ret, nil
  363. }
  364. // NewClientConfig creates a new ClientConfig instance with sensible defaults
  365. func NewClientConfig() *ClientConfig {
  366. return &ClientConfig{
  367. MetadataRetries: 3,
  368. WaitForElection: 250 * time.Millisecond,
  369. }
  370. }
  371. // Validate checks a ClientConfig instance. This will return a
  372. // ConfigurationError if the specified values don't make sense.
  373. func (config *ClientConfig) Validate() error {
  374. if config.MetadataRetries <= 0 {
  375. return ConfigurationError("Invalid MetadataRetries. Try 10")
  376. }
  377. if config.WaitForElection <= time.Duration(0) {
  378. return ConfigurationError("Invalid WaitForElection. Try 250*time.Millisecond")
  379. }
  380. if config.DefaultBrokerConf != nil {
  381. if err := config.DefaultBrokerConf.Validate(); err != nil {
  382. return err
  383. }
  384. }
  385. return nil
  386. }