client.go 15 KB

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