github.com/rbisecke/kafka-go@v0.4.27/conn.go (about) 1 package kafka 2 3 import ( 4 "bufio" 5 "errors" 6 "fmt" 7 "io" 8 "math" 9 "net" 10 "os" 11 "path/filepath" 12 "runtime" 13 "sync" 14 "sync/atomic" 15 "time" 16 ) 17 18 var ( 19 errInvalidWriteTopic = errors.New("writes must NOT set Topic on kafka.Message") 20 errInvalidWritePartition = errors.New("writes must NOT set Partition on kafka.Message") 21 ) 22 23 // Conn represents a connection to a kafka broker. 24 // 25 // Instances of Conn are safe to use concurrently from multiple goroutines. 26 type Conn struct { 27 // base network connection 28 conn net.Conn 29 30 // number of inflight requests on the connection. 31 inflight int32 32 33 // offset management (synchronized on the mutex field) 34 mutex sync.Mutex 35 offset int64 36 37 // read buffer (synchronized on rlock) 38 rlock sync.Mutex 39 rbuf bufio.Reader 40 41 // write buffer (synchronized on wlock) 42 wlock sync.Mutex 43 wbuf bufio.Writer 44 wb writeBuffer 45 46 // deadline management 47 wdeadline connDeadline 48 rdeadline connDeadline 49 50 // immutable values of the connection object 51 clientID string 52 topic string 53 partition int32 54 fetchMaxBytes int32 55 fetchMinSize int32 56 broker int32 57 rack string 58 59 // correlation ID generator (synchronized on wlock) 60 correlationID int32 61 62 // number of replica acks required when publishing to a partition 63 requiredAcks int32 64 65 // lazily loaded API versions used by this connection 66 apiVersions atomic.Value // apiVersionMap 67 68 transactionalID *string 69 } 70 71 type apiVersionMap map[apiKey]ApiVersion 72 73 func (v apiVersionMap) negotiate(key apiKey, sortedSupportedVersions ...apiVersion) apiVersion { 74 x := v[key] 75 76 for i := len(sortedSupportedVersions) - 1; i >= 0; i-- { 77 s := sortedSupportedVersions[i] 78 79 if apiVersion(x.MaxVersion) >= s { 80 return s 81 } 82 } 83 84 return -1 85 } 86 87 // ConnConfig is a configuration object used to create new instances of Conn. 88 type ConnConfig struct { 89 ClientID string 90 Topic string 91 Partition int 92 Broker int 93 Rack string 94 95 // The transactional id to use for transactional delivery. Idempotent 96 // deliver should be enabled if transactional id is configured. 97 // For more details look at transactional.id description here: http://kafka.apache.org/documentation.html#producerconfigs 98 // Empty string means that this connection can't be transactional. 99 TransactionalID string 100 } 101 102 // ReadBatchConfig is a configuration object used for reading batches of messages. 103 type ReadBatchConfig struct { 104 // MinBytes indicates to the broker the minimum batch size that the consumer 105 // will accept. Setting a high minimum when consuming from a low-volume topic 106 // may result in delayed delivery when the broker does not have enough data to 107 // satisfy the defined minimum. 108 MinBytes int 109 110 // MaxBytes indicates to the broker the maximum batch size that the consumer 111 // will accept. The broker will truncate a message to satisfy this maximum, so 112 // choose a value that is high enough for your largest message size. 113 MaxBytes int 114 115 // IsolationLevel controls the visibility of transactional records. 116 // ReadUncommitted makes all records visible. With ReadCommitted only 117 // non-transactional and committed records are visible. 118 IsolationLevel IsolationLevel 119 120 // MaxWait is the amount of time for the broker while waiting to hit the 121 // min/max byte targets. This setting is independent of any network-level 122 // timeouts or deadlines. 123 // 124 // For backward compatibility, when this field is left zero, kafka-go will 125 // infer the max wait from the connection's read deadline. 126 MaxWait time.Duration 127 } 128 129 type IsolationLevel int8 130 131 const ( 132 ReadUncommitted IsolationLevel = 0 133 ReadCommitted IsolationLevel = 1 134 ) 135 136 // DefaultClientID is the default value used as ClientID of kafka 137 // connections. 138 var DefaultClientID string 139 140 func init() { 141 progname := filepath.Base(os.Args[0]) 142 hostname, _ := os.Hostname() 143 DefaultClientID = fmt.Sprintf("%s@%s (github.com/rbisecke/kafka-go)", progname, hostname) 144 } 145 146 // NewConn returns a new kafka connection for the given topic and partition. 147 func NewConn(conn net.Conn, topic string, partition int) *Conn { 148 return NewConnWith(conn, ConnConfig{ 149 Topic: topic, 150 Partition: partition, 151 }) 152 } 153 154 func emptyToNullable(transactionalID string) (result *string) { 155 if transactionalID != "" { 156 result = &transactionalID 157 } 158 return result 159 } 160 161 // NewConnWith returns a new kafka connection configured with config. 162 // The offset is initialized to FirstOffset. 163 func NewConnWith(conn net.Conn, config ConnConfig) *Conn { 164 if len(config.ClientID) == 0 { 165 config.ClientID = DefaultClientID 166 } 167 168 if config.Partition < 0 || config.Partition > math.MaxInt32 { 169 panic(fmt.Sprintf("invalid partition number: %d", config.Partition)) 170 } 171 172 c := &Conn{ 173 conn: conn, 174 rbuf: *bufio.NewReader(conn), 175 wbuf: *bufio.NewWriter(conn), 176 clientID: config.ClientID, 177 topic: config.Topic, 178 partition: int32(config.Partition), 179 broker: int32(config.Broker), 180 rack: config.Rack, 181 offset: FirstOffset, 182 requiredAcks: -1, 183 transactionalID: emptyToNullable(config.TransactionalID), 184 } 185 186 c.wb.w = &c.wbuf 187 188 // The fetch request needs to ask for a MaxBytes value that is at least 189 // enough to load the control data of the response. To avoid having to 190 // recompute it on every read, it is cached here in the Conn value. 191 c.fetchMinSize = (fetchResponseV2{ 192 Topics: []fetchResponseTopicV2{{ 193 TopicName: config.Topic, 194 Partitions: []fetchResponsePartitionV2{{ 195 Partition: int32(config.Partition), 196 MessageSet: messageSet{{}}, 197 }}, 198 }}, 199 }).size() 200 c.fetchMaxBytes = math.MaxInt32 - c.fetchMinSize 201 return c 202 } 203 204 func (c *Conn) negotiateVersion(key apiKey, sortedSupportedVersions ...apiVersion) (apiVersion, error) { 205 v, err := c.loadVersions() 206 if err != nil { 207 return -1, err 208 } 209 a := v.negotiate(key, sortedSupportedVersions...) 210 if a < 0 { 211 return -1, fmt.Errorf("no matching versions were found between the client and the broker for API key %d", key) 212 } 213 return a, nil 214 } 215 216 func (c *Conn) loadVersions() (apiVersionMap, error) { 217 v, _ := c.apiVersions.Load().(apiVersionMap) 218 if v != nil { 219 return v, nil 220 } 221 222 brokerVersions, err := c.ApiVersions() 223 if err != nil { 224 return nil, err 225 } 226 227 v = make(apiVersionMap, len(brokerVersions)) 228 229 for _, a := range brokerVersions { 230 v[apiKey(a.ApiKey)] = a 231 } 232 233 c.apiVersions.Store(v) 234 return v, nil 235 } 236 237 // Broker returns a Broker value representing the kafka broker that this 238 // connection was established to. 239 func (c *Conn) Broker() Broker { 240 addr := c.conn.RemoteAddr() 241 host, port, _ := splitHostPortNumber(addr.String()) 242 return Broker{ 243 Host: host, 244 Port: port, 245 ID: int(c.broker), 246 Rack: c.rack, 247 } 248 } 249 250 // Controller requests kafka for the current controller and returns its URL 251 func (c *Conn) Controller() (broker Broker, err error) { 252 err = c.readOperation( 253 func(deadline time.Time, id int32) error { 254 return c.writeRequest(metadata, v1, id, topicMetadataRequestV1([]string{})) 255 }, 256 func(deadline time.Time, size int) error { 257 var res metadataResponseV1 258 259 if err := c.readResponse(size, &res); err != nil { 260 return err 261 } 262 for _, brokerMeta := range res.Brokers { 263 if brokerMeta.NodeID == res.ControllerID { 264 broker = Broker{ 265 ID: int(brokerMeta.NodeID), 266 Port: int(brokerMeta.Port), 267 Host: brokerMeta.Host, 268 Rack: brokerMeta.Rack, 269 } 270 break 271 } 272 } 273 return nil 274 }, 275 ) 276 return broker, err 277 } 278 279 // Brokers retrieve the broker list from the Kafka metadata 280 func (c *Conn) Brokers() ([]Broker, error) { 281 var brokers []Broker 282 err := c.readOperation( 283 func(deadline time.Time, id int32) error { 284 return c.writeRequest(metadata, v1, id, topicMetadataRequestV1([]string{})) 285 }, 286 func(deadline time.Time, size int) error { 287 var res metadataResponseV1 288 289 if err := c.readResponse(size, &res); err != nil { 290 return err 291 } 292 293 brokers = make([]Broker, len(res.Brokers)) 294 for i, brokerMeta := range res.Brokers { 295 brokers[i] = Broker{ 296 ID: int(brokerMeta.NodeID), 297 Port: int(brokerMeta.Port), 298 Host: brokerMeta.Host, 299 Rack: brokerMeta.Rack, 300 } 301 } 302 return nil 303 }, 304 ) 305 return brokers, err 306 } 307 308 // DeleteTopics deletes the specified topics. 309 func (c *Conn) DeleteTopics(topics ...string) error { 310 _, err := c.deleteTopics(deleteTopicsRequestV0{ 311 Topics: topics, 312 }) 313 return err 314 } 315 316 // findCoordinator finds the coordinator for the specified group or transaction 317 // 318 // See http://kafka.apache.org/protocol.html#The_Messages_FindCoordinator 319 func (c *Conn) findCoordinator(request findCoordinatorRequestV0) (findCoordinatorResponseV0, error) { 320 var response findCoordinatorResponseV0 321 322 err := c.readOperation( 323 func(deadline time.Time, id int32) error { 324 return c.writeRequest(findCoordinator, v0, id, request) 325 }, 326 func(deadline time.Time, size int) error { 327 return expectZeroSize(func() (remain int, err error) { 328 return (&response).readFrom(&c.rbuf, size) 329 }()) 330 }, 331 ) 332 if err != nil { 333 return findCoordinatorResponseV0{}, err 334 } 335 if response.ErrorCode != 0 { 336 return findCoordinatorResponseV0{}, Error(response.ErrorCode) 337 } 338 339 return response, nil 340 } 341 342 // heartbeat sends a heartbeat message required by consumer groups 343 // 344 // See http://kafka.apache.org/protocol.html#The_Messages_Heartbeat 345 func (c *Conn) heartbeat(request heartbeatRequestV0) (heartbeatResponseV0, error) { 346 var response heartbeatResponseV0 347 348 err := c.writeOperation( 349 func(deadline time.Time, id int32) error { 350 return c.writeRequest(heartbeat, v0, id, request) 351 }, 352 func(deadline time.Time, size int) error { 353 return expectZeroSize(func() (remain int, err error) { 354 return (&response).readFrom(&c.rbuf, size) 355 }()) 356 }, 357 ) 358 if err != nil { 359 return heartbeatResponseV0{}, err 360 } 361 if response.ErrorCode != 0 { 362 return heartbeatResponseV0{}, Error(response.ErrorCode) 363 } 364 365 return response, nil 366 } 367 368 // joinGroup attempts to join a consumer group 369 // 370 // See http://kafka.apache.org/protocol.html#The_Messages_JoinGroup 371 func (c *Conn) joinGroup(request joinGroupRequestV1) (joinGroupResponseV1, error) { 372 var response joinGroupResponseV1 373 374 err := c.writeOperation( 375 func(deadline time.Time, id int32) error { 376 return c.writeRequest(joinGroup, v1, id, request) 377 }, 378 func(deadline time.Time, size int) error { 379 return expectZeroSize(func() (remain int, err error) { 380 return (&response).readFrom(&c.rbuf, size) 381 }()) 382 }, 383 ) 384 if err != nil { 385 return joinGroupResponseV1{}, err 386 } 387 if response.ErrorCode != 0 { 388 return joinGroupResponseV1{}, Error(response.ErrorCode) 389 } 390 391 return response, nil 392 } 393 394 // leaveGroup leaves the consumer from the consumer group 395 // 396 // See http://kafka.apache.org/protocol.html#The_Messages_LeaveGroup 397 func (c *Conn) leaveGroup(request leaveGroupRequestV0) (leaveGroupResponseV0, error) { 398 var response leaveGroupResponseV0 399 400 err := c.writeOperation( 401 func(deadline time.Time, id int32) error { 402 return c.writeRequest(leaveGroup, v0, id, request) 403 }, 404 func(deadline time.Time, size int) error { 405 return expectZeroSize(func() (remain int, err error) { 406 return (&response).readFrom(&c.rbuf, size) 407 }()) 408 }, 409 ) 410 if err != nil { 411 return leaveGroupResponseV0{}, err 412 } 413 if response.ErrorCode != 0 { 414 return leaveGroupResponseV0{}, Error(response.ErrorCode) 415 } 416 417 return response, nil 418 } 419 420 // listGroups lists all the consumer groups 421 // 422 // See http://kafka.apache.org/protocol.html#The_Messages_ListGroups 423 func (c *Conn) listGroups(request listGroupsRequestV1) (listGroupsResponseV1, error) { 424 var response listGroupsResponseV1 425 426 err := c.readOperation( 427 func(deadline time.Time, id int32) error { 428 return c.writeRequest(listGroups, v1, id, request) 429 }, 430 func(deadline time.Time, size int) error { 431 return expectZeroSize(func() (remain int, err error) { 432 return (&response).readFrom(&c.rbuf, size) 433 }()) 434 }, 435 ) 436 if err != nil { 437 return listGroupsResponseV1{}, err 438 } 439 if response.ErrorCode != 0 { 440 return listGroupsResponseV1{}, Error(response.ErrorCode) 441 } 442 443 return response, nil 444 } 445 446 // offsetCommit commits the specified topic partition offsets 447 // 448 // See http://kafka.apache.org/protocol.html#The_Messages_OffsetCommit 449 func (c *Conn) offsetCommit(request offsetCommitRequestV2) (offsetCommitResponseV2, error) { 450 var response offsetCommitResponseV2 451 452 err := c.writeOperation( 453 func(deadline time.Time, id int32) error { 454 return c.writeRequest(offsetCommit, v2, id, request) 455 }, 456 func(deadline time.Time, size int) error { 457 return expectZeroSize(func() (remain int, err error) { 458 return (&response).readFrom(&c.rbuf, size) 459 }()) 460 }, 461 ) 462 if err != nil { 463 return offsetCommitResponseV2{}, err 464 } 465 for _, r := range response.Responses { 466 for _, pr := range r.PartitionResponses { 467 if pr.ErrorCode != 0 { 468 return offsetCommitResponseV2{}, Error(pr.ErrorCode) 469 } 470 } 471 } 472 473 return response, nil 474 } 475 476 // offsetFetch fetches the offsets for the specified topic partitions. 477 // -1 indicates that there is no offset saved for the partition. 478 // 479 // See http://kafka.apache.org/protocol.html#The_Messages_OffsetFetch 480 func (c *Conn) offsetFetch(request offsetFetchRequestV1) (offsetFetchResponseV1, error) { 481 var response offsetFetchResponseV1 482 483 err := c.readOperation( 484 func(deadline time.Time, id int32) error { 485 return c.writeRequest(offsetFetch, v1, id, request) 486 }, 487 func(deadline time.Time, size int) error { 488 return expectZeroSize(func() (remain int, err error) { 489 return (&response).readFrom(&c.rbuf, size) 490 }()) 491 }, 492 ) 493 if err != nil { 494 return offsetFetchResponseV1{}, err 495 } 496 for _, r := range response.Responses { 497 for _, pr := range r.PartitionResponses { 498 if pr.ErrorCode != 0 { 499 return offsetFetchResponseV1{}, Error(pr.ErrorCode) 500 } 501 } 502 } 503 504 return response, nil 505 } 506 507 // syncGroup completes the handshake to join a consumer group 508 // 509 // See http://kafka.apache.org/protocol.html#The_Messages_SyncGroup 510 func (c *Conn) syncGroup(request syncGroupRequestV0) (syncGroupResponseV0, error) { 511 var response syncGroupResponseV0 512 513 err := c.readOperation( 514 func(deadline time.Time, id int32) error { 515 return c.writeRequest(syncGroup, v0, id, request) 516 }, 517 func(deadline time.Time, size int) error { 518 return expectZeroSize(func() (remain int, err error) { 519 return (&response).readFrom(&c.rbuf, size) 520 }()) 521 }, 522 ) 523 if err != nil { 524 return syncGroupResponseV0{}, err 525 } 526 if response.ErrorCode != 0 { 527 return syncGroupResponseV0{}, Error(response.ErrorCode) 528 } 529 530 return response, nil 531 } 532 533 // Close closes the kafka connection. 534 func (c *Conn) Close() error { 535 return c.conn.Close() 536 } 537 538 // LocalAddr returns the local network address. 539 func (c *Conn) LocalAddr() net.Addr { 540 return c.conn.LocalAddr() 541 } 542 543 // RemoteAddr returns the remote network address. 544 func (c *Conn) RemoteAddr() net.Addr { 545 return c.conn.RemoteAddr() 546 } 547 548 // SetDeadline sets the read and write deadlines associated with the connection. 549 // It is equivalent to calling both SetReadDeadline and SetWriteDeadline. 550 // 551 // A deadline is an absolute time after which I/O operations fail with a timeout 552 // (see type Error) instead of blocking. The deadline applies to all future and 553 // pending I/O, not just the immediately following call to Read or Write. After 554 // a deadline has been exceeded, the connection may be closed if it was found to 555 // be in an unrecoverable state. 556 // 557 // A zero value for t means I/O operations will not time out. 558 func (c *Conn) SetDeadline(t time.Time) error { 559 c.rdeadline.setDeadline(t) 560 c.wdeadline.setDeadline(t) 561 return nil 562 } 563 564 // SetReadDeadline sets the deadline for future Read calls and any 565 // currently-blocked Read call. 566 // A zero value for t means Read will not time out. 567 func (c *Conn) SetReadDeadline(t time.Time) error { 568 c.rdeadline.setDeadline(t) 569 return nil 570 } 571 572 // SetWriteDeadline sets the deadline for future Write calls and any 573 // currently-blocked Write call. 574 // Even if write times out, it may return n > 0, indicating that some of the 575 // data was successfully written. 576 // A zero value for t means Write will not time out. 577 func (c *Conn) SetWriteDeadline(t time.Time) error { 578 c.wdeadline.setDeadline(t) 579 return nil 580 } 581 582 // Offset returns the current offset of the connection as pair of integers, 583 // where the first one is an offset value and the second one indicates how 584 // to interpret it. 585 // 586 // See Seek for more details about the offset and whence values. 587 func (c *Conn) Offset() (offset int64, whence int) { 588 c.mutex.Lock() 589 offset = c.offset 590 c.mutex.Unlock() 591 592 switch offset { 593 case FirstOffset: 594 offset = 0 595 whence = SeekStart 596 case LastOffset: 597 offset = 0 598 whence = SeekEnd 599 default: 600 whence = SeekAbsolute 601 } 602 return 603 } 604 605 const ( 606 SeekStart = 0 // Seek relative to the first offset available in the partition. 607 SeekAbsolute = 1 // Seek to an absolute offset. 608 SeekEnd = 2 // Seek relative to the last offset available in the partition. 609 SeekCurrent = 3 // Seek relative to the current offset. 610 611 // This flag may be combined to any of the SeekAbsolute and SeekCurrent 612 // constants to skip the bound check that the connection would do otherwise. 613 // Programs can use this flag to avoid making a metadata request to the kafka 614 // broker to read the current first and last offsets of the partition. 615 SeekDontCheck = 1 << 30 616 ) 617 618 // Seek sets the offset for the next read or write operation according to whence, which 619 // should be one of SeekStart, SeekAbsolute, SeekEnd, or SeekCurrent. 620 // When seeking relative to the end, the offset is subtracted from the current offset. 621 // Note that for historical reasons, these do not align with the usual whence constants 622 // as in lseek(2) or os.Seek. 623 // The method returns the new absolute offset of the connection. 624 func (c *Conn) Seek(offset int64, whence int) (int64, error) { 625 seekDontCheck := (whence & SeekDontCheck) != 0 626 whence &= ^SeekDontCheck 627 628 switch whence { 629 case SeekStart, SeekAbsolute, SeekEnd, SeekCurrent: 630 default: 631 return 0, fmt.Errorf("whence must be one of 0, 1, 2, or 3. (whence = %d)", whence) 632 } 633 634 if seekDontCheck { 635 if whence == SeekAbsolute { 636 c.mutex.Lock() 637 c.offset = offset 638 c.mutex.Unlock() 639 return offset, nil 640 } 641 642 if whence == SeekCurrent { 643 c.mutex.Lock() 644 c.offset += offset 645 offset = c.offset 646 c.mutex.Unlock() 647 return offset, nil 648 } 649 } 650 651 if whence == SeekAbsolute { 652 c.mutex.Lock() 653 unchanged := offset == c.offset 654 c.mutex.Unlock() 655 if unchanged { 656 return offset, nil 657 } 658 } 659 660 if whence == SeekCurrent { 661 c.mutex.Lock() 662 offset = c.offset + offset 663 c.mutex.Unlock() 664 } 665 666 first, last, err := c.ReadOffsets() 667 if err != nil { 668 return 0, err 669 } 670 671 switch whence { 672 case SeekStart: 673 offset = first + offset 674 case SeekEnd: 675 offset = last - offset 676 } 677 678 if offset < first || offset > last { 679 return 0, OffsetOutOfRange 680 } 681 682 c.mutex.Lock() 683 c.offset = offset 684 c.mutex.Unlock() 685 return offset, nil 686 } 687 688 // Read reads the message at the current offset from the connection, advancing 689 // the offset on success so the next call to a read method will produce the next 690 // message. 691 // The method returns the number of bytes read, or an error if something went 692 // wrong. 693 // 694 // While it is safe to call Read concurrently from multiple goroutines it may 695 // be hard for the program to predict the results as the connection offset will 696 // be read and written by multiple goroutines, they could read duplicates, or 697 // messages may be seen by only some of the goroutines. 698 // 699 // The method fails with io.ErrShortBuffer if the buffer passed as argument is 700 // too small to hold the message value. 701 // 702 // This method is provided to satisfy the net.Conn interface but is much less 703 // efficient than using the more general purpose ReadBatch method. 704 func (c *Conn) Read(b []byte) (int, error) { 705 batch := c.ReadBatch(1, len(b)) 706 n, err := batch.Read(b) 707 return n, coalesceErrors(silentEOF(err), batch.Close()) 708 } 709 710 // ReadMessage reads the message at the current offset from the connection, 711 // advancing the offset on success so the next call to a read method will 712 // produce the next message. 713 // 714 // Because this method allocate memory buffers for the message key and value 715 // it is less memory-efficient than Read, but has the advantage of never 716 // failing with io.ErrShortBuffer. 717 // 718 // While it is safe to call Read concurrently from multiple goroutines it may 719 // be hard for the program to predict the results as the connection offset will 720 // be read and written by multiple goroutines, they could read duplicates, or 721 // messages may be seen by only some of the goroutines. 722 // 723 // This method is provided for convenience purposes but is much less efficient 724 // than using the more general purpose ReadBatch method. 725 func (c *Conn) ReadMessage(maxBytes int) (Message, error) { 726 batch := c.ReadBatch(1, maxBytes) 727 msg, err := batch.ReadMessage() 728 return msg, coalesceErrors(silentEOF(err), batch.Close()) 729 } 730 731 // ReadBatch reads a batch of messages from the kafka server. The method always 732 // returns a non-nil Batch value. If an error occurred, either sending the fetch 733 // request or reading the response, the error will be made available by the 734 // returned value of the batch's Close method. 735 // 736 // While it is safe to call ReadBatch concurrently from multiple goroutines it 737 // may be hard for the program to predict the results as the connection offset 738 // will be read and written by multiple goroutines, they could read duplicates, 739 // or messages may be seen by only some of the goroutines. 740 // 741 // A program doesn't specify the number of messages in wants from a batch, but 742 // gives the minimum and maximum number of bytes that it wants to receive from 743 // the kafka server. 744 func (c *Conn) ReadBatch(minBytes, maxBytes int) *Batch { 745 return c.ReadBatchWith(ReadBatchConfig{ 746 MinBytes: minBytes, 747 MaxBytes: maxBytes, 748 }) 749 } 750 751 // ReadBatchWith in every way is similar to ReadBatch. ReadBatch is configured 752 // with the default values in ReadBatchConfig except for minBytes and maxBytes. 753 func (c *Conn) ReadBatchWith(cfg ReadBatchConfig) *Batch { 754 var adjustedDeadline time.Time 755 maxFetch := int(c.fetchMaxBytes) 756 757 if cfg.MinBytes < 0 || cfg.MinBytes > maxFetch { 758 return &Batch{err: fmt.Errorf("kafka.(*Conn).ReadBatch: minBytes of %d out of [1,%d] bounds", cfg.MinBytes, maxFetch)} 759 } 760 if cfg.MaxBytes < 0 || cfg.MaxBytes > maxFetch { 761 return &Batch{err: fmt.Errorf("kafka.(*Conn).ReadBatch: maxBytes of %d out of [1,%d] bounds", cfg.MaxBytes, maxFetch)} 762 } 763 if cfg.MinBytes > cfg.MaxBytes { 764 return &Batch{err: fmt.Errorf("kafka.(*Conn).ReadBatch: minBytes (%d) > maxBytes (%d)", cfg.MinBytes, cfg.MaxBytes)} 765 } 766 767 offset, whence := c.Offset() 768 769 offset, err := c.Seek(offset, whence|SeekDontCheck) 770 if err != nil { 771 return &Batch{err: dontExpectEOF(err)} 772 } 773 774 fetchVersion, err := c.negotiateVersion(fetch, v2, v5, v10) 775 if err != nil { 776 return &Batch{err: dontExpectEOF(err)} 777 } 778 779 id, err := c.doRequest(&c.rdeadline, func(deadline time.Time, id int32) error { 780 now := time.Now() 781 var timeout time.Duration 782 if cfg.MaxWait > 0 { 783 // explicitly-configured case: no changes are made to the deadline, 784 // and the timeout is sent exactly as specified. 785 timeout = cfg.MaxWait 786 } else { 787 // default case: use the original logic to adjust the conn's 788 // deadline.T 789 deadline = adjustDeadlineForRTT(deadline, now, defaultRTT) 790 timeout = deadlineToTimeout(deadline, now) 791 } 792 // save this variable outside of the closure for later use in detecting 793 // truncated messages. 794 adjustedDeadline = deadline 795 switch fetchVersion { 796 case v10: 797 return c.wb.writeFetchRequestV10( 798 id, 799 c.clientID, 800 c.topic, 801 c.partition, 802 offset, 803 cfg.MinBytes, 804 cfg.MaxBytes+int(c.fetchMinSize), 805 timeout, 806 int8(cfg.IsolationLevel), 807 ) 808 case v5: 809 return c.wb.writeFetchRequestV5( 810 id, 811 c.clientID, 812 c.topic, 813 c.partition, 814 offset, 815 cfg.MinBytes, 816 cfg.MaxBytes+int(c.fetchMinSize), 817 timeout, 818 int8(cfg.IsolationLevel), 819 ) 820 default: 821 return c.wb.writeFetchRequestV2( 822 id, 823 c.clientID, 824 c.topic, 825 c.partition, 826 offset, 827 cfg.MinBytes, 828 cfg.MaxBytes+int(c.fetchMinSize), 829 timeout, 830 ) 831 } 832 }) 833 if err != nil { 834 return &Batch{err: dontExpectEOF(err)} 835 } 836 837 _, size, lock, err := c.waitResponse(&c.rdeadline, id) 838 if err != nil { 839 return &Batch{err: dontExpectEOF(err)} 840 } 841 842 var throttle int32 843 var highWaterMark int64 844 var remain int 845 846 switch fetchVersion { 847 case v10: 848 throttle, highWaterMark, remain, err = readFetchResponseHeaderV10(&c.rbuf, size) 849 case v5: 850 throttle, highWaterMark, remain, err = readFetchResponseHeaderV5(&c.rbuf, size) 851 default: 852 throttle, highWaterMark, remain, err = readFetchResponseHeaderV2(&c.rbuf, size) 853 } 854 if err == errShortRead { 855 err = checkTimeoutErr(adjustedDeadline) 856 } 857 858 var msgs *messageSetReader 859 if err == nil { 860 msgs, err = newMessageSetReader(&c.rbuf, remain) 861 } 862 if err == errShortRead { 863 err = checkTimeoutErr(adjustedDeadline) 864 } 865 return &Batch{ 866 conn: c, 867 msgs: msgs, 868 deadline: adjustedDeadline, 869 throttle: makeDuration(throttle), 870 lock: lock, 871 topic: c.topic, // topic is copied to Batch to prevent race with Batch.close 872 partition: int(c.partition), // partition is copied to Batch to prevent race with Batch.close 873 offset: offset, 874 highWaterMark: highWaterMark, 875 // there shouldn't be a short read on initially setting up the batch. 876 // as such, any io.EOF is re-mapped to an io.ErrUnexpectedEOF so that we 877 // don't accidentally signal that we successfully reached the end of the 878 // batch. 879 err: dontExpectEOF(err), 880 } 881 } 882 883 // ReadOffset returns the offset of the first message with a timestamp equal or 884 // greater to t. 885 func (c *Conn) ReadOffset(t time.Time) (int64, error) { 886 return c.readOffset(timestamp(t)) 887 } 888 889 // ReadFirstOffset returns the first offset available on the connection. 890 func (c *Conn) ReadFirstOffset() (int64, error) { 891 return c.readOffset(FirstOffset) 892 } 893 894 // ReadLastOffset returns the last offset available on the connection. 895 func (c *Conn) ReadLastOffset() (int64, error) { 896 return c.readOffset(LastOffset) 897 } 898 899 // ReadOffsets returns the absolute first and last offsets of the topic used by 900 // the connection. 901 func (c *Conn) ReadOffsets() (first, last int64, err error) { 902 // We have to submit two different requests to fetch the first and last 903 // offsets because kafka refuses requests that ask for multiple offsets 904 // on the same topic and partition. 905 if first, err = c.ReadFirstOffset(); err != nil { 906 return 907 } 908 if last, err = c.ReadLastOffset(); err != nil { 909 first = 0 // don't leak the value on error 910 return 911 } 912 return 913 } 914 915 func (c *Conn) readOffset(t int64) (offset int64, err error) { 916 err = c.readOperation( 917 func(deadline time.Time, id int32) error { 918 return c.wb.writeListOffsetRequestV1(id, c.clientID, c.topic, c.partition, t) 919 }, 920 func(deadline time.Time, size int) error { 921 return expectZeroSize(readArrayWith(&c.rbuf, size, func(r *bufio.Reader, size int) (int, error) { 922 // We skip the topic name because we've made a request for 923 // a single topic. 924 size, err := discardString(r, size) 925 if err != nil { 926 return size, err 927 } 928 929 // Reading the array of partitions, there will be only one 930 // partition which gives the offset we're looking for. 931 return readArrayWith(r, size, func(r *bufio.Reader, size int) (int, error) { 932 var p partitionOffsetV1 933 size, err := p.readFrom(r, size) 934 if err != nil { 935 return size, err 936 } 937 if p.ErrorCode != 0 { 938 return size, Error(p.ErrorCode) 939 } 940 offset = p.Offset 941 return size, nil 942 }) 943 })) 944 }, 945 ) 946 return 947 } 948 949 // ReadPartitions returns the list of available partitions for the given list of 950 // topics. 951 // 952 // If the method is called with no topic, it uses the topic configured on the 953 // connection. If there are none, the method fetches all partitions of the kafka 954 // cluster. 955 func (c *Conn) ReadPartitions(topics ...string) (partitions []Partition, err error) { 956 if len(topics) == 0 { 957 if len(c.topic) != 0 { 958 defaultTopics := [...]string{c.topic} 959 topics = defaultTopics[:] 960 } else { 961 // topics needs to be explicitly nil-ed out or the broker will 962 // interpret it as a request for 0 partitions instead of all. 963 topics = nil 964 } 965 } 966 967 err = c.readOperation( 968 func(deadline time.Time, id int32) error { 969 return c.writeRequest(metadata, v1, id, topicMetadataRequestV1(topics)) 970 }, 971 func(deadline time.Time, size int) error { 972 var res metadataResponseV1 973 974 if err := c.readResponse(size, &res); err != nil { 975 return err 976 } 977 978 brokers := make(map[int32]Broker, len(res.Brokers)) 979 for _, b := range res.Brokers { 980 brokers[b.NodeID] = Broker{ 981 Host: b.Host, 982 Port: int(b.Port), 983 ID: int(b.NodeID), 984 Rack: b.Rack, 985 } 986 } 987 988 for _, t := range res.Topics { 989 if t.TopicErrorCode != 0 && (c.topic == "" || t.TopicName == c.topic) { 990 // We only report errors if they happened for the topic of 991 // the connection, otherwise the topic will simply have no 992 // partitions in the result set. 993 return Error(t.TopicErrorCode) 994 } 995 for _, p := range t.Partitions { 996 partitions = append(partitions, Partition{ 997 Topic: t.TopicName, 998 Leader: brokers[p.Leader], 999 Replicas: makeBrokers(brokers, p.Replicas...), 1000 Isr: makeBrokers(brokers, p.Isr...), 1001 ID: int(p.PartitionID), 1002 }) 1003 } 1004 } 1005 return nil 1006 }, 1007 ) 1008 return 1009 } 1010 1011 func makeBrokers(brokers map[int32]Broker, ids ...int32) []Broker { 1012 b := make([]Broker, 0, len(ids)) 1013 for _, id := range ids { 1014 if br, ok := brokers[id]; ok { 1015 b = append(b, br) 1016 } 1017 } 1018 return b 1019 } 1020 1021 // Write writes a message to the kafka broker that this connection was 1022 // established to. The method returns the number of bytes written, or an error 1023 // if something went wrong. 1024 // 1025 // The operation either succeeds or fail, it never partially writes the message. 1026 // 1027 // This method is exposed to satisfy the net.Conn interface but is less efficient 1028 // than the more general purpose WriteMessages method. 1029 func (c *Conn) Write(b []byte) (int, error) { 1030 return c.WriteCompressedMessages(nil, Message{Value: b}) 1031 } 1032 1033 // WriteMessages writes a batch of messages to the connection's topic and 1034 // partition, returning the number of bytes written. The write is an atomic 1035 // operation, it either fully succeeds or fails. 1036 func (c *Conn) WriteMessages(msgs ...Message) (int, error) { 1037 return c.WriteCompressedMessages(nil, msgs...) 1038 } 1039 1040 // WriteCompressedMessages writes a batch of messages to the connection's topic 1041 // and partition, returning the number of bytes written. The write is an atomic 1042 // operation, it either fully succeeds or fails. 1043 // 1044 // If the compression codec is not nil, the messages will be compressed. 1045 func (c *Conn) WriteCompressedMessages(codec CompressionCodec, msgs ...Message) (nbytes int, err error) { 1046 nbytes, _, _, _, err = c.writeCompressedMessages(codec, msgs...) 1047 return 1048 } 1049 1050 // WriteCompressedMessagesAt writes a batch of messages to the connection's topic 1051 // and partition, returning the number of bytes written, partition and offset numbers 1052 // and timestamp assigned by the kafka broker to the message set. The write is an atomic 1053 // operation, it either fully succeeds or fails. 1054 // 1055 // If the compression codec is not nil, the messages will be compressed. 1056 func (c *Conn) WriteCompressedMessagesAt(codec CompressionCodec, msgs ...Message) (nbytes int, partition int32, offset int64, appendTime time.Time, err error) { 1057 return c.writeCompressedMessages(codec, msgs...) 1058 } 1059 1060 func (c *Conn) writeCompressedMessages(codec CompressionCodec, msgs ...Message) (nbytes int, partition int32, offset int64, appendTime time.Time, err error) { 1061 if len(msgs) == 0 { 1062 return 1063 } 1064 1065 writeTime := time.Now() 1066 for i, msg := range msgs { 1067 // users may believe they can set the Topic and/or Partition 1068 // on the kafka message. 1069 if msg.Topic != "" && msg.Topic != c.topic { 1070 err = errInvalidWriteTopic 1071 return 1072 } 1073 if msg.Partition != 0 { 1074 err = errInvalidWritePartition 1075 return 1076 } 1077 1078 if msg.Time.IsZero() { 1079 msgs[i].Time = writeTime 1080 } 1081 1082 nbytes += len(msg.Key) + len(msg.Value) 1083 } 1084 1085 var produceVersion apiVersion 1086 if produceVersion, err = c.negotiateVersion(produce, v2, v3, v7); err != nil { 1087 return 1088 } 1089 1090 err = c.writeOperation( 1091 func(deadline time.Time, id int32) error { 1092 now := time.Now() 1093 deadline = adjustDeadlineForRTT(deadline, now, defaultRTT) 1094 switch produceVersion { 1095 case v7: 1096 recordBatch, err := 1097 newRecordBatch( 1098 codec, 1099 msgs..., 1100 ) 1101 if err != nil { 1102 return err 1103 } 1104 return c.wb.writeProduceRequestV7( 1105 id, 1106 c.clientID, 1107 c.topic, 1108 c.partition, 1109 deadlineToTimeout(deadline, now), 1110 int16(atomic.LoadInt32(&c.requiredAcks)), 1111 c.transactionalID, 1112 recordBatch, 1113 ) 1114 case v3: 1115 recordBatch, err := 1116 newRecordBatch( 1117 codec, 1118 msgs..., 1119 ) 1120 if err != nil { 1121 return err 1122 } 1123 return c.wb.writeProduceRequestV3( 1124 id, 1125 c.clientID, 1126 c.topic, 1127 c.partition, 1128 deadlineToTimeout(deadline, now), 1129 int16(atomic.LoadInt32(&c.requiredAcks)), 1130 c.transactionalID, 1131 recordBatch, 1132 ) 1133 default: 1134 return c.wb.writeProduceRequestV2( 1135 codec, 1136 id, 1137 c.clientID, 1138 c.topic, 1139 c.partition, 1140 deadlineToTimeout(deadline, now), 1141 int16(atomic.LoadInt32(&c.requiredAcks)), 1142 msgs..., 1143 ) 1144 } 1145 }, 1146 func(deadline time.Time, size int) error { 1147 return expectZeroSize(readArrayWith(&c.rbuf, size, func(r *bufio.Reader, size int) (int, error) { 1148 // Skip the topic, we've produced the message to only one topic, 1149 // no need to waste resources loading it in memory. 1150 size, err := discardString(r, size) 1151 if err != nil { 1152 return size, err 1153 } 1154 1155 // Read the list of partitions, there should be only one since 1156 // we've produced a message to a single partition. 1157 size, err = readArrayWith(r, size, func(r *bufio.Reader, size int) (int, error) { 1158 switch produceVersion { 1159 case v7: 1160 var p produceResponsePartitionV7 1161 size, err := p.readFrom(r, size) 1162 if err == nil && p.ErrorCode != 0 { 1163 err = Error(p.ErrorCode) 1164 } 1165 if err == nil { 1166 partition = p.Partition 1167 offset = p.Offset 1168 appendTime = time.Unix(0, p.Timestamp*int64(time.Millisecond)) 1169 } 1170 return size, err 1171 default: 1172 var p produceResponsePartitionV2 1173 size, err := p.readFrom(r, size) 1174 if err == nil && p.ErrorCode != 0 { 1175 err = Error(p.ErrorCode) 1176 } 1177 if err == nil { 1178 partition = p.Partition 1179 offset = p.Offset 1180 appendTime = time.Unix(0, p.Timestamp*int64(time.Millisecond)) 1181 } 1182 return size, err 1183 } 1184 }) 1185 if err != nil { 1186 return size, err 1187 } 1188 1189 // The response is trailed by the throttle time, also skipping 1190 // since it's not interesting here. 1191 return discardInt32(r, size) 1192 })) 1193 }, 1194 ) 1195 1196 if err != nil { 1197 nbytes = 0 1198 } 1199 1200 return 1201 } 1202 1203 // SetRequiredAcks sets the number of acknowledges from replicas that the 1204 // connection requests when producing messages. 1205 func (c *Conn) SetRequiredAcks(n int) error { 1206 switch n { 1207 case -1, 1: 1208 atomic.StoreInt32(&c.requiredAcks, int32(n)) 1209 return nil 1210 default: 1211 return InvalidRequiredAcks 1212 } 1213 } 1214 1215 func (c *Conn) writeRequestHeader(apiKey apiKey, apiVersion apiVersion, correlationID int32, size int32) { 1216 hdr := c.requestHeader(apiKey, apiVersion, correlationID) 1217 hdr.Size = (hdr.size() + size) - 4 1218 hdr.writeTo(&c.wb) 1219 } 1220 1221 func (c *Conn) writeRequest(apiKey apiKey, apiVersion apiVersion, correlationID int32, req request) error { 1222 hdr := c.requestHeader(apiKey, apiVersion, correlationID) 1223 hdr.Size = (hdr.size() + req.size()) - 4 1224 hdr.writeTo(&c.wb) 1225 req.writeTo(&c.wb) 1226 return c.wbuf.Flush() 1227 } 1228 1229 func (c *Conn) readResponse(size int, res interface{}) error { 1230 size, err := read(&c.rbuf, size, res) 1231 switch err.(type) { 1232 case Error: 1233 var e error 1234 if size, e = discardN(&c.rbuf, size, size); e != nil { 1235 err = e 1236 } 1237 } 1238 return expectZeroSize(size, err) 1239 } 1240 1241 func (c *Conn) peekResponseSizeAndID() (int32, int32, error) { 1242 b, err := c.rbuf.Peek(8) 1243 if err != nil { 1244 return 0, 0, err 1245 } 1246 size, id := makeInt32(b[:4]), makeInt32(b[4:]) 1247 return size, id, nil 1248 } 1249 1250 func (c *Conn) skipResponseSizeAndID() { 1251 c.rbuf.Discard(8) 1252 } 1253 1254 func (c *Conn) readDeadline() time.Time { 1255 return c.rdeadline.deadline() 1256 } 1257 1258 func (c *Conn) writeDeadline() time.Time { 1259 return c.wdeadline.deadline() 1260 } 1261 1262 func (c *Conn) readOperation(write func(time.Time, int32) error, read func(time.Time, int) error) error { 1263 return c.do(&c.rdeadline, write, read) 1264 } 1265 1266 func (c *Conn) writeOperation(write func(time.Time, int32) error, read func(time.Time, int) error) error { 1267 return c.do(&c.wdeadline, write, read) 1268 } 1269 1270 func (c *Conn) enter() { 1271 atomic.AddInt32(&c.inflight, +1) 1272 } 1273 1274 func (c *Conn) leave() { 1275 atomic.AddInt32(&c.inflight, -1) 1276 } 1277 1278 func (c *Conn) concurrency() int { 1279 return int(atomic.LoadInt32(&c.inflight)) 1280 } 1281 1282 func (c *Conn) do(d *connDeadline, write func(time.Time, int32) error, read func(time.Time, int) error) error { 1283 id, err := c.doRequest(d, write) 1284 if err != nil { 1285 return err 1286 } 1287 1288 deadline, size, lock, err := c.waitResponse(d, id) 1289 if err != nil { 1290 return err 1291 } 1292 1293 if err = read(deadline, size); err != nil { 1294 switch err.(type) { 1295 case Error: 1296 default: 1297 c.conn.Close() 1298 } 1299 } 1300 1301 d.unsetConnReadDeadline() 1302 lock.Unlock() 1303 return err 1304 } 1305 1306 func (c *Conn) doRequest(d *connDeadline, write func(time.Time, int32) error) (id int32, err error) { 1307 c.enter() 1308 c.wlock.Lock() 1309 c.correlationID++ 1310 id = c.correlationID 1311 err = write(d.setConnWriteDeadline(c.conn), id) 1312 d.unsetConnWriteDeadline() 1313 1314 if err != nil { 1315 // When an error occurs there's no way to know if the connection is in a 1316 // recoverable state so we're better off just giving up at this point to 1317 // avoid any risk of corrupting the following operations. 1318 c.conn.Close() 1319 c.leave() 1320 } 1321 1322 c.wlock.Unlock() 1323 return 1324 } 1325 1326 func (c *Conn) waitResponse(d *connDeadline, id int32) (deadline time.Time, size int, lock *sync.Mutex, err error) { 1327 for { 1328 var rsz int32 1329 var rid int32 1330 1331 c.rlock.Lock() 1332 deadline = d.setConnReadDeadline(c.conn) 1333 rsz, rid, err = c.peekResponseSizeAndID() 1334 1335 if err != nil { 1336 d.unsetConnReadDeadline() 1337 c.conn.Close() 1338 c.rlock.Unlock() 1339 break 1340 } 1341 1342 if id == rid { 1343 c.skipResponseSizeAndID() 1344 size, lock = int(rsz-4), &c.rlock 1345 // Don't unlock the read mutex to yield ownership to the caller. 1346 break 1347 } 1348 1349 if c.concurrency() == 1 { 1350 // If the goroutine is the only one waiting on this connection it 1351 // should be impossible to read a correlation id different from the 1352 // one it expects. This is a sign that the data we are reading on 1353 // the wire is corrupted and the connection needs to be closed. 1354 err = io.ErrNoProgress 1355 c.rlock.Unlock() 1356 break 1357 } 1358 1359 // Optimistically release the read lock if a response has already 1360 // been received but the current operation is not the target for it. 1361 c.rlock.Unlock() 1362 runtime.Gosched() 1363 } 1364 1365 c.leave() 1366 return 1367 } 1368 1369 func (c *Conn) requestHeader(apiKey apiKey, apiVersion apiVersion, correlationID int32) requestHeader { 1370 return requestHeader{ 1371 ApiKey: int16(apiKey), 1372 ApiVersion: int16(apiVersion), 1373 CorrelationID: correlationID, 1374 ClientID: c.clientID, 1375 } 1376 } 1377 1378 func (c *Conn) ApiVersions() ([]ApiVersion, error) { 1379 deadline := &c.rdeadline 1380 1381 if deadline.deadline().IsZero() { 1382 // ApiVersions is called automatically when API version negotiation 1383 // needs to happen, so we are not guaranteed that a read deadline has 1384 // been set yet. Fallback to use the write deadline in case it was 1385 // set, for example when version negotiation is initiated during a 1386 // produce request. 1387 deadline = &c.wdeadline 1388 } 1389 1390 id, err := c.doRequest(deadline, func(_ time.Time, id int32) error { 1391 h := requestHeader{ 1392 ApiKey: int16(apiVersions), 1393 ApiVersion: int16(v0), 1394 CorrelationID: id, 1395 ClientID: c.clientID, 1396 } 1397 h.Size = (h.size() - 4) 1398 h.writeTo(&c.wb) 1399 return c.wbuf.Flush() 1400 }) 1401 if err != nil { 1402 return nil, err 1403 } 1404 1405 _, size, lock, err := c.waitResponse(deadline, id) 1406 if err != nil { 1407 return nil, err 1408 } 1409 defer lock.Unlock() 1410 1411 var errorCode int16 1412 if size, err = readInt16(&c.rbuf, size, &errorCode); err != nil { 1413 return nil, err 1414 } 1415 var arrSize int32 1416 if size, err = readInt32(&c.rbuf, size, &arrSize); err != nil { 1417 return nil, err 1418 } 1419 r := make([]ApiVersion, arrSize) 1420 for i := 0; i < int(arrSize); i++ { 1421 if size, err = readInt16(&c.rbuf, size, &r[i].ApiKey); err != nil { 1422 return nil, err 1423 } 1424 if size, err = readInt16(&c.rbuf, size, &r[i].MinVersion); err != nil { 1425 return nil, err 1426 } 1427 if size, err = readInt16(&c.rbuf, size, &r[i].MaxVersion); err != nil { 1428 return nil, err 1429 } 1430 } 1431 1432 if errorCode != 0 { 1433 return r, Error(errorCode) 1434 } 1435 1436 return r, nil 1437 } 1438 1439 // connDeadline is a helper type to implement read/write deadline management on 1440 // the kafka connection. 1441 type connDeadline struct { 1442 mutex sync.Mutex 1443 value time.Time 1444 rconn net.Conn 1445 wconn net.Conn 1446 } 1447 1448 func (d *connDeadline) deadline() time.Time { 1449 d.mutex.Lock() 1450 t := d.value 1451 d.mutex.Unlock() 1452 return t 1453 } 1454 1455 func (d *connDeadline) setDeadline(t time.Time) { 1456 d.mutex.Lock() 1457 d.value = t 1458 1459 if d.rconn != nil { 1460 d.rconn.SetReadDeadline(t) 1461 } 1462 1463 if d.wconn != nil { 1464 d.wconn.SetWriteDeadline(t) 1465 } 1466 1467 d.mutex.Unlock() 1468 } 1469 1470 func (d *connDeadline) setConnReadDeadline(conn net.Conn) time.Time { 1471 d.mutex.Lock() 1472 deadline := d.value 1473 d.rconn = conn 1474 d.rconn.SetReadDeadline(deadline) 1475 d.mutex.Unlock() 1476 return deadline 1477 } 1478 1479 func (d *connDeadline) setConnWriteDeadline(conn net.Conn) time.Time { 1480 d.mutex.Lock() 1481 deadline := d.value 1482 d.wconn = conn 1483 d.wconn.SetWriteDeadline(deadline) 1484 d.mutex.Unlock() 1485 return deadline 1486 } 1487 1488 func (d *connDeadline) unsetConnReadDeadline() { 1489 d.mutex.Lock() 1490 d.rconn = nil 1491 d.mutex.Unlock() 1492 } 1493 1494 func (d *connDeadline) unsetConnWriteDeadline() { 1495 d.mutex.Lock() 1496 d.wconn = nil 1497 d.mutex.Unlock() 1498 } 1499 1500 // saslHandshake sends the SASL handshake message. This will determine whether 1501 // the Mechanism is supported by the cluster. If it's not, this function will 1502 // error out with UnsupportedSASLMechanism. 1503 // 1504 // If the mechanism is unsupported, the handshake request will reply with the 1505 // list of the cluster's configured mechanisms, which could potentially be used 1506 // to facilitate negotiation. At the moment, we are not negotiating the 1507 // mechanism as we believe that brokers are usually known to the client, and 1508 // therefore the client should already know which mechanisms are supported. 1509 // 1510 // See http://kafka.apache.org/protocol.html#The_Messages_SaslHandshake 1511 func (c *Conn) saslHandshake(mechanism string) error { 1512 // The wire format for V0 and V1 is identical, but the version 1513 // number will affect how the SASL authentication 1514 // challenge/responses are sent 1515 var resp saslHandshakeResponseV0 1516 1517 version, err := c.negotiateVersion(saslHandshake, v0, v1) 1518 if err != nil { 1519 return err 1520 } 1521 1522 err = c.writeOperation( 1523 func(deadline time.Time, id int32) error { 1524 return c.writeRequest(saslHandshake, version, id, &saslHandshakeRequestV0{Mechanism: mechanism}) 1525 }, 1526 func(deadline time.Time, size int) error { 1527 return expectZeroSize(func() (int, error) { 1528 return (&resp).readFrom(&c.rbuf, size) 1529 }()) 1530 }, 1531 ) 1532 if err == nil && resp.ErrorCode != 0 { 1533 err = Error(resp.ErrorCode) 1534 } 1535 return err 1536 } 1537 1538 // saslAuthenticate sends the SASL authenticate message. This function must 1539 // be immediately preceded by a successful saslHandshake. 1540 // 1541 // See http://kafka.apache.org/protocol.html#The_Messages_SaslAuthenticate 1542 func (c *Conn) saslAuthenticate(data []byte) ([]byte, error) { 1543 // if we sent a v1 handshake, then we must encapsulate the authentication 1544 // request in a saslAuthenticateRequest. otherwise, we read and write raw 1545 // bytes. 1546 version, err := c.negotiateVersion(saslHandshake, v0, v1) 1547 if err != nil { 1548 return nil, err 1549 } 1550 if version == v1 { 1551 request := saslAuthenticateRequestV0{Data: data} 1552 var response saslAuthenticateResponseV0 1553 1554 err := c.writeOperation( 1555 func(deadline time.Time, id int32) error { 1556 return c.writeRequest(saslAuthenticate, v0, id, request) 1557 }, 1558 func(deadline time.Time, size int) error { 1559 return expectZeroSize(func() (remain int, err error) { 1560 return (&response).readFrom(&c.rbuf, size) 1561 }()) 1562 }, 1563 ) 1564 if err == nil && response.ErrorCode != 0 { 1565 err = Error(response.ErrorCode) 1566 } 1567 return response.Data, err 1568 } 1569 1570 // fall back to opaque bytes on the wire. the broker is expecting these if 1571 // it just processed a v0 sasl handshake. 1572 c.wb.writeInt32(int32(len(data))) 1573 if _, err := c.wb.Write(data); err != nil { 1574 return nil, err 1575 } 1576 if err := c.wb.Flush(); err != nil { 1577 return nil, err 1578 } 1579 1580 var respLen int32 1581 if _, err := readInt32(&c.rbuf, 4, &respLen); err != nil { 1582 return nil, err 1583 } 1584 1585 resp, _, err := readNewBytes(&c.rbuf, int(respLen), int(respLen)) 1586 return resp, err 1587 }