broker.go 9.5 KB

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