client.go 15 KB

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