broker.go 9.0 KB

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