client.go 17 KB

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