broker.go 9.4 KB

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