broker.go 8.4 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388
  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.conf = conf
  75. b.done = make(chan bool)
  76. b.responses = make(chan responsePromise, b.conf.Net.MaxOpenRequests-1)
  77. if b.id >= 0 {
  78. Logger.Printf("Connected to broker at %s (registered as #%d)\n", b.addr, b.id)
  79. } else {
  80. Logger.Printf("Connected to broker at %s (unregistered)\n", b.addr)
  81. }
  82. go withRecover(b.responseReceiver)
  83. })
  84. return nil
  85. }
  86. // Connected returns true if the broker is connected and false otherwise. If the broker is not
  87. // connected but it had tried to connect, the error from that connection attempt is also returned.
  88. func (b *Broker) Connected() (bool, error) {
  89. b.lock.Lock()
  90. defer b.lock.Unlock()
  91. return b.conn != nil, b.connErr
  92. }
  93. func (b *Broker) Close() error {
  94. b.lock.Lock()
  95. defer b.lock.Unlock()
  96. if b.conn == nil {
  97. return ErrNotConnected
  98. }
  99. close(b.responses)
  100. <-b.done
  101. err := b.conn.Close()
  102. b.conn = nil
  103. b.connErr = nil
  104. b.done = nil
  105. b.responses = nil
  106. atomic.StoreInt32(&b.opened, 0)
  107. if err == nil {
  108. Logger.Printf("Closed connection to broker %s\n", b.addr)
  109. } else {
  110. Logger.Printf("Error while closing connection to broker %s: %s\n", b.addr, err)
  111. }
  112. return err
  113. }
  114. // ID returns the broker ID retrieved from Kafka's metadata, or -1 if that is not known.
  115. func (b *Broker) ID() int32 {
  116. return b.id
  117. }
  118. // Addr returns the broker address as either retrieved from Kafka's metadata or passed to NewBroker.
  119. func (b *Broker) Addr() string {
  120. return b.addr
  121. }
  122. func (b *Broker) GetMetadata(request *MetadataRequest) (*MetadataResponse, error) {
  123. response := new(MetadataResponse)
  124. err := b.sendAndReceive(request, response)
  125. if err != nil {
  126. return nil, err
  127. }
  128. return response, nil
  129. }
  130. func (b *Broker) GetConsumerMetadata(request *ConsumerMetadataRequest) (*ConsumerMetadataResponse, error) {
  131. response := new(ConsumerMetadataResponse)
  132. err := b.sendAndReceive(request, response)
  133. if err != nil {
  134. return nil, err
  135. }
  136. return response, nil
  137. }
  138. func (b *Broker) GetAvailableOffsets(request *OffsetRequest) (*OffsetResponse, error) {
  139. response := new(OffsetResponse)
  140. err := b.sendAndReceive(request, response)
  141. if err != nil {
  142. return nil, err
  143. }
  144. return response, nil
  145. }
  146. func (b *Broker) Produce(request *ProduceRequest) (*ProduceResponse, error) {
  147. var response *ProduceResponse
  148. var err error
  149. if request.RequiredAcks == NoResponse {
  150. err = b.sendAndReceive(request, nil)
  151. } else {
  152. response = new(ProduceResponse)
  153. err = b.sendAndReceive(request, response)
  154. }
  155. if err != nil {
  156. return nil, err
  157. }
  158. return response, nil
  159. }
  160. func (b *Broker) Fetch(request *FetchRequest) (*FetchResponse, error) {
  161. response := new(FetchResponse)
  162. err := b.sendAndReceive(request, response)
  163. if err != nil {
  164. return nil, err
  165. }
  166. return response, nil
  167. }
  168. func (b *Broker) CommitOffset(request *OffsetCommitRequest) (*OffsetCommitResponse, error) {
  169. response := new(OffsetCommitResponse)
  170. err := b.sendAndReceive(request, response)
  171. if err != nil {
  172. return nil, err
  173. }
  174. return response, nil
  175. }
  176. func (b *Broker) FetchOffset(request *OffsetFetchRequest) (*OffsetFetchResponse, error) {
  177. response := new(OffsetFetchResponse)
  178. err := b.sendAndReceive(request, response)
  179. if err != nil {
  180. return nil, err
  181. }
  182. return response, nil
  183. }
  184. func (b *Broker) send(rb requestBody, promiseResponse bool) (*responsePromise, error) {
  185. b.lock.Lock()
  186. defer b.lock.Unlock()
  187. if b.conn == nil {
  188. if b.connErr != nil {
  189. return nil, b.connErr
  190. }
  191. return nil, ErrNotConnected
  192. }
  193. req := &request{correlationID: b.correlationID, clientID: b.conf.ClientID, body: rb}
  194. buf, err := encode(req)
  195. if err != nil {
  196. return nil, err
  197. }
  198. err = b.conn.SetWriteDeadline(time.Now().Add(b.conf.Net.WriteTimeout))
  199. if err != nil {
  200. return nil, err
  201. }
  202. _, err = b.conn.Write(buf)
  203. if err != nil {
  204. return nil, err
  205. }
  206. b.correlationID++
  207. if !promiseResponse {
  208. return nil, nil
  209. }
  210. promise := responsePromise{req.correlationID, make(chan []byte), make(chan error)}
  211. b.responses <- promise
  212. return &promise, nil
  213. }
  214. func (b *Broker) sendAndReceive(req requestBody, res decoder) error {
  215. promise, err := b.send(req, res != nil)
  216. if err != nil {
  217. return err
  218. }
  219. if promise == nil {
  220. return nil
  221. }
  222. select {
  223. case buf := <-promise.packets:
  224. return decode(buf, res)
  225. case err = <-promise.errors:
  226. return err
  227. }
  228. }
  229. func (b *Broker) decode(pd packetDecoder) (err error) {
  230. b.id, err = pd.getInt32()
  231. if err != nil {
  232. return err
  233. }
  234. host, err := pd.getString()
  235. if err != nil {
  236. return err
  237. }
  238. port, err := pd.getInt32()
  239. if err != nil {
  240. return err
  241. }
  242. b.addr = net.JoinHostPort(host, fmt.Sprint(port))
  243. if _, _, err := net.SplitHostPort(b.addr); err != nil {
  244. return err
  245. }
  246. return nil
  247. }
  248. func (b *Broker) encode(pe packetEncoder) (err error) {
  249. host, portstr, err := net.SplitHostPort(b.addr)
  250. if err != nil {
  251. return err
  252. }
  253. port, err := strconv.Atoi(portstr)
  254. if err != nil {
  255. return err
  256. }
  257. pe.putInt32(b.id)
  258. err = pe.putString(host)
  259. if err != nil {
  260. return err
  261. }
  262. pe.putInt32(int32(port))
  263. return nil
  264. }
  265. func (b *Broker) responseReceiver() {
  266. header := make([]byte, 8)
  267. for response := range b.responses {
  268. err := b.conn.SetReadDeadline(time.Now().Add(b.conf.Net.ReadTimeout))
  269. if err != nil {
  270. response.errors <- err
  271. continue
  272. }
  273. _, err = io.ReadFull(b.conn, header)
  274. if err != nil {
  275. response.errors <- err
  276. continue
  277. }
  278. decodedHeader := responseHeader{}
  279. err = decode(header, &decodedHeader)
  280. if err != nil {
  281. response.errors <- err
  282. continue
  283. }
  284. if decodedHeader.correlationID != response.correlationID {
  285. // TODO if decoded ID < cur ID, discard until we catch up
  286. // TODO if decoded ID > cur ID, save it so when cur ID catches up we have a response
  287. response.errors <- PacketDecodingError{fmt.Sprintf("correlation ID didn't match, wanted %d, got %d", response.correlationID, decodedHeader.correlationID)}
  288. continue
  289. }
  290. buf := make([]byte, decodedHeader.length-4)
  291. _, err = io.ReadFull(b.conn, buf)
  292. if err != nil {
  293. // XXX: the above ReadFull call inherits the same ReadDeadline set at the top of this loop, so it may
  294. // fail with a timeout error. If this happens, our connection is permanently toast since we will no longer
  295. // be aligned correctly on the stream (we'll be reading garbage Kafka headers from the middle of data).
  296. // Can we/should we fail harder in that case?
  297. response.errors <- err
  298. continue
  299. }
  300. response.packets <- buf
  301. }
  302. close(b.done)
  303. }