broker.go 8.0 KB

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