client.go 16 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554
  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. metadata map[string]map[int32]*PartitionMetadata // maps topics to partition ids to metadata
  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. metadata: make(map[string]map[int32]*PartitionMetadata),
  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.metadata = 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.metadata))
  132. for topic := range client.metadata {
  133. ret = append(ret, topic)
  134. }
  135. return ret, nil
  136. }
  137. func (client *Client) getMetadata(topic string, partitionID int32) (*PartitionMetadata, error) {
  138. metadata := client.cachedMetadata(topic, partitionID)
  139. if metadata == nil {
  140. err := client.RefreshTopicMetadata(topic)
  141. if err != nil {
  142. return nil, err
  143. }
  144. metadata = client.cachedMetadata(topic, partitionID)
  145. }
  146. if metadata == nil {
  147. return nil, UnknownTopicOrPartition
  148. }
  149. return metadata, nil
  150. }
  151. func (client *Client) Replicas(topic string, partitionID int32) ([]int32, error) {
  152. metadata, err := client.getMetadata(topic, partitionID)
  153. if err != nil {
  154. return nil, err
  155. }
  156. return dupeAndSort(metadata.Replicas), nil
  157. }
  158. func (client *Client) ReplicasInSync(topic string, partitionID int32) ([]int32, error) {
  159. metadata, err := client.getMetadata(topic, partitionID)
  160. if err != nil {
  161. return nil, err
  162. }
  163. return dupeAndSort(metadata.Isr), nil
  164. }
  165. // Leader returns the broker object that is the leader of the current topic/partition, as
  166. // determined by querying the cluster metadata.
  167. func (client *Client) Leader(topic string, partitionID int32) (*Broker, error) {
  168. leader := client.cachedLeader(topic, partitionID)
  169. if leader == nil {
  170. err := client.RefreshTopicMetadata(topic)
  171. if err != nil {
  172. return nil, err
  173. }
  174. leader = client.cachedLeader(topic, partitionID)
  175. }
  176. if leader == nil {
  177. return nil, UnknownTopicOrPartition
  178. }
  179. return leader, nil
  180. }
  181. // RefreshTopicMetadata takes a list of topics and queries the cluster to refresh the
  182. // available metadata for those topics.
  183. func (client *Client) RefreshTopicMetadata(topics ...string) error {
  184. return client.refreshMetadata(topics, client.config.MetadataRetries)
  185. }
  186. // RefreshAllMetadata queries the cluster to refresh the available metadata for all topics.
  187. func (client *Client) RefreshAllMetadata() error {
  188. // Kafka refreshes all when you encode it an empty array...
  189. return client.refreshMetadata(make([]string, 0), client.config.MetadataRetries)
  190. }
  191. // GetOffset queries the cluster to get the most recent available offset at the given
  192. // time on the topic/partition combination.
  193. func (client *Client) GetOffset(topic string, partitionID int32, where OffsetTime) (int64, error) {
  194. broker, err := client.Leader(topic, partitionID)
  195. if err != nil {
  196. return -1, err
  197. }
  198. request := &OffsetRequest{}
  199. request.AddBlock(topic, partitionID, where, 1)
  200. response, err := broker.GetAvailableOffsets(client.id, request)
  201. if err != nil {
  202. return -1, err
  203. }
  204. block := response.GetBlock(topic, partitionID)
  205. if block == nil {
  206. return -1, IncompleteResponse
  207. }
  208. if block.Err != NoError {
  209. return -1, block.Err
  210. }
  211. if len(block.Offsets) != 1 {
  212. return -1, OffsetOutOfRange
  213. }
  214. return block.Offsets[0], nil
  215. }
  216. // misc private helper functions
  217. // XXX: see https://github.com/Shopify/sarama/issues/15
  218. // and https://github.com/Shopify/sarama/issues/23
  219. // disconnectBroker is a bad hacky way to accomplish broker management. It should be replaced with
  220. // something sane and the replacement should be made part of the public Client API
  221. func (client *Client) disconnectBroker(broker *Broker) {
  222. client.lock.Lock()
  223. defer client.lock.Unlock()
  224. Logger.Printf("Disconnecting Broker %d\n", broker.ID())
  225. client.deadBrokerAddrs[broker.addr] = struct{}{}
  226. if broker == client.seedBroker {
  227. client.seedBrokerAddrs = client.seedBrokerAddrs[1:]
  228. if len(client.seedBrokerAddrs) > 0 {
  229. client.seedBroker = NewBroker(client.seedBrokerAddrs[0])
  230. _ = client.seedBroker.Open(client.config.DefaultBrokerConf)
  231. } else {
  232. client.seedBroker = nil
  233. }
  234. } else {
  235. // we don't need to update the leaders hash, it will automatically get refreshed next time because
  236. // the broker lookup will return nil
  237. delete(client.brokers, broker.ID())
  238. }
  239. safeAsyncClose(broker)
  240. }
  241. func (client *Client) Closed() bool {
  242. return client.brokers == nil
  243. }
  244. func (client *Client) refreshMetadata(topics []string, retries int) error {
  245. // This function is a sort of central point for most functions that create new
  246. // resources. Check to see if we're dealing with a closed Client and error
  247. // out immediately if so.
  248. if client.Closed() {
  249. return ClosedClient
  250. }
  251. // Kafka will throw exceptions on an empty topic and not return a proper
  252. // error. This handles the case by returning an error instead of sending it
  253. // off to Kafka. See: https://github.com/Shopify/sarama/pull/38#issuecomment-26362310
  254. for _, topic := range topics {
  255. if len(topic) == 0 {
  256. return NoSuchTopic
  257. }
  258. }
  259. for broker := client.any(); broker != nil; broker = client.any() {
  260. Logger.Printf("Fetching metadata from broker %s\n", broker.addr)
  261. response, err := broker.GetMetadata(client.id, &MetadataRequest{Topics: topics})
  262. switch err {
  263. case nil:
  264. // valid response, use it
  265. retry, err := client.update(response)
  266. switch {
  267. case err != nil:
  268. return err
  269. case len(retry) == 0:
  270. return nil
  271. default:
  272. if retries <= 0 {
  273. return LeaderNotAvailable
  274. }
  275. Logger.Printf("Failed to fetch metadata from broker %s, waiting %dms... (%d retries remaining)\n", broker.addr, client.config.WaitForElection/time.Millisecond, retries)
  276. time.Sleep(client.config.WaitForElection) // wait for leader election
  277. return client.refreshMetadata(retry, retries-1)
  278. }
  279. case EncodingError:
  280. // didn't even send, return the error
  281. return err
  282. }
  283. // some other error, remove that broker and try again
  284. Logger.Println("Unexpected error from GetMetadata, closing broker:", err)
  285. client.disconnectBroker(broker)
  286. }
  287. if retries > 0 {
  288. Logger.Printf("Out of available brokers. Resurrecting dead brokers after %dms... (%d retries remaining)\n", client.config.WaitForElection/time.Millisecond, retries)
  289. time.Sleep(client.config.WaitForElection)
  290. client.resurrectDeadBrokers()
  291. return client.refreshMetadata(topics, retries-1)
  292. } else {
  293. Logger.Printf("Out of available brokers.\n")
  294. }
  295. return OutOfBrokers
  296. }
  297. func (client *Client) resurrectDeadBrokers() {
  298. client.lock.Lock()
  299. defer client.lock.Unlock()
  300. for _, addr := range client.seedBrokerAddrs {
  301. client.deadBrokerAddrs[addr] = struct{}{}
  302. }
  303. client.seedBrokerAddrs = []string{}
  304. for addr := range client.deadBrokerAddrs {
  305. client.seedBrokerAddrs = append(client.seedBrokerAddrs, addr)
  306. }
  307. client.deadBrokerAddrs = make(map[string]struct{})
  308. client.seedBroker = NewBroker(client.seedBrokerAddrs[0])
  309. _ = client.seedBroker.Open(client.config.DefaultBrokerConf)
  310. }
  311. func (client *Client) any() *Broker {
  312. client.lock.RLock()
  313. defer client.lock.RUnlock()
  314. for _, broker := range client.brokers {
  315. return broker
  316. }
  317. return client.seedBroker
  318. }
  319. func (client *Client) cachedLeader(topic string, partitionID int32) *Broker {
  320. client.lock.RLock()
  321. defer client.lock.RUnlock()
  322. partitions := client.metadata[topic]
  323. if partitions != nil {
  324. metadata, ok := partitions[partitionID]
  325. if ok {
  326. return client.brokers[metadata.Leader]
  327. }
  328. }
  329. return nil
  330. }
  331. func (client *Client) cachedMetadata(topic string, partitionID int32) *PartitionMetadata {
  332. client.lock.RLock()
  333. defer client.lock.RUnlock()
  334. partitions := client.metadata[topic]
  335. if partitions != nil {
  336. return partitions[partitionID]
  337. }
  338. return nil
  339. }
  340. func (client *Client) cachedPartitions(topic string) []int32 {
  341. client.lock.RLock()
  342. defer client.lock.RUnlock()
  343. partitions := client.metadata[topic]
  344. if partitions == nil {
  345. return nil
  346. }
  347. ret := make([]int32, 0, len(partitions))
  348. for id := range partitions {
  349. ret = append(ret, id)
  350. }
  351. sort.Sort(int32Slice(ret))
  352. return ret
  353. }
  354. func (client *Client) backgroundMetadataUpdater() {
  355. if client.config.BackgroundRefreshFrequency == time.Duration(0) {
  356. return
  357. }
  358. ticker := time.NewTicker(client.config.BackgroundRefreshFrequency)
  359. for {
  360. select {
  361. case <-ticker.C:
  362. if err := client.RefreshAllMetadata(); err != nil {
  363. Logger.Println("Client background metadata update:", err)
  364. }
  365. case <-client.closer:
  366. ticker.Stop()
  367. return
  368. }
  369. }
  370. }
  371. // if no fatal error, returns a list of topics that need retrying due to LeaderNotAvailable
  372. func (client *Client) update(data *MetadataResponse) ([]string, error) {
  373. client.lock.Lock()
  374. defer client.lock.Unlock()
  375. // For all the brokers we received:
  376. // - if it is a new ID, save it
  377. // - if it is an existing ID, but the address we have is stale, discard the old one and save it
  378. // - otherwise ignore it, replacing our existing one would just bounce the connection
  379. // We asynchronously try to open connections to the new brokers. We don't care if they
  380. // fail, since maybe that broker is unreachable but doesn't have a topic we care about.
  381. // If it fails and we do care, whoever tries to use it will get the connection error.
  382. for _, broker := range data.Brokers {
  383. if client.brokers[broker.ID()] == nil {
  384. _ = broker.Open(client.config.DefaultBrokerConf)
  385. client.brokers[broker.ID()] = broker
  386. Logger.Printf("Registered new broker #%d at %s", broker.ID(), broker.Addr())
  387. } else if broker.Addr() != client.brokers[broker.ID()].Addr() {
  388. safeAsyncClose(client.brokers[broker.ID()])
  389. _ = broker.Open(client.config.DefaultBrokerConf)
  390. client.brokers[broker.ID()] = broker
  391. Logger.Printf("Replaced registered broker #%d with %s", broker.ID(), broker.Addr())
  392. }
  393. }
  394. toRetry := make(map[string]bool)
  395. var err error
  396. for _, topic := range data.Topics {
  397. switch topic.Err {
  398. case NoError:
  399. break
  400. case LeaderNotAvailable:
  401. toRetry[topic.Name] = true
  402. default:
  403. err = topic.Err
  404. }
  405. client.metadata[topic.Name] = make(map[int32]*PartitionMetadata, len(topic.Partitions))
  406. for _, partition := range topic.Partitions {
  407. switch partition.Err {
  408. case NoError:
  409. broker := client.brokers[partition.Leader]
  410. if _, present := client.deadBrokerAddrs[broker.Addr()]; present {
  411. if connected, _ := broker.Connected(); !connected {
  412. toRetry[topic.Name] = true
  413. delete(client.metadata[topic.Name], partition.ID)
  414. continue
  415. }
  416. }
  417. client.metadata[topic.Name][partition.ID] = partition
  418. case LeaderNotAvailable:
  419. toRetry[topic.Name] = true
  420. delete(client.metadata[topic.Name], partition.ID)
  421. default:
  422. err = partition.Err
  423. }
  424. }
  425. }
  426. if err != nil {
  427. return nil, err
  428. }
  429. ret := make([]string, 0, len(toRetry))
  430. for topic := range toRetry {
  431. ret = append(ret, topic)
  432. }
  433. return ret, nil
  434. }
  435. // NewClientConfig creates a new ClientConfig instance with sensible defaults
  436. func NewClientConfig() *ClientConfig {
  437. return &ClientConfig{
  438. MetadataRetries: 3,
  439. WaitForElection: 250 * time.Millisecond,
  440. BackgroundRefreshFrequency: 10 * time.Minute,
  441. }
  442. }
  443. // Validate checks a ClientConfig instance. This will return a
  444. // ConfigurationError if the specified values don't make sense.
  445. func (config *ClientConfig) Validate() error {
  446. if config.MetadataRetries <= 0 {
  447. return ConfigurationError("Invalid MetadataRetries. Try 10")
  448. }
  449. if config.WaitForElection <= time.Duration(0) {
  450. return ConfigurationError("Invalid WaitForElection. Try 250*time.Millisecond")
  451. }
  452. if config.DefaultBrokerConf != nil {
  453. if err := config.DefaultBrokerConf.Validate(); err != nil {
  454. return err
  455. }
  456. }
  457. if config.BackgroundRefreshFrequency < time.Duration(0) {
  458. return ConfigurationError("Invalid BackgroundRefreshFrequency.")
  459. }
  460. return nil
  461. }