client.go 17 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593
  1. package sarama
  2. import (
  3. "sort"
  4. "sync"
  5. "time"
  6. )
  7. // Client is a generic Kafka client. It manages connections to one or more Kafka brokers.
  8. // You MUST call Close() on a client to avoid leaks, it will not be garbage-collected
  9. // automatically when it passes out of scope. A single client can be safely shared by
  10. // multiple concurrent Producers and Consumers.
  11. type Client struct {
  12. conf *Config
  13. closer chan none
  14. // the broker addresses given to us through the constructor are not guaranteed to be returned in
  15. // the cluster metadata (I *think* it only returns brokers who are currently leading partitions?)
  16. // so we store them separately
  17. seedBrokerAddrs []string
  18. seedBroker *Broker
  19. deadBrokerAddrs map[string]none
  20. brokers map[int32]*Broker // maps broker ids to brokers
  21. metadata map[string]map[int32]*PartitionMetadata // maps topics to partition ids to metadata
  22. // If the number of partitions is large, we can get some churn calling cachedPartitions,
  23. // so the result is cached. It is important to update this value whenever metadata is changed
  24. cachedPartitionsResults map[string][maxPartitionIndex][]int32
  25. lock sync.RWMutex // protects access to the maps, only one since they're always written together
  26. }
  27. // NewClient creates a new Client. It connects to one of the given broker addresses
  28. // and uses that broker to automatically fetch metadata on the rest of the kafka cluster. If metadata cannot
  29. // be retrieved from any of the given broker addresses, the client is not created.
  30. func NewClient(addrs []string, conf *Config) (*Client, error) {
  31. Logger.Println("Initializing new client")
  32. if conf == nil {
  33. conf = NewConfig()
  34. }
  35. if err := conf.Validate(); err != nil {
  36. return nil, err
  37. }
  38. if len(addrs) < 1 {
  39. return nil, ConfigurationError("You must provide at least one broker address")
  40. }
  41. client := &Client{
  42. conf: conf,
  43. closer: make(chan none),
  44. seedBrokerAddrs: addrs,
  45. seedBroker: NewBroker(addrs[0]),
  46. deadBrokerAddrs: make(map[string]none),
  47. brokers: make(map[int32]*Broker),
  48. metadata: make(map[string]map[int32]*PartitionMetadata),
  49. cachedPartitionsResults: make(map[string][maxPartitionIndex][]int32),
  50. }
  51. _ = client.seedBroker.Open(conf)
  52. // do an initial fetch of all cluster metadata by specifing an empty list of topics
  53. err := client.RefreshAllMetadata()
  54. switch err {
  55. case nil:
  56. break
  57. case ErrLeaderNotAvailable, ErrReplicaNotAvailable:
  58. // indicates that maybe part of the cluster is down, but is not fatal to creating the client
  59. Logger.Println(err)
  60. default:
  61. _ = client.Close()
  62. return nil, err
  63. }
  64. go withRecover(client.backgroundMetadataUpdater)
  65. Logger.Println("Successfully initialized new client")
  66. return client, nil
  67. }
  68. // Close shuts down all broker connections managed by this client. It is required to call this function before
  69. // a client object passes out of scope, as it will otherwise leak memory. You must close any Producers or Consumers
  70. // using a client before you close the client.
  71. func (client *Client) Close() error {
  72. // Check to see whether the client is closed
  73. if client.Closed() {
  74. // Chances are this is being called from a defer() and the error will go unobserved
  75. // so we go ahead and log the event in this case.
  76. Logger.Printf("Close() called on already closed client")
  77. return ErrClosedClient
  78. }
  79. client.lock.Lock()
  80. defer client.lock.Unlock()
  81. Logger.Println("Closing Client")
  82. for _, broker := range client.brokers {
  83. safeAsyncClose(broker)
  84. }
  85. client.brokers = nil
  86. client.metadata = nil
  87. if client.seedBroker != nil {
  88. safeAsyncClose(client.seedBroker)
  89. }
  90. close(client.closer)
  91. return nil
  92. }
  93. // Closed returns true if the client has already had Close called on it
  94. func (client *Client) Closed() bool {
  95. return client.brokers == nil
  96. }
  97. // Topics returns the set of available topics as retrieved from the cluster metadata.
  98. func (client *Client) Topics() ([]string, error) {
  99. // Check to see whether the client is closed
  100. if client.Closed() {
  101. return nil, ErrClosedClient
  102. }
  103. client.lock.RLock()
  104. defer client.lock.RUnlock()
  105. ret := make([]string, 0, len(client.metadata))
  106. for topic := range client.metadata {
  107. ret = append(ret, topic)
  108. }
  109. return ret, nil
  110. }
  111. // Partitions returns the sorted list of all partition IDs for the given topic.
  112. func (client *Client) Partitions(topic string) ([]int32, error) {
  113. // Check to see whether the client is closed
  114. if client.Closed() {
  115. return nil, ErrClosedClient
  116. }
  117. partitions := client.cachedPartitions(topic, allPartitions)
  118. if len(partitions) == 0 {
  119. err := client.RefreshTopicMetadata(topic)
  120. if err != nil {
  121. return nil, err
  122. }
  123. partitions = client.cachedPartitions(topic, allPartitions)
  124. }
  125. if partitions == nil {
  126. return nil, ErrUnknownTopicOrPartition
  127. }
  128. return partitions, nil
  129. }
  130. // WritablePartitions returns the sorted list of all writable partition IDs for the given topic,
  131. // where "writable" means "having a valid leader accepting writes".
  132. func (client *Client) WritablePartitions(topic string) ([]int32, error) {
  133. // Check to see whether the client is closed
  134. if client.Closed() {
  135. return nil, ErrClosedClient
  136. }
  137. partitions := client.cachedPartitions(topic, writablePartitions)
  138. // len==0 catches when it's nil (no such topic) and the odd case when every single
  139. // partition is undergoing leader election simultaneously. Callers have to be able to handle
  140. // this function returning an empty slice (which is a valid return value) but catching it
  141. // here the first time (note we *don't* catch it below where we return ErrUnknownTopicOrPartition) triggers
  142. // a metadata refresh as a nicety so callers can just try again and don't have to manually
  143. // trigger a refresh (otherwise they'd just keep getting a stale cached copy).
  144. if len(partitions) == 0 {
  145. err := client.RefreshTopicMetadata(topic)
  146. if err != nil {
  147. return nil, err
  148. }
  149. partitions = client.cachedPartitions(topic, writablePartitions)
  150. }
  151. if partitions == nil {
  152. return nil, ErrUnknownTopicOrPartition
  153. }
  154. return partitions, nil
  155. }
  156. // Replicas returns the set of all replica IDs for the given partition.
  157. func (client *Client) Replicas(topic string, partitionID int32) ([]int32, error) {
  158. if client.Closed() {
  159. return nil, ErrClosedClient
  160. }
  161. metadata, err := client.getMetadata(topic, partitionID)
  162. if err != nil {
  163. return nil, err
  164. }
  165. if metadata.Err == ErrReplicaNotAvailable {
  166. return nil, metadata.Err
  167. }
  168. return dupeAndSort(metadata.Replicas), nil
  169. }
  170. // ReplicasInSync returns the set of all in-sync replica IDs for the given partition.
  171. // Note: kafka's metadata here is known to be stale in many cases, and should not generally be trusted.
  172. // This method should be considered effectively deprecated.
  173. func (client *Client) ReplicasInSync(topic string, partitionID int32) ([]int32, error) {
  174. if client.Closed() {
  175. return nil, ErrClosedClient
  176. }
  177. metadata, err := client.getMetadata(topic, partitionID)
  178. if err != nil {
  179. return nil, err
  180. }
  181. if metadata.Err == ErrReplicaNotAvailable {
  182. return nil, metadata.Err
  183. }
  184. return dupeAndSort(metadata.Isr), nil
  185. }
  186. // Leader returns the broker object that is the leader of the current topic/partition, as
  187. // determined by querying the cluster metadata.
  188. func (client *Client) Leader(topic string, partitionID int32) (*Broker, error) {
  189. leader, err := client.cachedLeader(topic, partitionID)
  190. if leader == nil {
  191. err := client.RefreshTopicMetadata(topic)
  192. if err != nil {
  193. return nil, err
  194. }
  195. leader, err = client.cachedLeader(topic, partitionID)
  196. }
  197. return leader, err
  198. }
  199. // RefreshTopicMetadata takes a list of topics and queries the cluster to refresh the
  200. // available metadata for those topics.
  201. func (client *Client) RefreshTopicMetadata(topics ...string) error {
  202. return client.refreshMetadata(topics, client.conf.Metadata.Retry.Max)
  203. }
  204. // RefreshAllMetadata queries the cluster to refresh the available metadata for all topics.
  205. func (client *Client) RefreshAllMetadata() error {
  206. // Kafka refreshes all when you encode it an empty array...
  207. return client.refreshMetadata(make([]string, 0), client.conf.Metadata.Retry.Max)
  208. }
  209. // GetOffset queries the cluster to get the most recent available offset at the given
  210. // time on the topic/partition combination.
  211. func (client *Client) GetOffset(topic string, partitionID int32, where OffsetTime) (int64, error) {
  212. broker, err := client.Leader(topic, partitionID)
  213. if err != nil {
  214. return -1, err
  215. }
  216. request := &OffsetRequest{}
  217. request.AddBlock(topic, partitionID, where, 1)
  218. response, err := broker.GetAvailableOffsets(request)
  219. if err != nil {
  220. return -1, err
  221. }
  222. block := response.GetBlock(topic, partitionID)
  223. if block == nil {
  224. return -1, ErrIncompleteResponse
  225. }
  226. if block.Err != ErrNoError {
  227. return -1, block.Err
  228. }
  229. if len(block.Offsets) != 1 {
  230. return -1, ErrOffsetOutOfRange
  231. }
  232. return block.Offsets[0], nil
  233. }
  234. // private broker management helpers
  235. // XXX: see https://github.com/Shopify/sarama/issues/15
  236. // and https://github.com/Shopify/sarama/issues/23
  237. // disconnectBroker is a bad hacky way to accomplish broker management. It should be replaced with
  238. // something sane and the replacement should be made part of the public Client API
  239. func (client *Client) disconnectBroker(broker *Broker) {
  240. client.lock.Lock()
  241. defer client.lock.Unlock()
  242. Logger.Printf("Disconnecting Broker %d\n", broker.ID())
  243. client.deadBrokerAddrs[broker.addr] = none{}
  244. if broker == client.seedBroker {
  245. client.seedBrokerAddrs = client.seedBrokerAddrs[1:]
  246. if len(client.seedBrokerAddrs) > 0 {
  247. client.seedBroker = NewBroker(client.seedBrokerAddrs[0])
  248. _ = client.seedBroker.Open(client.conf)
  249. } else {
  250. client.seedBroker = nil
  251. }
  252. } else {
  253. // we don't need to update the leaders hash, it will automatically get refreshed next time because
  254. // the broker lookup will return nil
  255. delete(client.brokers, broker.ID())
  256. }
  257. safeAsyncClose(broker)
  258. }
  259. func (client *Client) resurrectDeadBrokers() {
  260. client.lock.Lock()
  261. defer client.lock.Unlock()
  262. for _, addr := range client.seedBrokerAddrs {
  263. client.deadBrokerAddrs[addr] = none{}
  264. }
  265. client.seedBrokerAddrs = []string{}
  266. for addr := range client.deadBrokerAddrs {
  267. client.seedBrokerAddrs = append(client.seedBrokerAddrs, addr)
  268. }
  269. client.deadBrokerAddrs = make(map[string]none)
  270. client.seedBroker = NewBroker(client.seedBrokerAddrs[0])
  271. _ = client.seedBroker.Open(client.conf)
  272. }
  273. func (client *Client) any() *Broker {
  274. client.lock.RLock()
  275. defer client.lock.RUnlock()
  276. if client.seedBroker != nil {
  277. return client.seedBroker
  278. }
  279. for _, broker := range client.brokers {
  280. return broker
  281. }
  282. return nil
  283. }
  284. // private caching/lazy metadata helpers
  285. type partitionType int
  286. const (
  287. allPartitions partitionType = iota
  288. writablePartitions
  289. // If you add any more types, update the partition cache in update()
  290. // Ensure this is the last partition type value
  291. maxPartitionIndex
  292. )
  293. func (client *Client) getMetadata(topic string, partitionID int32) (*PartitionMetadata, error) {
  294. metadata := client.cachedMetadata(topic, partitionID)
  295. if metadata == nil {
  296. err := client.RefreshTopicMetadata(topic)
  297. if err != nil {
  298. return nil, err
  299. }
  300. metadata = client.cachedMetadata(topic, partitionID)
  301. }
  302. if metadata == nil {
  303. return nil, ErrUnknownTopicOrPartition
  304. }
  305. return metadata, nil
  306. }
  307. func (client *Client) cachedMetadata(topic string, partitionID int32) *PartitionMetadata {
  308. client.lock.RLock()
  309. defer client.lock.RUnlock()
  310. partitions := client.metadata[topic]
  311. if partitions != nil {
  312. return partitions[partitionID]
  313. }
  314. return nil
  315. }
  316. func (client *Client) cachedPartitions(topic string, partitionSet partitionType) []int32 {
  317. client.lock.RLock()
  318. defer client.lock.RUnlock()
  319. partitions, exists := client.cachedPartitionsResults[topic]
  320. if !exists {
  321. return nil
  322. }
  323. return partitions[partitionSet]
  324. }
  325. func (client *Client) setPartitionCache(topic string, partitionSet partitionType) []int32 {
  326. partitions := client.metadata[topic]
  327. if partitions == nil {
  328. return nil
  329. }
  330. ret := make([]int32, 0, len(partitions))
  331. for _, partition := range partitions {
  332. if partitionSet == writablePartitions && partition.Err == ErrLeaderNotAvailable {
  333. continue
  334. }
  335. ret = append(ret, partition.ID)
  336. }
  337. sort.Sort(int32Slice(ret))
  338. return ret
  339. }
  340. func (client *Client) cachedLeader(topic string, partitionID int32) (*Broker, error) {
  341. client.lock.RLock()
  342. defer client.lock.RUnlock()
  343. partitions := client.metadata[topic]
  344. if partitions != nil {
  345. metadata, ok := partitions[partitionID]
  346. if ok {
  347. if metadata.Err == ErrLeaderNotAvailable {
  348. return nil, ErrLeaderNotAvailable
  349. }
  350. b := client.brokers[metadata.Leader]
  351. if b == nil {
  352. return nil, ErrLeaderNotAvailable
  353. }
  354. return b, nil
  355. }
  356. }
  357. return nil, ErrUnknownTopicOrPartition
  358. }
  359. // core metadata update logic
  360. func (client *Client) backgroundMetadataUpdater() {
  361. if client.conf.Metadata.RefreshFrequency == time.Duration(0) {
  362. return
  363. }
  364. ticker := time.NewTicker(client.conf.Metadata.RefreshFrequency)
  365. for {
  366. select {
  367. case <-ticker.C:
  368. if err := client.RefreshAllMetadata(); err != nil {
  369. Logger.Println("Client background metadata update:", err)
  370. }
  371. case <-client.closer:
  372. ticker.Stop()
  373. return
  374. }
  375. }
  376. }
  377. func (client *Client) refreshMetadata(topics []string, retriesRemaining int) error {
  378. // This function is a sort of central point for most functions that create new
  379. // resources. Check to see if we're dealing with a closed Client and error
  380. // out immediately if so.
  381. if client.Closed() {
  382. return ErrClosedClient
  383. }
  384. // Prior to 0.8.2, Kafka will throw exceptions on an empty topic and not return a proper
  385. // error. This handles the case by returning an error instead of sending it
  386. // off to Kafka. See: https://github.com/Shopify/sarama/pull/38#issuecomment-26362310
  387. for _, topic := range topics {
  388. if len(topic) == 0 {
  389. return ErrInvalidTopic // this is the error that 0.8.2 and later correctly return
  390. }
  391. }
  392. for broker := client.any(); broker != nil; broker = client.any() {
  393. if len(topics) > 0 {
  394. Logger.Printf("Fetching metadata for %v from broker %s\n", topics, broker.addr)
  395. } else {
  396. Logger.Printf("Fetching metadata for all topics from broker %s\n", broker.addr)
  397. }
  398. response, err := broker.GetMetadata(&MetadataRequest{Topics: topics})
  399. switch err.(type) {
  400. case nil:
  401. // valid response, use it
  402. retry, err := client.update(response)
  403. if len(retry) > 0 {
  404. if retriesRemaining <= 0 {
  405. Logger.Println("Some partitions are leaderless, but we're out of retries")
  406. return nil
  407. }
  408. Logger.Printf("Some partitions are leaderless, waiting %dms for election... (%d retries remaining)\n",
  409. client.conf.Metadata.Retry.Backoff/time.Millisecond, retriesRemaining)
  410. time.Sleep(client.conf.Metadata.Retry.Backoff) // wait for leader election
  411. return client.refreshMetadata(retry, retriesRemaining-1)
  412. }
  413. return err
  414. case PacketEncodingError:
  415. // didn't even send, return the error
  416. return err
  417. default:
  418. // some other error, remove that broker and try again
  419. Logger.Println("Error from broker while fetching metadata:", err)
  420. client.disconnectBroker(broker)
  421. }
  422. }
  423. Logger.Println("Out of available brokers.")
  424. if retriesRemaining > 0 {
  425. Logger.Printf("Resurrecting dead brokers after %dms... (%d retries remaining)\n",
  426. client.conf.Metadata.Retry.Backoff/time.Millisecond, retriesRemaining)
  427. time.Sleep(client.conf.Metadata.Retry.Backoff)
  428. client.resurrectDeadBrokers()
  429. return client.refreshMetadata(topics, retriesRemaining-1)
  430. }
  431. return ErrOutOfBrokers
  432. }
  433. // if no fatal error, returns a list of topics that need retrying due to ErrLeaderNotAvailable
  434. func (client *Client) update(data *MetadataResponse) ([]string, error) {
  435. client.lock.Lock()
  436. defer client.lock.Unlock()
  437. // For all the brokers we received:
  438. // - if it is a new ID, save it
  439. // - if it is an existing ID, but the address we have is stale, discard the old one and save it
  440. // - otherwise ignore it, replacing our existing one would just bounce the connection
  441. // We asynchronously try to open connections to the new brokers. We don't care if they
  442. // fail, since maybe that broker is unreachable but doesn't have a topic we care about.
  443. // If it fails and we do care, whoever tries to use it will get the connection error.
  444. for _, broker := range data.Brokers {
  445. if client.brokers[broker.ID()] == nil {
  446. _ = broker.Open(client.conf)
  447. client.brokers[broker.ID()] = broker
  448. Logger.Printf("Registered new broker #%d at %s", broker.ID(), broker.Addr())
  449. } else if broker.Addr() != client.brokers[broker.ID()].Addr() {
  450. safeAsyncClose(client.brokers[broker.ID()])
  451. _ = broker.Open(client.conf)
  452. client.brokers[broker.ID()] = broker
  453. Logger.Printf("Replaced registered broker #%d with %s", broker.ID(), broker.Addr())
  454. }
  455. }
  456. toRetry := make(map[string]bool)
  457. var err error
  458. for _, topic := range data.Topics {
  459. switch topic.Err {
  460. case ErrNoError:
  461. break
  462. case ErrLeaderNotAvailable:
  463. toRetry[topic.Name] = true
  464. default:
  465. err = topic.Err
  466. }
  467. client.metadata[topic.Name] = make(map[int32]*PartitionMetadata, len(topic.Partitions))
  468. delete(client.cachedPartitionsResults, topic.Name)
  469. for _, partition := range topic.Partitions {
  470. client.metadata[topic.Name][partition.ID] = partition
  471. if partition.Err == ErrLeaderNotAvailable {
  472. toRetry[topic.Name] = true
  473. }
  474. }
  475. var partitionCache [maxPartitionIndex][]int32
  476. partitionCache[allPartitions] = client.setPartitionCache(topic.Name, allPartitions)
  477. partitionCache[writablePartitions] = client.setPartitionCache(topic.Name, writablePartitions)
  478. client.cachedPartitionsResults[topic.Name] = partitionCache
  479. }
  480. if err != nil {
  481. return nil, err
  482. }
  483. ret := make([]string, 0, len(toRetry))
  484. for topic := range toRetry {
  485. ret = append(ret, topic)
  486. }
  487. return ret, nil
  488. }