client.go 15 KB

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