client.go 17 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594
  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.
  33. GetOffset(topic string, partitionID int32, where OffsetTime) (int64, error)
  34. // Close shuts down all broker connections managed by this client. It is required to call this function before
  35. // a client object passes out of scope, as it will otherwise leak memory. You must close any Producers or Consumers
  36. // using a client before you close the client.
  37. Close() error
  38. // Closed returns true if the client has already had Close called on it
  39. Closed() bool
  40. }
  41. type client struct {
  42. conf *Config
  43. closer chan none
  44. // the broker addresses given to us through the constructor are not guaranteed to be returned in
  45. // the cluster metadata (I *think* it only returns brokers who are currently leading partitions?)
  46. // so we store them separately
  47. seedBrokerAddrs []string
  48. seedBroker *Broker
  49. deadBrokerAddrs map[string]none
  50. brokers map[int32]*Broker // maps broker ids to brokers
  51. metadata map[string]map[int32]*PartitionMetadata // maps topics to partition ids to metadata
  52. // If the number of partitions is large, we can get some churn calling cachedPartitions,
  53. // so the result is cached. It is important to update this value whenever metadata is changed
  54. cachedPartitionsResults map[string][maxPartitionIndex][]int32
  55. lock sync.RWMutex // protects access to the maps, only one since they're always written together
  56. }
  57. // NewClient creates a new Client. It connects to one of the given broker addresses
  58. // and uses that broker to automatically fetch metadata on the rest of the kafka cluster. If metadata cannot
  59. // be retrieved from any of the given broker addresses, the client is not created.
  60. func NewClient(addrs []string, conf *Config) (Client, error) {
  61. Logger.Println("Initializing new client")
  62. if conf == nil {
  63. conf = NewConfig()
  64. }
  65. if err := conf.Validate(); err != nil {
  66. return nil, err
  67. }
  68. if len(addrs) < 1 {
  69. return nil, ConfigurationError("You must provide at least one broker address")
  70. }
  71. client := &client{
  72. conf: conf,
  73. closer: make(chan none),
  74. seedBrokerAddrs: addrs,
  75. seedBroker: NewBroker(addrs[0]),
  76. deadBrokerAddrs: make(map[string]none),
  77. brokers: make(map[int32]*Broker),
  78. metadata: make(map[string]map[int32]*PartitionMetadata),
  79. cachedPartitionsResults: make(map[string][maxPartitionIndex][]int32),
  80. }
  81. _ = client.seedBroker.Open(conf)
  82. // do an initial fetch of all cluster metadata by specifing an empty list of topics
  83. err := client.RefreshMetadata()
  84. switch err {
  85. case nil:
  86. break
  87. case ErrLeaderNotAvailable, ErrReplicaNotAvailable:
  88. // indicates that maybe part of the cluster is down, but is not fatal to creating the client
  89. Logger.Println(err)
  90. default:
  91. _ = client.Close()
  92. return nil, err
  93. }
  94. go withRecover(client.backgroundMetadataUpdater)
  95. Logger.Println("Successfully initialized new client")
  96. return client, nil
  97. }
  98. func (client *client) Config() *Config {
  99. return client.conf
  100. }
  101. func (client *client) Close() error {
  102. // Check to see whether the client is closed
  103. if client.Closed() {
  104. // Chances are this is being called from a defer() and the error will go unobserved
  105. // so we go ahead and log the event in this case.
  106. Logger.Printf("Close() called on already closed client")
  107. return ErrClosedClient
  108. }
  109. client.lock.Lock()
  110. defer client.lock.Unlock()
  111. Logger.Println("Closing Client")
  112. for _, broker := range client.brokers {
  113. safeAsyncClose(broker)
  114. }
  115. client.brokers = nil
  116. client.metadata = nil
  117. if client.seedBroker != nil {
  118. safeAsyncClose(client.seedBroker)
  119. }
  120. close(client.closer)
  121. return nil
  122. }
  123. func (client *client) Closed() bool {
  124. return client.brokers == nil
  125. }
  126. func (client *client) Topics() ([]string, error) {
  127. // Check to see whether the client is closed
  128. if client.Closed() {
  129. return nil, ErrClosedClient
  130. }
  131. client.lock.RLock()
  132. defer client.lock.RUnlock()
  133. ret := make([]string, 0, len(client.metadata))
  134. for topic := range client.metadata {
  135. ret = append(ret, topic)
  136. }
  137. return ret, nil
  138. }
  139. func (client *client) Partitions(topic string) ([]int32, error) {
  140. // Check to see whether the client is closed
  141. if client.Closed() {
  142. return nil, ErrClosedClient
  143. }
  144. partitions := client.cachedPartitions(topic, allPartitions)
  145. if len(partitions) == 0 {
  146. err := client.RefreshMetadata(topic)
  147. if err != nil {
  148. return nil, err
  149. }
  150. partitions = client.cachedPartitions(topic, allPartitions)
  151. }
  152. if partitions == nil {
  153. return nil, ErrUnknownTopicOrPartition
  154. }
  155. return partitions, nil
  156. }
  157. func (client *client) WritablePartitions(topic string) ([]int32, error) {
  158. // Check to see whether the client is closed
  159. if client.Closed() {
  160. return nil, ErrClosedClient
  161. }
  162. partitions := client.cachedPartitions(topic, writablePartitions)
  163. // len==0 catches when it's nil (no such topic) and the odd case when every single
  164. // partition is undergoing leader election simultaneously. Callers have to be able to handle
  165. // this function returning an empty slice (which is a valid return value) but catching it
  166. // here the first time (note we *don't* catch it below where we return ErrUnknownTopicOrPartition) triggers
  167. // a metadata refresh as a nicety so callers can just try again and don't have to manually
  168. // trigger a refresh (otherwise they'd just keep getting a stale cached copy).
  169. if len(partitions) == 0 {
  170. err := client.RefreshMetadata(topic)
  171. if err != nil {
  172. return nil, err
  173. }
  174. partitions = client.cachedPartitions(topic, writablePartitions)
  175. }
  176. if partitions == nil {
  177. return nil, ErrUnknownTopicOrPartition
  178. }
  179. return partitions, nil
  180. }
  181. func (client *client) Replicas(topic string, partitionID int32) ([]int32, error) {
  182. if client.Closed() {
  183. return nil, ErrClosedClient
  184. }
  185. metadata, err := client.getMetadata(topic, partitionID)
  186. if err != nil {
  187. return nil, err
  188. }
  189. if metadata.Err == ErrReplicaNotAvailable {
  190. return nil, metadata.Err
  191. }
  192. return dupeAndSort(metadata.Replicas), nil
  193. }
  194. func (client *client) Leader(topic string, partitionID int32) (*Broker, error) {
  195. leader, err := client.cachedLeader(topic, partitionID)
  196. if leader == nil {
  197. err := client.RefreshMetadata(topic)
  198. if err != nil {
  199. return nil, err
  200. }
  201. leader, err = client.cachedLeader(topic, partitionID)
  202. }
  203. return leader, err
  204. }
  205. func (client *client) RefreshMetadata(topics ...string) error {
  206. return client.tryRefreshMetadata(topics, client.conf.Metadata.Retry.Max)
  207. }
  208. func (client *client) GetOffset(topic string, partitionID int32, where OffsetTime) (int64, error) {
  209. broker, err := client.Leader(topic, partitionID)
  210. if err != nil {
  211. return -1, err
  212. }
  213. request := &OffsetRequest{}
  214. request.AddBlock(topic, partitionID, where, 1)
  215. response, err := broker.GetAvailableOffsets(request)
  216. if err != nil {
  217. return -1, err
  218. }
  219. block := response.GetBlock(topic, partitionID)
  220. if block == nil {
  221. return -1, ErrIncompleteResponse
  222. }
  223. if block.Err != ErrNoError {
  224. return -1, block.Err
  225. }
  226. if len(block.Offsets) != 1 {
  227. return -1, ErrOffsetOutOfRange
  228. }
  229. return block.Offsets[0], nil
  230. }
  231. // private broker management helpers
  232. // XXX: see https://github.com/Shopify/sarama/issues/15
  233. // and https://github.com/Shopify/sarama/issues/23
  234. // disconnectBroker is a bad hacky way to accomplish broker management. It should be replaced with
  235. // something sane and the replacement should be made part of the public Client API
  236. func (client *client) disconnectBroker(broker *Broker) {
  237. client.lock.Lock()
  238. defer client.lock.Unlock()
  239. Logger.Printf("Disconnecting Broker %d\n", broker.ID())
  240. client.deadBrokerAddrs[broker.addr] = none{}
  241. if broker == client.seedBroker {
  242. client.seedBrokerAddrs = client.seedBrokerAddrs[1:]
  243. if len(client.seedBrokerAddrs) > 0 {
  244. client.seedBroker = NewBroker(client.seedBrokerAddrs[0])
  245. _ = client.seedBroker.Open(client.conf)
  246. } else {
  247. client.seedBroker = nil
  248. }
  249. } else {
  250. // we don't need to update the leaders hash, it will automatically get refreshed next time because
  251. // the broker lookup will return nil
  252. delete(client.brokers, broker.ID())
  253. }
  254. safeAsyncClose(broker)
  255. }
  256. func (client *client) resurrectDeadBrokers() {
  257. client.lock.Lock()
  258. defer client.lock.Unlock()
  259. for _, addr := range client.seedBrokerAddrs {
  260. client.deadBrokerAddrs[addr] = none{}
  261. }
  262. client.seedBrokerAddrs = []string{}
  263. for addr := range client.deadBrokerAddrs {
  264. client.seedBrokerAddrs = append(client.seedBrokerAddrs, addr)
  265. }
  266. client.deadBrokerAddrs = make(map[string]none)
  267. client.seedBroker = NewBroker(client.seedBrokerAddrs[0])
  268. _ = client.seedBroker.Open(client.conf)
  269. }
  270. func (client *client) any() *Broker {
  271. client.lock.RLock()
  272. defer client.lock.RUnlock()
  273. if client.seedBroker != nil {
  274. _ = client.seedBroker.Open(client.conf)
  275. return client.seedBroker
  276. }
  277. for _, broker := range client.brokers {
  278. _ = broker.Open(client.conf)
  279. return broker
  280. }
  281. return nil
  282. }
  283. // private caching/lazy metadata helpers
  284. type partitionType int
  285. const (
  286. allPartitions partitionType = iota
  287. writablePartitions
  288. // If you add any more types, update the partition cache in update()
  289. // Ensure this is the last partition type value
  290. maxPartitionIndex
  291. )
  292. func (client *client) getMetadata(topic string, partitionID int32) (*PartitionMetadata, error) {
  293. metadata := client.cachedMetadata(topic, partitionID)
  294. if metadata == nil {
  295. err := client.RefreshMetadata(topic)
  296. if err != nil {
  297. return nil, err
  298. }
  299. metadata = client.cachedMetadata(topic, partitionID)
  300. }
  301. if metadata == nil {
  302. return nil, ErrUnknownTopicOrPartition
  303. }
  304. return metadata, nil
  305. }
  306. func (client *client) cachedMetadata(topic string, partitionID int32) *PartitionMetadata {
  307. client.lock.RLock()
  308. defer client.lock.RUnlock()
  309. partitions := client.metadata[topic]
  310. if partitions != nil {
  311. return partitions[partitionID]
  312. }
  313. return nil
  314. }
  315. func (client *client) cachedPartitions(topic string, partitionSet partitionType) []int32 {
  316. client.lock.RLock()
  317. defer client.lock.RUnlock()
  318. partitions, exists := client.cachedPartitionsResults[topic]
  319. if !exists {
  320. return nil
  321. }
  322. return partitions[partitionSet]
  323. }
  324. func (client *client) setPartitionCache(topic string, partitionSet partitionType) []int32 {
  325. partitions := client.metadata[topic]
  326. if partitions == nil {
  327. return nil
  328. }
  329. ret := make([]int32, 0, len(partitions))
  330. for _, partition := range partitions {
  331. if partitionSet == writablePartitions && partition.Err == ErrLeaderNotAvailable {
  332. continue
  333. }
  334. ret = append(ret, partition.ID)
  335. }
  336. sort.Sort(int32Slice(ret))
  337. return ret
  338. }
  339. func (client *client) cachedLeader(topic string, partitionID int32) (*Broker, error) {
  340. client.lock.RLock()
  341. defer client.lock.RUnlock()
  342. partitions := client.metadata[topic]
  343. if partitions != nil {
  344. metadata, ok := partitions[partitionID]
  345. if ok {
  346. if metadata.Err == ErrLeaderNotAvailable {
  347. return nil, ErrLeaderNotAvailable
  348. }
  349. b := client.brokers[metadata.Leader]
  350. if b == nil {
  351. return nil, ErrLeaderNotAvailable
  352. }
  353. _ = b.Open(client.conf)
  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.RefreshMetadata(); 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) tryRefreshMetadata(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.tryRefreshMetadata(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.tryRefreshMetadata(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. for _, broker := range data.Brokers {
  442. if client.brokers[broker.ID()] == nil {
  443. client.brokers[broker.ID()] = broker
  444. Logger.Printf("Registered new broker #%d at %s", broker.ID(), broker.Addr())
  445. } else if broker.Addr() != client.brokers[broker.ID()].Addr() {
  446. safeAsyncClose(client.brokers[broker.ID()])
  447. client.brokers[broker.ID()] = broker
  448. Logger.Printf("Replaced registered broker #%d with %s", broker.ID(), broker.Addr())
  449. }
  450. }
  451. toRetry := make(map[string]bool)
  452. var err error
  453. for _, topic := range data.Topics {
  454. switch topic.Err {
  455. case ErrNoError:
  456. break
  457. case ErrLeaderNotAvailable, ErrUnknownTopicOrPartition:
  458. toRetry[topic.Name] = true
  459. default:
  460. err = topic.Err
  461. }
  462. client.metadata[topic.Name] = make(map[int32]*PartitionMetadata, len(topic.Partitions))
  463. delete(client.cachedPartitionsResults, topic.Name)
  464. for _, partition := range topic.Partitions {
  465. client.metadata[topic.Name][partition.ID] = partition
  466. if partition.Err == ErrLeaderNotAvailable {
  467. toRetry[topic.Name] = true
  468. }
  469. }
  470. var partitionCache [maxPartitionIndex][]int32
  471. partitionCache[allPartitions] = client.setPartitionCache(topic.Name, allPartitions)
  472. partitionCache[writablePartitions] = client.setPartitionCache(topic.Name, writablePartitions)
  473. client.cachedPartitionsResults[topic.Name] = partitionCache
  474. }
  475. if err != nil {
  476. return nil, err
  477. }
  478. ret := make([]string, 0, len(toRetry))
  479. for topic := range toRetry {
  480. ret = append(ret, topic)
  481. }
  482. return ret, nil
  483. }