client.go 17 KB

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