broker.go 12 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522
  1. package sarama
  2. import (
  3. "crypto/tls"
  4. "encoding/binary"
  5. "fmt"
  6. "io"
  7. "net"
  8. "strconv"
  9. "sync"
  10. "sync/atomic"
  11. "time"
  12. )
  13. // Broker represents a single Kafka broker connection. All operations on this object are entirely concurrency-safe.
  14. type Broker struct {
  15. id int32
  16. addr string
  17. conf *Config
  18. correlationID int32
  19. conn net.Conn
  20. connErr error
  21. lock sync.Mutex
  22. opened int32
  23. responses chan responsePromise
  24. done chan bool
  25. }
  26. type responsePromise struct {
  27. correlationID int32
  28. packets chan []byte
  29. errors chan error
  30. }
  31. // NewBroker creates and returns a Broker targetting the given host:port address.
  32. // This does not attempt to actually connect, you have to call Open() for that.
  33. func NewBroker(addr string) *Broker {
  34. return &Broker{id: -1, addr: addr}
  35. }
  36. // Open tries to connect to the Broker if it is not already connected or connecting, but does not block
  37. // waiting for the connection to complete. This means that any subsequent operations on the broker will
  38. // block waiting for the connection to succeed or fail. To get the effect of a fully synchronous Open call,
  39. // follow it by a call to Connected(). The only errors Open will return directly are ConfigurationError or
  40. // AlreadyConnected. If conf is nil, the result of NewConfig() is used.
  41. func (b *Broker) Open(conf *Config) error {
  42. if !atomic.CompareAndSwapInt32(&b.opened, 0, 1) {
  43. return ErrAlreadyConnected
  44. }
  45. if conf == nil {
  46. conf = NewConfig()
  47. }
  48. err := conf.Validate()
  49. if err != nil {
  50. return err
  51. }
  52. b.lock.Lock()
  53. go withRecover(func() {
  54. defer b.lock.Unlock()
  55. dialer := net.Dialer{
  56. Timeout: conf.Net.DialTimeout,
  57. KeepAlive: conf.Net.KeepAlive,
  58. }
  59. if conf.Net.TLS.Enable {
  60. b.conn, b.connErr = tls.DialWithDialer(&dialer, "tcp", b.addr, conf.Net.TLS.Config)
  61. } else {
  62. b.conn, b.connErr = dialer.Dial("tcp", b.addr)
  63. }
  64. if b.connErr != nil {
  65. Logger.Printf("Failed to connect to broker %s: %s\n", b.addr, b.connErr)
  66. b.conn = nil
  67. atomic.StoreInt32(&b.opened, 0)
  68. return
  69. }
  70. b.conn = newBufConn(b.conn)
  71. b.conf = conf
  72. if conf.Net.SASL.Enable {
  73. b.connErr = b.sendAndReceiveSASLPlainAuth()
  74. if b.connErr != nil {
  75. err = b.conn.Close()
  76. if err == nil {
  77. Logger.Printf("Closed connection to broker %s\n", b.addr)
  78. } else {
  79. Logger.Printf("Error while closing connection to broker %s: %s\n", b.addr, err)
  80. }
  81. b.conn = nil
  82. atomic.StoreInt32(&b.opened, 0)
  83. return
  84. }
  85. }
  86. b.done = make(chan bool)
  87. b.responses = make(chan responsePromise, b.conf.Net.MaxOpenRequests-1)
  88. if b.id >= 0 {
  89. Logger.Printf("Connected to broker at %s (registered as #%d)\n", b.addr, b.id)
  90. } else {
  91. Logger.Printf("Connected to broker at %s (unregistered)\n", b.addr)
  92. }
  93. go withRecover(b.responseReceiver)
  94. })
  95. return nil
  96. }
  97. // Connected returns true if the broker is connected and false otherwise. If the broker is not
  98. // connected but it had tried to connect, the error from that connection attempt is also returned.
  99. func (b *Broker) Connected() (bool, error) {
  100. b.lock.Lock()
  101. defer b.lock.Unlock()
  102. return b.conn != nil, b.connErr
  103. }
  104. func (b *Broker) Close() error {
  105. b.lock.Lock()
  106. defer b.lock.Unlock()
  107. if b.conn == nil {
  108. return ErrNotConnected
  109. }
  110. close(b.responses)
  111. <-b.done
  112. err := b.conn.Close()
  113. b.conn = nil
  114. b.connErr = nil
  115. b.done = nil
  116. b.responses = nil
  117. if err == nil {
  118. Logger.Printf("Closed connection to broker %s\n", b.addr)
  119. } else {
  120. Logger.Printf("Error while closing connection to broker %s: %s\n", b.addr, err)
  121. }
  122. atomic.StoreInt32(&b.opened, 0)
  123. return err
  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(request *MetadataRequest) (*MetadataResponse, error) {
  134. response := new(MetadataResponse)
  135. err := b.sendAndReceive(request, response)
  136. if err != nil {
  137. return nil, err
  138. }
  139. return response, nil
  140. }
  141. func (b *Broker) GetConsumerMetadata(request *ConsumerMetadataRequest) (*ConsumerMetadataResponse, error) {
  142. response := new(ConsumerMetadataResponse)
  143. err := b.sendAndReceive(request, response)
  144. if err != nil {
  145. return nil, err
  146. }
  147. return response, nil
  148. }
  149. func (b *Broker) GetAvailableOffsets(request *OffsetRequest) (*OffsetResponse, error) {
  150. response := new(OffsetResponse)
  151. err := b.sendAndReceive(request, response)
  152. if err != nil {
  153. return nil, err
  154. }
  155. return response, nil
  156. }
  157. func (b *Broker) Produce(request *ProduceRequest) (*ProduceResponse, error) {
  158. var response *ProduceResponse
  159. var err error
  160. if request.RequiredAcks == NoResponse {
  161. err = b.sendAndReceive(request, nil)
  162. } else {
  163. response = new(ProduceResponse)
  164. err = b.sendAndReceive(request, response)
  165. }
  166. if err != nil {
  167. return nil, err
  168. }
  169. return response, nil
  170. }
  171. func (b *Broker) Fetch(request *FetchRequest) (*FetchResponse, error) {
  172. response := new(FetchResponse)
  173. err := b.sendAndReceive(request, response)
  174. if err != nil {
  175. return nil, err
  176. }
  177. return response, nil
  178. }
  179. func (b *Broker) CommitOffset(request *OffsetCommitRequest) (*OffsetCommitResponse, error) {
  180. response := new(OffsetCommitResponse)
  181. err := b.sendAndReceive(request, response)
  182. if err != nil {
  183. return nil, err
  184. }
  185. return response, nil
  186. }
  187. func (b *Broker) FetchOffset(request *OffsetFetchRequest) (*OffsetFetchResponse, error) {
  188. response := new(OffsetFetchResponse)
  189. err := b.sendAndReceive(request, response)
  190. if err != nil {
  191. return nil, err
  192. }
  193. return response, nil
  194. }
  195. func (b *Broker) JoinGroup(request *JoinGroupRequest) (*JoinGroupResponse, error) {
  196. response := new(JoinGroupResponse)
  197. err := b.sendAndReceive(request, response)
  198. if err != nil {
  199. return nil, err
  200. }
  201. return response, nil
  202. }
  203. func (b *Broker) SyncGroup(request *SyncGroupRequest) (*SyncGroupResponse, error) {
  204. response := new(SyncGroupResponse)
  205. err := b.sendAndReceive(request, response)
  206. if err != nil {
  207. return nil, err
  208. }
  209. return response, nil
  210. }
  211. func (b *Broker) LeaveGroup(request *LeaveGroupRequest) (*LeaveGroupResponse, error) {
  212. response := new(LeaveGroupResponse)
  213. err := b.sendAndReceive(request, response)
  214. if err != nil {
  215. return nil, err
  216. }
  217. return response, nil
  218. }
  219. func (b *Broker) Heartbeat(request *HeartbeatRequest) (*HeartbeatResponse, error) {
  220. response := new(HeartbeatResponse)
  221. err := b.sendAndReceive(request, response)
  222. if err != nil {
  223. return nil, err
  224. }
  225. return response, nil
  226. }
  227. func (b *Broker) ListGroups(request *ListGroupsRequest) (*ListGroupsResponse, error) {
  228. response := new(ListGroupsResponse)
  229. err := b.sendAndReceive(request, response)
  230. if err != nil {
  231. return nil, err
  232. }
  233. return response, nil
  234. }
  235. func (b *Broker) DescribeGroups(request *DescribeGroupsRequest) (*DescribeGroupsResponse, error) {
  236. response := new(DescribeGroupsResponse)
  237. err := b.sendAndReceive(request, response)
  238. if err != nil {
  239. return nil, err
  240. }
  241. return response, nil
  242. }
  243. func (b *Broker) send(rb requestBody, promiseResponse bool) (*responsePromise, error) {
  244. b.lock.Lock()
  245. defer b.lock.Unlock()
  246. if b.conn == nil {
  247. if b.connErr != nil {
  248. return nil, b.connErr
  249. }
  250. return nil, ErrNotConnected
  251. }
  252. req := &request{correlationID: b.correlationID, clientID: b.conf.ClientID, body: rb}
  253. buf, err := encode(req)
  254. if err != nil {
  255. return nil, err
  256. }
  257. err = b.conn.SetWriteDeadline(time.Now().Add(b.conf.Net.WriteTimeout))
  258. if err != nil {
  259. return nil, err
  260. }
  261. _, err = b.conn.Write(buf)
  262. if err != nil {
  263. return nil, err
  264. }
  265. b.correlationID++
  266. if !promiseResponse {
  267. return nil, nil
  268. }
  269. promise := responsePromise{req.correlationID, make(chan []byte), make(chan error)}
  270. b.responses <- promise
  271. return &promise, nil
  272. }
  273. func (b *Broker) sendAndReceive(req requestBody, res decoder) error {
  274. promise, err := b.send(req, res != nil)
  275. if err != nil {
  276. return err
  277. }
  278. if promise == nil {
  279. return nil
  280. }
  281. select {
  282. case buf := <-promise.packets:
  283. return decode(buf, res)
  284. case err = <-promise.errors:
  285. return err
  286. }
  287. }
  288. func (b *Broker) decode(pd packetDecoder) (err error) {
  289. b.id, err = pd.getInt32()
  290. if err != nil {
  291. return err
  292. }
  293. host, err := pd.getString()
  294. if err != nil {
  295. return err
  296. }
  297. port, err := pd.getInt32()
  298. if err != nil {
  299. return err
  300. }
  301. b.addr = net.JoinHostPort(host, fmt.Sprint(port))
  302. if _, _, err := net.SplitHostPort(b.addr); err != nil {
  303. return err
  304. }
  305. return nil
  306. }
  307. func (b *Broker) encode(pe packetEncoder) (err error) {
  308. host, portstr, err := net.SplitHostPort(b.addr)
  309. if err != nil {
  310. return err
  311. }
  312. port, err := strconv.Atoi(portstr)
  313. if err != nil {
  314. return err
  315. }
  316. pe.putInt32(b.id)
  317. err = pe.putString(host)
  318. if err != nil {
  319. return err
  320. }
  321. pe.putInt32(int32(port))
  322. return nil
  323. }
  324. func (b *Broker) responseReceiver() {
  325. var dead error
  326. header := make([]byte, 8)
  327. for response := range b.responses {
  328. if dead != nil {
  329. response.errors <- dead
  330. continue
  331. }
  332. err := b.conn.SetReadDeadline(time.Now().Add(b.conf.Net.ReadTimeout))
  333. if err != nil {
  334. dead = err
  335. response.errors <- err
  336. continue
  337. }
  338. _, err = io.ReadFull(b.conn, header)
  339. if err != nil {
  340. dead = err
  341. response.errors <- err
  342. continue
  343. }
  344. decodedHeader := responseHeader{}
  345. err = decode(header, &decodedHeader)
  346. if err != nil {
  347. dead = err
  348. response.errors <- err
  349. continue
  350. }
  351. if decodedHeader.correlationID != response.correlationID {
  352. // TODO if decoded ID < cur ID, discard until we catch up
  353. // TODO if decoded ID > cur ID, save it so when cur ID catches up we have a response
  354. dead = PacketDecodingError{fmt.Sprintf("correlation ID didn't match, wanted %d, got %d", response.correlationID, decodedHeader.correlationID)}
  355. response.errors <- dead
  356. continue
  357. }
  358. buf := make([]byte, decodedHeader.length-4)
  359. _, err = io.ReadFull(b.conn, buf)
  360. if err != nil {
  361. dead = err
  362. response.errors <- err
  363. continue
  364. }
  365. response.packets <- buf
  366. }
  367. close(b.done)
  368. }
  369. // Kafka 0.10.0 plans to support SASL Plain and Kerberos as per PR #812 (KIP-43)/(JIRA KAFKA-3149)
  370. // Some hosted kafka services such as IBM Message Hub already offer SASL/PLAIN auth with Kafka 0.9
  371. //
  372. // In SASL Plain, Kafka expects the auth header to be in the following format
  373. // Message format (from https://tools.ietf.org/html/rfc4616):
  374. //
  375. // message = [authzid] UTF8NUL authcid UTF8NUL passwd
  376. // authcid = 1*SAFE ; MUST accept up to 255 octets
  377. // authzid = 1*SAFE ; MUST accept up to 255 octets
  378. // passwd = 1*SAFE ; MUST accept up to 255 octets
  379. // UTF8NUL = %x00 ; UTF-8 encoded NUL character
  380. //
  381. // SAFE = UTF1 / UTF2 / UTF3 / UTF4
  382. // ;; any UTF-8 encoded Unicode character except NUL
  383. //
  384. // When credentials are valid, Kafka returns a 4 byte array of null characters.
  385. // When credentials are invalid, Kafka closes the connection. This does not seem to be the ideal way
  386. // of responding to bad credentials but thats how its being done today.
  387. func (b *Broker) sendAndReceiveSASLPlainAuth() error {
  388. length := 1 + len(b.conf.Net.SASL.User) + 1 + len(b.conf.Net.SASL.Password)
  389. authBytes := make([]byte, length+4) //4 byte length header + auth data
  390. binary.BigEndian.PutUint32(authBytes, uint32(length))
  391. copy(authBytes[4:], []byte("\x00"+b.conf.Net.SASL.User+"\x00"+b.conf.Net.SASL.Password))
  392. err := b.conn.SetWriteDeadline(time.Now().Add(b.conf.Net.WriteTimeout))
  393. if err != nil {
  394. Logger.Printf("Failed to set write deadline when doing SASL auth with broker %s: %s\n", b.addr, err.Error())
  395. return err
  396. }
  397. _, err = b.conn.Write(authBytes)
  398. if err != nil {
  399. Logger.Printf("Failed to write SASL auth header to broker %s: %s\n", b.addr, err.Error())
  400. return err
  401. }
  402. header := make([]byte, 4)
  403. n, err := io.ReadFull(b.conn, header)
  404. // If the credentials are valid, we would get a 4 byte response filled with null characters.
  405. // Otherwise, the broker closes the connection and we get an EOF
  406. if err != nil {
  407. Logger.Printf("Failed to read response while authenticating with SASL to broker %s: %s\n", b.addr, err.Error())
  408. return err
  409. }
  410. Logger.Printf("SASL authentication successful with broker %s:%v - %v\n", b.addr, n, header)
  411. return nil
  412. }