client.go 18 KB

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