client.go 15 KB

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