broker.go 9.6 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462
  1. package sarama
  2. import (
  3. "crypto/tls"
  4. "fmt"
  5. "io"
  6. "net"
  7. "strconv"
  8. "sync"
  9. "sync/atomic"
  10. "time"
  11. )
  12. // Broker represents a single Kafka broker connection. All operations on this object are entirely concurrency-safe.
  13. type Broker struct {
  14. id int32
  15. addr string
  16. conf *Config
  17. correlationID int32
  18. conn net.Conn
  19. connErr error
  20. lock sync.Mutex
  21. opened int32
  22. responses chan responsePromise
  23. done chan bool
  24. }
  25. type responsePromise struct {
  26. correlationID int32
  27. packets chan []byte
  28. errors chan error
  29. }
  30. // NewBroker creates and returns a Broker targetting the given host:port address.
  31. // This does not attempt to actually connect, you have to call Open() for that.
  32. func NewBroker(addr string) *Broker {
  33. return &Broker{id: -1, addr: addr}
  34. }
  35. // Open tries to connect to the Broker if it is not already connected or connecting, but does not block
  36. // waiting for the connection to complete. This means that any subsequent operations on the broker will
  37. // block waiting for the connection to succeed or fail. To get the effect of a fully synchronous Open call,
  38. // follow it by a call to Connected(). The only errors Open will return directly are ConfigurationError or
  39. // AlreadyConnected. If conf is nil, the result of NewConfig() is used.
  40. func (b *Broker) Open(conf *Config) error {
  41. if conf == nil {
  42. conf = NewConfig()
  43. }
  44. err := conf.Validate()
  45. if err != nil {
  46. return err
  47. }
  48. if !atomic.CompareAndSwapInt32(&b.opened, 0, 1) {
  49. return ErrAlreadyConnected
  50. }
  51. b.lock.Lock()
  52. if b.conn != nil {
  53. b.lock.Unlock()
  54. Logger.Printf("Failed to connect to broker %s: %s\n", b.addr, ErrAlreadyConnected)
  55. return ErrAlreadyConnected
  56. }
  57. go withRecover(func() {
  58. defer b.lock.Unlock()
  59. dialer := net.Dialer{
  60. Timeout: conf.Net.DialTimeout,
  61. KeepAlive: conf.Net.KeepAlive,
  62. }
  63. if conf.Net.TLS.Enable {
  64. b.conn, b.connErr = tls.DialWithDialer(&dialer, "tcp", b.addr, conf.Net.TLS.Config)
  65. } else {
  66. b.conn, b.connErr = dialer.Dial("tcp", b.addr)
  67. }
  68. if b.connErr != nil {
  69. b.conn = nil
  70. atomic.StoreInt32(&b.opened, 0)
  71. Logger.Printf("Failed to connect to broker %s: %s\n", b.addr, b.connErr)
  72. return
  73. }
  74. b.conn = newBufConn(b.conn)
  75. b.conf = conf
  76. b.done = make(chan bool)
  77. b.responses = make(chan responsePromise, b.conf.Net.MaxOpenRequests-1)
  78. if b.id >= 0 {
  79. Logger.Printf("Connected to broker at %s (registered as #%d)\n", b.addr, b.id)
  80. } else {
  81. Logger.Printf("Connected to broker at %s (unregistered)\n", b.addr)
  82. }
  83. go withRecover(b.responseReceiver)
  84. })
  85. return nil
  86. }
  87. // Connected returns true if the broker is connected and false otherwise. If the broker is not
  88. // connected but it had tried to connect, the error from that connection attempt is also returned.
  89. func (b *Broker) Connected() (bool, error) {
  90. b.lock.Lock()
  91. defer b.lock.Unlock()
  92. return b.conn != nil, b.connErr
  93. }
  94. func (b *Broker) Close() error {
  95. b.lock.Lock()
  96. defer b.lock.Unlock()
  97. if b.conn == nil {
  98. return ErrNotConnected
  99. }
  100. close(b.responses)
  101. <-b.done
  102. err := b.conn.Close()
  103. b.conn = nil
  104. b.connErr = nil
  105. b.done = nil
  106. b.responses = nil
  107. atomic.StoreInt32(&b.opened, 0)
  108. if err == nil {
  109. Logger.Printf("Closed connection to broker %s\n", b.addr)
  110. } else {
  111. Logger.Printf("Error while closing connection to broker %s: %s\n", b.addr, err)
  112. }
  113. return err
  114. }
  115. // ID returns the broker ID retrieved from Kafka's metadata, or -1 if that is not known.
  116. func (b *Broker) ID() int32 {
  117. return b.id
  118. }
  119. // Addr returns the broker address as either retrieved from Kafka's metadata or passed to NewBroker.
  120. func (b *Broker) Addr() string {
  121. return b.addr
  122. }
  123. func (b *Broker) GetMetadata(request *MetadataRequest) (*MetadataResponse, error) {
  124. response := new(MetadataResponse)
  125. err := b.sendAndReceive(request, response)
  126. if err != nil {
  127. return nil, err
  128. }
  129. return response, nil
  130. }
  131. func (b *Broker) GetConsumerMetadata(request *ConsumerMetadataRequest) (*ConsumerMetadataResponse, error) {
  132. response := new(ConsumerMetadataResponse)
  133. err := b.sendAndReceive(request, response)
  134. if err != nil {
  135. return nil, err
  136. }
  137. return response, nil
  138. }
  139. func (b *Broker) GetAvailableOffsets(request *OffsetRequest) (*OffsetResponse, error) {
  140. response := new(OffsetResponse)
  141. err := b.sendAndReceive(request, response)
  142. if err != nil {
  143. return nil, err
  144. }
  145. return response, nil
  146. }
  147. func (b *Broker) Produce(request *ProduceRequest) (*ProduceResponse, error) {
  148. var response *ProduceResponse
  149. var err error
  150. if request.RequiredAcks == NoResponse {
  151. err = b.sendAndReceive(request, nil)
  152. } else {
  153. response = new(ProduceResponse)
  154. err = b.sendAndReceive(request, response)
  155. }
  156. if err != nil {
  157. return nil, err
  158. }
  159. return response, nil
  160. }
  161. func (b *Broker) Fetch(request *FetchRequest) (*FetchResponse, error) {
  162. response := new(FetchResponse)
  163. err := b.sendAndReceive(request, response)
  164. if err != nil {
  165. return nil, err
  166. }
  167. return response, nil
  168. }
  169. func (b *Broker) CommitOffset(request *OffsetCommitRequest) (*OffsetCommitResponse, error) {
  170. response := new(OffsetCommitResponse)
  171. err := b.sendAndReceive(request, response)
  172. if err != nil {
  173. return nil, err
  174. }
  175. return response, nil
  176. }
  177. func (b *Broker) FetchOffset(request *OffsetFetchRequest) (*OffsetFetchResponse, error) {
  178. response := new(OffsetFetchResponse)
  179. err := b.sendAndReceive(request, response)
  180. if err != nil {
  181. return nil, err
  182. }
  183. return response, nil
  184. }
  185. func (b *Broker) JoinGroup(request *JoinGroupRequest) (*JoinGroupResponse, error) {
  186. response := new(JoinGroupResponse)
  187. err := b.sendAndReceive(request, response)
  188. if err != nil {
  189. return nil, err
  190. }
  191. return response, nil
  192. }
  193. func (b *Broker) SyncGroup(request *SyncGroupRequest) (*SyncGroupResponse, error) {
  194. response := new(SyncGroupResponse)
  195. err := b.sendAndReceive(request, response)
  196. if err != nil {
  197. return nil, err
  198. }
  199. return response, nil
  200. }
  201. func (b *Broker) LeaveGroup(request *LeaveGroupRequest) (*LeaveGroupResponse, error) {
  202. response := new(LeaveGroupResponse)
  203. err := b.sendAndReceive(request, response)
  204. if err != nil {
  205. return nil, err
  206. }
  207. return response, nil
  208. }
  209. func (b *Broker) Heartbeat(request *HeartbeatRequest) (*HeartbeatResponse, error) {
  210. response := new(HeartbeatResponse)
  211. err := b.sendAndReceive(request, response)
  212. if err != nil {
  213. return nil, err
  214. }
  215. return response, nil
  216. }
  217. func (b *Broker) ListGroups(request *ListGroupsRequest) (*ListGroupsResponse, error) {
  218. response := new(ListGroupsResponse)
  219. err := b.sendAndReceive(request, response)
  220. if err != nil {
  221. return nil, err
  222. }
  223. return response, nil
  224. }
  225. func (b *Broker) DescribeGroups(request *DescribeGroupsRequest) (*DescribeGroupsResponse, error) {
  226. response := new(DescribeGroupsResponse)
  227. err := b.sendAndReceive(request, response)
  228. if err != nil {
  229. return nil, err
  230. }
  231. return response, nil
  232. }
  233. func (b *Broker) send(rb requestBody, promiseResponse bool) (*responsePromise, error) {
  234. b.lock.Lock()
  235. defer b.lock.Unlock()
  236. if b.conn == nil {
  237. if b.connErr != nil {
  238. return nil, b.connErr
  239. }
  240. return nil, ErrNotConnected
  241. }
  242. req := &request{correlationID: b.correlationID, clientID: b.conf.ClientID, body: rb}
  243. buf, err := encode(req)
  244. if err != nil {
  245. return nil, err
  246. }
  247. err = b.conn.SetWriteDeadline(time.Now().Add(b.conf.Net.WriteTimeout))
  248. if err != nil {
  249. return nil, err
  250. }
  251. _, err = b.conn.Write(buf)
  252. if err != nil {
  253. return nil, err
  254. }
  255. b.correlationID++
  256. if !promiseResponse {
  257. return nil, nil
  258. }
  259. promise := responsePromise{req.correlationID, make(chan []byte), make(chan error)}
  260. b.responses <- promise
  261. return &promise, nil
  262. }
  263. func (b *Broker) sendAndReceive(req requestBody, res decoder) error {
  264. promise, err := b.send(req, res != nil)
  265. if err != nil {
  266. return err
  267. }
  268. if promise == nil {
  269. return nil
  270. }
  271. select {
  272. case buf := <-promise.packets:
  273. return decode(buf, res)
  274. case err = <-promise.errors:
  275. return err
  276. }
  277. }
  278. func (b *Broker) decode(pd packetDecoder) (err error) {
  279. b.id, err = pd.getInt32()
  280. if err != nil {
  281. return err
  282. }
  283. host, err := pd.getString()
  284. if err != nil {
  285. return err
  286. }
  287. port, err := pd.getInt32()
  288. if err != nil {
  289. return err
  290. }
  291. b.addr = net.JoinHostPort(host, fmt.Sprint(port))
  292. if _, _, err := net.SplitHostPort(b.addr); err != nil {
  293. return err
  294. }
  295. return nil
  296. }
  297. func (b *Broker) encode(pe packetEncoder) (err error) {
  298. host, portstr, err := net.SplitHostPort(b.addr)
  299. if err != nil {
  300. return err
  301. }
  302. port, err := strconv.Atoi(portstr)
  303. if err != nil {
  304. return err
  305. }
  306. pe.putInt32(b.id)
  307. err = pe.putString(host)
  308. if err != nil {
  309. return err
  310. }
  311. pe.putInt32(int32(port))
  312. return nil
  313. }
  314. func (b *Broker) responseReceiver() {
  315. var dead error
  316. header := make([]byte, 8)
  317. for response := range b.responses {
  318. if dead != nil {
  319. response.errors <- dead
  320. continue
  321. }
  322. err := b.conn.SetReadDeadline(time.Now().Add(b.conf.Net.ReadTimeout))
  323. if err != nil {
  324. dead = err
  325. response.errors <- err
  326. continue
  327. }
  328. _, err = io.ReadFull(b.conn, header)
  329. if err != nil {
  330. dead = err
  331. response.errors <- err
  332. continue
  333. }
  334. decodedHeader := responseHeader{}
  335. err = decode(header, &decodedHeader)
  336. if err != nil {
  337. dead = err
  338. response.errors <- err
  339. continue
  340. }
  341. if decodedHeader.correlationID != response.correlationID {
  342. // TODO if decoded ID < cur ID, discard until we catch up
  343. // TODO if decoded ID > cur ID, save it so when cur ID catches up we have a response
  344. dead = PacketDecodingError{fmt.Sprintf("correlation ID didn't match, wanted %d, got %d", response.correlationID, decodedHeader.correlationID)}
  345. response.errors <- dead
  346. continue
  347. }
  348. buf := make([]byte, decodedHeader.length-4)
  349. _, err = io.ReadFull(b.conn, buf)
  350. if err != nil {
  351. dead = err
  352. response.errors <- err
  353. continue
  354. }
  355. response.packets <- buf
  356. }
  357. close(b.done)
  358. }