broker.go 9.3 KB

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