github.com/streamdal/segmentio-kafka-go@v0.4.47-streamdal/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 "sync" 13 "sync/atomic" 14 "time" 15 ) 16 17 var ( 18 errInvalidWriteTopic = errors.New("writes must NOT set Topic on kafka.Message") 19 errInvalidWritePartition = errors.New("writes must NOT set Partition on kafka.Message") 20 ) 21 22 // Conn represents a connection to a kafka broker. 23 // 24 // Instances of Conn are safe to use concurrently from multiple goroutines. 25 type Conn struct { 26 // base network connection 27 conn net.Conn 28 29 // number of inflight requests on the connection. 30 inflight int32 31 32 // offset management (synchronized on the mutex field) 33 mutex sync.Mutex 34 offset int64 35 36 // read buffer (synchronized on rlock) 37 rlock sync.Mutex 38 rbuf bufio.Reader 39 40 // write buffer (synchronized on wlock) 41 wlock sync.Mutex 42 wbuf bufio.Writer 43 wb writeBuffer 44 45 // deadline management 46 wdeadline connDeadline 47 rdeadline connDeadline 48 49 // immutable values of the connection object 50 clientID string 51 topic string 52 partition int32 53 fetchMaxBytes int32 54 fetchMinSize int32 55 broker int32 56 rack string 57 58 // correlation ID generator (synchronized on wlock) 59 correlationID int32 60 61 // number of replica acks required when publishing to a partition 62 requiredAcks int32 63 64 // lazily loaded API versions used by this connection 65 apiVersions atomic.Value // apiVersionMap 66 67 transactionalID *string 68 } 69 70 type apiVersionMap map[apiKey]ApiVersion 71 72 func (v apiVersionMap) negotiate(key apiKey, sortedSupportedVersions ...apiVersion) apiVersion { 73 x := v[key] 74 75 for i := len(sortedSupportedVersions) - 1; i >= 0; i-- { 76 s := sortedSupportedVersions[i] 77 78 if apiVersion(x.MaxVersion) >= s { 79 return s 80 } 81 } 82 83 return -1 84 } 85 86 // ConnConfig is a configuration object used to create new instances of Conn. 87 type ConnConfig struct { 88 ClientID string 89 Topic string 90 Partition int 91 Broker int 92 Rack string 93 94 // The transactional id to use for transactional delivery. Idempotent 95 // deliver should be enabled if transactional id is configured. 96 // For more details look at transactional.id description here: http://kafka.apache.org/documentation.html#producerconfigs 97 // Empty string means that this connection can't be transactional. 98 TransactionalID string 99 } 100 101 // ReadBatchConfig is a configuration object used for reading batches of messages. 102 type ReadBatchConfig struct { 103 // MinBytes indicates to the broker the minimum batch size that the consumer 104 // will accept. Setting a high minimum when consuming from a low-volume topic 105 // may result in delayed delivery when the broker does not have enough data to 106 // satisfy the defined minimum. 107 MinBytes int 108 109 // MaxBytes indicates to the broker the maximum batch size that the consumer 110 // will accept. The broker will truncate a message to satisfy this maximum, so 111 // choose a value that is high enough for your largest message size. 112 MaxBytes int 113 114 // IsolationLevel controls the visibility of transactional records. 115 // ReadUncommitted makes all records visible. With ReadCommitted only 116 // non-transactional and committed records are visible. 117 IsolationLevel IsolationLevel 118 119 // MaxWait is the amount of time for the broker while waiting to hit the 120 // min/max byte targets. This setting is independent of any network-level 121 // timeouts or deadlines. 122 // 123 // For backward compatibility, when this field is left zero, kafka-go will 124 // infer the max wait from the connection's read deadline. 125 MaxWait time.Duration 126 } 127 128 type IsolationLevel int8 129 130 const ( 131 ReadUncommitted IsolationLevel = 0 132 ReadCommitted IsolationLevel = 1 133 ) 134 135 var ( 136 // DefaultClientID is the default value used as ClientID of kafka 137 // connections. 138 DefaultClientID string 139 ) 140 141 func init() { 142 progname := filepath.Base(os.Args[0]) 143 hostname, _ := os.Hostname() 144 DefaultClientID = fmt.Sprintf("%s@%s (github.com/segmentio/kafka-go)", progname, hostname) 145 } 146 147 // NewConn returns a new kafka connection for the given topic and partition. 148 func NewConn(conn net.Conn, topic string, partition int) *Conn { 149 return NewConnWith(conn, ConnConfig{ 150 Topic: topic, 151 Partition: partition, 152 }) 153 } 154 155 func emptyToNullable(transactionalID string) (result *string) { 156 if transactionalID != "" { 157 result = &transactionalID 158 } 159 return result 160 } 161 162 // NewConnWith returns a new kafka connection configured with config. 163 // The offset is initialized to FirstOffset. 164 func NewConnWith(conn net.Conn, config ConnConfig) *Conn { 165 if len(config.ClientID) == 0 { 166 config.ClientID = DefaultClientID 167 } 168 169 if config.Partition < 0 || config.Partition > math.MaxInt32 { 170 panic(fmt.Sprintf("invalid partition number: %d", config.Partition)) 171 } 172 173 c := &Conn{ 174 conn: conn, 175 rbuf: *bufio.NewReader(conn), 176 wbuf: *bufio.NewWriter(conn), 177 clientID: config.ClientID, 178 topic: config.Topic, 179 partition: int32(config.Partition), 180 broker: int32(config.Broker), 181 rack: config.Rack, 182 offset: FirstOffset, 183 requiredAcks: -1, 184 transactionalID: emptyToNullable(config.TransactionalID), 185 } 186 187 c.wb.w = &c.wbuf 188 189 // The fetch request needs to ask for a MaxBytes value that is at least 190 // enough to load the control data of the response. To avoid having to 191 // recompute it on every read, it is cached here in the Conn value. 192 c.fetchMinSize = (fetchResponseV2{ 193 Topics: []fetchResponseTopicV2{{ 194 TopicName: config.Topic, 195 Partitions: []fetchResponsePartitionV2{{ 196 Partition: int32(config.Partition), 197 MessageSet: messageSet{{}}, 198 }}, 199 }}, 200 }).size() 201 c.fetchMaxBytes = math.MaxInt32 - c.fetchMinSize 202 return c 203 } 204 205 func (c *Conn) negotiateVersion(key apiKey, sortedSupportedVersions ...apiVersion) (apiVersion, error) { 206 v, err := c.loadVersions() 207 if err != nil { 208 return -1, err 209 } 210 a := v.negotiate(key, sortedSupportedVersions...) 211 if a < 0 { 212 return -1, fmt.Errorf("no matching versions were found between the client and the broker for API key %d", key) 213 } 214 return a, nil 215 } 216 217 func (c *Conn) loadVersions() (apiVersionMap, error) { 218 v, _ := c.apiVersions.Load().(apiVersionMap) 219 if v != nil { 220 return v, nil 221 } 222 223 brokerVersions, err := c.ApiVersions() 224 if err != nil { 225 return nil, err 226 } 227 228 v = make(apiVersionMap, len(brokerVersions)) 229 230 for _, a := range brokerVersions { 231 v[apiKey(a.ApiKey)] = a 232 } 233 234 c.apiVersions.Store(v) 235 return v, nil 236 } 237 238 // Broker returns a Broker value representing the kafka broker that this 239 // connection was established to. 240 func (c *Conn) Broker() Broker { 241 addr := c.conn.RemoteAddr() 242 host, port, _ := splitHostPortNumber(addr.String()) 243 return Broker{ 244 Host: host, 245 Port: port, 246 ID: int(c.broker), 247 Rack: c.rack, 248 } 249 } 250 251 // Controller requests kafka for the current controller and returns its URL. 252 func (c *Conn) Controller() (broker Broker, err error) { 253 err = c.readOperation( 254 func(deadline time.Time, id int32) error { 255 return c.writeRequest(metadata, v1, id, topicMetadataRequestV1([]string{})) 256 }, 257 func(deadline time.Time, size int) error { 258 var res metadataResponseV1 259 260 if err := c.readResponse(size, &res); err != nil { 261 return err 262 } 263 for _, brokerMeta := range res.Brokers { 264 if brokerMeta.NodeID == res.ControllerID { 265 broker = Broker{ID: int(brokerMeta.NodeID), 266 Port: int(brokerMeta.Port), 267 Host: brokerMeta.Host, 268 Rack: brokerMeta.Rack} 269 break 270 } 271 } 272 return nil 273 }, 274 ) 275 return broker, err 276 } 277 278 // Brokers retrieve the broker list from the Kafka metadata. 279 func (c *Conn) Brokers() ([]Broker, error) { 280 var brokers []Broker 281 err := c.readOperation( 282 func(deadline time.Time, id int32) error { 283 return c.writeRequest(metadata, v1, id, topicMetadataRequestV1([]string{})) 284 }, 285 func(deadline time.Time, size int) error { 286 var res metadataResponseV1 287 288 if err := c.readResponse(size, &res); err != nil { 289 return err 290 } 291 292 brokers = make([]Broker, len(res.Brokers)) 293 for i, brokerMeta := range res.Brokers { 294 brokers[i] = Broker{ 295 ID: int(brokerMeta.NodeID), 296 Port: int(brokerMeta.Port), 297 Host: brokerMeta.Host, 298 Rack: brokerMeta.Rack, 299 } 300 } 301 return nil 302 }, 303 ) 304 return brokers, err 305 } 306 307 // DeleteTopics deletes the specified topics. 308 func (c *Conn) DeleteTopics(topics ...string) error { 309 _, err := c.deleteTopics(deleteTopicsRequestV0{ 310 Topics: topics, 311 }) 312 return err 313 } 314 315 // findCoordinator finds the coordinator for the specified group or transaction 316 // 317 // See http://kafka.apache.org/protocol.html#The_Messages_FindCoordinator 318 func (c *Conn) findCoordinator(request findCoordinatorRequestV0) (findCoordinatorResponseV0, error) { 319 var response findCoordinatorResponseV0 320 321 err := c.readOperation( 322 func(deadline time.Time, id int32) error { 323 return c.writeRequest(findCoordinator, v0, id, request) 324 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 755 var adjustedDeadline time.Time 756 var maxFetch = int(c.fetchMaxBytes) 757 758 if cfg.MinBytes < 0 || cfg.MinBytes > maxFetch { 759 return &Batch{err: fmt.Errorf("kafka.(*Conn).ReadBatch: minBytes of %d out of [1,%d] bounds", cfg.MinBytes, maxFetch)} 760 } 761 if cfg.MaxBytes < 0 || cfg.MaxBytes > maxFetch { 762 return &Batch{err: fmt.Errorf("kafka.(*Conn).ReadBatch: maxBytes of %d out of [1,%d] bounds", cfg.MaxBytes, maxFetch)} 763 } 764 if cfg.MinBytes > cfg.MaxBytes { 765 return &Batch{err: fmt.Errorf("kafka.(*Conn).ReadBatch: minBytes (%d) > maxBytes (%d)", cfg.MinBytes, cfg.MaxBytes)} 766 } 767 768 offset, whence := c.Offset() 769 770 offset, err := c.Seek(offset, whence|SeekDontCheck) 771 if err != nil { 772 return &Batch{err: dontExpectEOF(err)} 773 } 774 775 fetchVersion, err := c.negotiateVersion(fetch, v2, v5, v10) 776 if err != nil { 777 return &Batch{err: dontExpectEOF(err)} 778 } 779 780 id, err := c.doRequest(&c.rdeadline, func(deadline time.Time, id int32) error { 781 now := time.Now() 782 var timeout time.Duration 783 if cfg.MaxWait > 0 { 784 // explicitly-configured case: no changes are made to the deadline, 785 // and the timeout is sent exactly as specified. 786 timeout = cfg.MaxWait 787 } else { 788 // default case: use the original logic to adjust the conn's 789 // deadline.T 790 deadline = adjustDeadlineForRTT(deadline, now, defaultRTT) 791 timeout = deadlineToTimeout(deadline, now) 792 } 793 // save this variable outside of the closure for later use in detecting 794 // truncated messages. 795 adjustedDeadline = deadline 796 switch fetchVersion { 797 case v10: 798 return c.wb.writeFetchRequestV10( 799 id, 800 c.clientID, 801 c.topic, 802 c.partition, 803 offset, 804 cfg.MinBytes, 805 cfg.MaxBytes+int(c.fetchMinSize), 806 timeout, 807 int8(cfg.IsolationLevel), 808 ) 809 case v5: 810 return c.wb.writeFetchRequestV5( 811 id, 812 c.clientID, 813 c.topic, 814 c.partition, 815 offset, 816 cfg.MinBytes, 817 cfg.MaxBytes+int(c.fetchMinSize), 818 timeout, 819 int8(cfg.IsolationLevel), 820 ) 821 default: 822 return c.wb.writeFetchRequestV2( 823 id, 824 c.clientID, 825 c.topic, 826 c.partition, 827 offset, 828 cfg.MinBytes, 829 cfg.MaxBytes+int(c.fetchMinSize), 830 timeout, 831 ) 832 } 833 }) 834 if err != nil { 835 return &Batch{err: dontExpectEOF(err)} 836 } 837 838 _, size, lock, err := c.waitResponse(&c.rdeadline, id) 839 if err != nil { 840 return &Batch{err: dontExpectEOF(err)} 841 } 842 843 var throttle int32 844 var highWaterMark int64 845 var remain int 846 847 switch fetchVersion { 848 case v10: 849 throttle, highWaterMark, remain, err = readFetchResponseHeaderV10(&c.rbuf, size) 850 case v5: 851 throttle, highWaterMark, remain, err = readFetchResponseHeaderV5(&c.rbuf, size) 852 default: 853 throttle, highWaterMark, remain, err = readFetchResponseHeaderV2(&c.rbuf, size) 854 } 855 if errors.Is(err, errShortRead) { 856 err = checkTimeoutErr(adjustedDeadline) 857 } 858 859 var msgs *messageSetReader 860 if err == nil { 861 if highWaterMark == offset { 862 msgs = &messageSetReader{empty: true} 863 } else { 864 msgs, err = newMessageSetReader(&c.rbuf, remain) 865 } 866 } 867 if errors.Is(err, errShortRead) { 868 err = checkTimeoutErr(adjustedDeadline) 869 } 870 871 return &Batch{ 872 conn: c, 873 msgs: msgs, 874 deadline: adjustedDeadline, 875 throttle: makeDuration(throttle), 876 lock: lock, 877 topic: c.topic, // topic is copied to Batch to prevent race with Batch.close 878 partition: int(c.partition), // partition is copied to Batch to prevent race with Batch.close 879 offset: offset, 880 highWaterMark: highWaterMark, 881 // there shouldn't be a short read on initially setting up the batch. 882 // as such, any io.EOF is re-mapped to an io.ErrUnexpectedEOF so that we 883 // don't accidentally signal that we successfully reached the end of the 884 // batch. 885 err: dontExpectEOF(err), 886 } 887 } 888 889 // ReadOffset returns the offset of the first message with a timestamp equal or 890 // greater to t. 891 func (c *Conn) ReadOffset(t time.Time) (int64, error) { 892 return c.readOffset(timestamp(t)) 893 } 894 895 // ReadFirstOffset returns the first offset available on the connection. 896 func (c *Conn) ReadFirstOffset() (int64, error) { 897 return c.readOffset(FirstOffset) 898 } 899 900 // ReadLastOffset returns the last offset available on the connection. 901 func (c *Conn) ReadLastOffset() (int64, error) { 902 return c.readOffset(LastOffset) 903 } 904 905 // ReadOffsets returns the absolute first and last offsets of the topic used by 906 // the connection. 907 func (c *Conn) ReadOffsets() (first, last int64, err error) { 908 // We have to submit two different requests to fetch the first and last 909 // offsets because kafka refuses requests that ask for multiple offsets 910 // on the same topic and partition. 911 if first, err = c.ReadFirstOffset(); err != nil { 912 return 913 } 914 if last, err = c.ReadLastOffset(); err != nil { 915 first = 0 // don't leak the value on error 916 return 917 } 918 return 919 } 920 921 func (c *Conn) readOffset(t int64) (offset int64, err error) { 922 err = c.readOperation( 923 func(deadline time.Time, id int32) error { 924 return c.wb.writeListOffsetRequestV1(id, c.clientID, c.topic, c.partition, t) 925 }, 926 func(deadline time.Time, size int) error { 927 return expectZeroSize(readArrayWith(&c.rbuf, size, func(r *bufio.Reader, size int) (int, error) { 928 // We skip the topic name because we've made a request for 929 // a single topic. 930 size, err := discardString(r, size) 931 if err != nil { 932 return size, err 933 } 934 935 // Reading the array of partitions, there will be only one 936 // partition which gives the offset we're looking for. 937 return readArrayWith(r, size, func(r *bufio.Reader, size int) (int, error) { 938 var p partitionOffsetV1 939 size, err := p.readFrom(r, size) 940 if err != nil { 941 return size, err 942 } 943 if p.ErrorCode != 0 { 944 return size, Error(p.ErrorCode) 945 } 946 offset = p.Offset 947 return size, nil 948 }) 949 })) 950 }, 951 ) 952 return 953 } 954 955 // ReadPartitions returns the list of available partitions for the given list of 956 // topics. 957 // 958 // If the method is called with no topic, it uses the topic configured on the 959 // connection. If there are none, the method fetches all partitions of the kafka 960 // cluster. 961 func (c *Conn) ReadPartitions(topics ...string) (partitions []Partition, err error) { 962 963 if len(topics) == 0 { 964 if len(c.topic) != 0 { 965 defaultTopics := [...]string{c.topic} 966 topics = defaultTopics[:] 967 } else { 968 // topics needs to be explicitly nil-ed out or the broker will 969 // interpret it as a request for 0 partitions instead of all. 970 topics = nil 971 } 972 } 973 metadataVersion, err := c.negotiateVersion(metadata, v1, v6) 974 if err != nil { 975 return nil, err 976 } 977 978 err = c.readOperation( 979 func(deadline time.Time, id int32) error { 980 switch metadataVersion { 981 case v6: 982 return c.writeRequest(metadata, v6, id, topicMetadataRequestV6{Topics: topics, AllowAutoTopicCreation: true}) 983 default: 984 return c.writeRequest(metadata, v1, id, topicMetadataRequestV1(topics)) 985 } 986 }, 987 func(deadline time.Time, size int) error { 988 partitions, err = c.readPartitionsResponse(metadataVersion, size) 989 return err 990 }, 991 ) 992 return 993 } 994 995 func (c *Conn) readPartitionsResponse(metadataVersion apiVersion, size int) ([]Partition, error) { 996 switch metadataVersion { 997 case v6: 998 var res metadataResponseV6 999 if err := c.readResponse(size, &res); err != nil { 1000 return nil, err 1001 } 1002 brokers := readBrokerMetadata(res.Brokers) 1003 return c.readTopicMetadatav6(brokers, res.Topics) 1004 default: 1005 var res metadataResponseV1 1006 if err := c.readResponse(size, &res); err != nil { 1007 return nil, err 1008 } 1009 brokers := readBrokerMetadata(res.Brokers) 1010 return c.readTopicMetadatav1(brokers, res.Topics) 1011 } 1012 } 1013 1014 func readBrokerMetadata(brokerMetadata []brokerMetadataV1) map[int32]Broker { 1015 brokers := make(map[int32]Broker, len(brokerMetadata)) 1016 for _, b := range brokerMetadata { 1017 brokers[b.NodeID] = Broker{ 1018 Host: b.Host, 1019 Port: int(b.Port), 1020 ID: int(b.NodeID), 1021 Rack: b.Rack, 1022 } 1023 } 1024 return brokers 1025 } 1026 1027 func (c *Conn) readTopicMetadatav1(brokers map[int32]Broker, topicMetadata []topicMetadataV1) (partitions []Partition, err error) { 1028 for _, t := range topicMetadata { 1029 if t.TopicErrorCode != 0 && (c.topic == "" || t.TopicName == c.topic) { 1030 // We only report errors if they happened for the topic of 1031 // the connection, otherwise the topic will simply have no 1032 // partitions in the result set. 1033 return nil, Error(t.TopicErrorCode) 1034 } 1035 for _, p := range t.Partitions { 1036 partitions = append(partitions, Partition{ 1037 Topic: t.TopicName, 1038 Leader: brokers[p.Leader], 1039 Replicas: makeBrokers(brokers, p.Replicas...), 1040 Isr: makeBrokers(brokers, p.Isr...), 1041 ID: int(p.PartitionID), 1042 OfflineReplicas: []Broker{}, 1043 }) 1044 } 1045 } 1046 return 1047 } 1048 1049 func (c *Conn) readTopicMetadatav6(brokers map[int32]Broker, topicMetadata []topicMetadataV6) (partitions []Partition, err error) { 1050 for _, t := range topicMetadata { 1051 if t.TopicErrorCode != 0 && (c.topic == "" || t.TopicName == c.topic) { 1052 // We only report errors if they happened for the topic of 1053 // the connection, otherwise the topic will simply have no 1054 // partitions in the result set. 1055 return nil, Error(t.TopicErrorCode) 1056 } 1057 for _, p := range t.Partitions { 1058 partitions = append(partitions, Partition{ 1059 Topic: t.TopicName, 1060 Leader: brokers[p.Leader], 1061 Replicas: makeBrokers(brokers, p.Replicas...), 1062 Isr: makeBrokers(brokers, p.Isr...), 1063 ID: int(p.PartitionID), 1064 OfflineReplicas: makeBrokers(brokers, p.OfflineReplicas...), 1065 }) 1066 } 1067 } 1068 return 1069 } 1070 1071 func makeBrokers(brokers map[int32]Broker, ids ...int32) []Broker { 1072 b := make([]Broker, len(ids)) 1073 for i, id := range ids { 1074 br, ok := brokers[id] 1075 if !ok { 1076 // When the broker id isn't found in the current list of known 1077 // brokers, use a placeholder to report that the cluster has 1078 // logical knowledge of the broker but no information about the 1079 // physical host where it is running. 1080 br.ID = int(id) 1081 } 1082 b[i] = br 1083 } 1084 return b 1085 } 1086 1087 // Write writes a message to the kafka broker that this connection was 1088 // established to. The method returns the number of bytes written, or an error 1089 // if something went wrong. 1090 // 1091 // The operation either succeeds or fail, it never partially writes the message. 1092 // 1093 // This method is exposed to satisfy the net.Conn interface but is less efficient 1094 // than the more general purpose WriteMessages method. 1095 func (c *Conn) Write(b []byte) (int, error) { 1096 return c.WriteCompressedMessages(nil, Message{Value: b}) 1097 } 1098 1099 // WriteMessages writes a batch of messages to the connection's topic and 1100 // partition, returning the number of bytes written. The write is an atomic 1101 // operation, it either fully succeeds or fails. 1102 func (c *Conn) WriteMessages(msgs ...Message) (int, error) { 1103 return c.WriteCompressedMessages(nil, msgs...) 1104 } 1105 1106 // WriteCompressedMessages writes a batch of messages to the connection's topic 1107 // and partition, returning the number of bytes written. The write is an atomic 1108 // operation, it either fully succeeds or fails. 1109 // 1110 // If the compression codec is not nil, the messages will be compressed. 1111 func (c *Conn) WriteCompressedMessages(codec CompressionCodec, msgs ...Message) (nbytes int, err error) { 1112 nbytes, _, _, _, err = c.writeCompressedMessages(codec, msgs...) 1113 return 1114 } 1115 1116 // WriteCompressedMessagesAt writes a batch of messages to the connection's topic 1117 // and partition, returning the number of bytes written, partition and offset numbers 1118 // and timestamp assigned by the kafka broker to the message set. The write is an atomic 1119 // operation, it either fully succeeds or fails. 1120 // 1121 // If the compression codec is not nil, the messages will be compressed. 1122 func (c *Conn) WriteCompressedMessagesAt(codec CompressionCodec, msgs ...Message) (nbytes int, partition int32, offset int64, appendTime time.Time, err error) { 1123 return c.writeCompressedMessages(codec, msgs...) 1124 } 1125 1126 func (c *Conn) writeCompressedMessages(codec CompressionCodec, msgs ...Message) (nbytes int, partition int32, offset int64, appendTime time.Time, err error) { 1127 if len(msgs) == 0 { 1128 return 1129 } 1130 1131 writeTime := time.Now() 1132 for i, msg := range msgs { 1133 // users may believe they can set the Topic and/or Partition 1134 // on the kafka message. 1135 if msg.Topic != "" && msg.Topic != c.topic { 1136 err = errInvalidWriteTopic 1137 return 1138 } 1139 if msg.Partition != 0 { 1140 err = errInvalidWritePartition 1141 return 1142 } 1143 1144 if msg.Time.IsZero() { 1145 msgs[i].Time = writeTime 1146 } 1147 1148 nbytes += len(msg.Key) + len(msg.Value) 1149 } 1150 1151 var produceVersion apiVersion 1152 if produceVersion, err = c.negotiateVersion(produce, v2, v3, v7); err != nil { 1153 return 1154 } 1155 1156 err = c.writeOperation( 1157 func(deadline time.Time, id int32) error { 1158 now := time.Now() 1159 deadline = adjustDeadlineForRTT(deadline, now, defaultRTT) 1160 switch produceVersion { 1161 case v7: 1162 recordBatch, err := 1163 newRecordBatch( 1164 codec, 1165 msgs..., 1166 ) 1167 if err != nil { 1168 return err 1169 } 1170 return c.wb.writeProduceRequestV7( 1171 id, 1172 c.clientID, 1173 c.topic, 1174 c.partition, 1175 deadlineToTimeout(deadline, now), 1176 int16(atomic.LoadInt32(&c.requiredAcks)), 1177 c.transactionalID, 1178 recordBatch, 1179 ) 1180 case v3: 1181 recordBatch, err := 1182 newRecordBatch( 1183 codec, 1184 msgs..., 1185 ) 1186 if err != nil { 1187 return err 1188 } 1189 return c.wb.writeProduceRequestV3( 1190 id, 1191 c.clientID, 1192 c.topic, 1193 c.partition, 1194 deadlineToTimeout(deadline, now), 1195 int16(atomic.LoadInt32(&c.requiredAcks)), 1196 c.transactionalID, 1197 recordBatch, 1198 ) 1199 default: 1200 return c.wb.writeProduceRequestV2( 1201 codec, 1202 id, 1203 c.clientID, 1204 c.topic, 1205 c.partition, 1206 deadlineToTimeout(deadline, now), 1207 int16(atomic.LoadInt32(&c.requiredAcks)), 1208 msgs..., 1209 ) 1210 } 1211 }, 1212 func(deadline time.Time, size int) error { 1213 return expectZeroSize(readArrayWith(&c.rbuf, size, func(r *bufio.Reader, size int) (int, error) { 1214 // Skip the topic, we've produced the message to only one topic, 1215 // no need to waste resources loading it in memory. 1216 size, err := discardString(r, size) 1217 if err != nil { 1218 return size, err 1219 } 1220 1221 // Read the list of partitions, there should be only one since 1222 // we've produced a message to a single partition. 1223 size, err = readArrayWith(r, size, func(r *bufio.Reader, size int) (int, error) { 1224 switch produceVersion { 1225 case v7: 1226 var p produceResponsePartitionV7 1227 size, err := p.readFrom(r, size) 1228 if err == nil && p.ErrorCode != 0 { 1229 err = Error(p.ErrorCode) 1230 } 1231 if err == nil { 1232 partition = p.Partition 1233 offset = p.Offset 1234 appendTime = time.Unix(0, p.Timestamp*int64(time.Millisecond)) 1235 } 1236 return size, err 1237 default: 1238 var p produceResponsePartitionV2 1239 size, err := p.readFrom(r, size) 1240 if err == nil && p.ErrorCode != 0 { 1241 err = Error(p.ErrorCode) 1242 } 1243 if err == nil { 1244 partition = p.Partition 1245 offset = p.Offset 1246 appendTime = time.Unix(0, p.Timestamp*int64(time.Millisecond)) 1247 } 1248 return size, err 1249 } 1250 1251 }) 1252 if err != nil { 1253 return size, err 1254 } 1255 1256 // The response is trailed by the throttle time, also skipping 1257 // since it's not interesting here. 1258 return discardInt32(r, size) 1259 })) 1260 }, 1261 ) 1262 1263 if err != nil { 1264 nbytes = 0 1265 } 1266 1267 return 1268 } 1269 1270 // SetRequiredAcks sets the number of acknowledges from replicas that the 1271 // connection requests when producing messages. 1272 func (c *Conn) SetRequiredAcks(n int) error { 1273 switch n { 1274 case -1, 1: 1275 atomic.StoreInt32(&c.requiredAcks, int32(n)) 1276 return nil 1277 default: 1278 return InvalidRequiredAcks 1279 } 1280 } 1281 1282 func (c *Conn) writeRequest(apiKey apiKey, apiVersion apiVersion, correlationID int32, req request) error { 1283 hdr := c.requestHeader(apiKey, apiVersion, correlationID) 1284 hdr.Size = (hdr.size() + req.size()) - 4 1285 hdr.writeTo(&c.wb) 1286 req.writeTo(&c.wb) 1287 return c.wbuf.Flush() 1288 } 1289 1290 func (c *Conn) readResponse(size int, res interface{}) error { 1291 size, err := read(&c.rbuf, size, res) 1292 if err != nil { 1293 var kafkaError Error 1294 if errors.As(err, &kafkaError) { 1295 size, err = discardN(&c.rbuf, size, size) 1296 } 1297 } 1298 return expectZeroSize(size, err) 1299 } 1300 1301 func (c *Conn) peekResponseSizeAndID() (int32, int32, error) { 1302 b, err := c.rbuf.Peek(8) 1303 if err != nil { 1304 return 0, 0, err 1305 } 1306 size, id := makeInt32(b[:4]), makeInt32(b[4:]) 1307 return size, id, nil 1308 } 1309 1310 func (c *Conn) skipResponseSizeAndID() { 1311 c.rbuf.Discard(8) 1312 } 1313 1314 func (c *Conn) readDeadline() time.Time { 1315 return c.rdeadline.deadline() 1316 } 1317 1318 func (c *Conn) writeDeadline() time.Time { 1319 return c.wdeadline.deadline() 1320 } 1321 1322 func (c *Conn) readOperation(write func(time.Time, int32) error, read func(time.Time, int) error) error { 1323 return c.do(&c.rdeadline, write, read) 1324 } 1325 1326 func (c *Conn) writeOperation(write func(time.Time, int32) error, read func(time.Time, int) error) error { 1327 return c.do(&c.wdeadline, write, read) 1328 } 1329 1330 func (c *Conn) enter() { 1331 atomic.AddInt32(&c.inflight, +1) 1332 } 1333 1334 func (c *Conn) leave() { 1335 atomic.AddInt32(&c.inflight, -1) 1336 } 1337 1338 func (c *Conn) concurrency() int { 1339 return int(atomic.LoadInt32(&c.inflight)) 1340 } 1341 1342 func (c *Conn) do(d *connDeadline, write func(time.Time, int32) error, read func(time.Time, int) error) error { 1343 id, err := c.doRequest(d, write) 1344 if err != nil { 1345 return err 1346 } 1347 1348 deadline, size, lock, err := c.waitResponse(d, id) 1349 if err != nil { 1350 return err 1351 } 1352 1353 if err = read(deadline, size); err != nil { 1354 var kafkaError Error 1355 if !errors.As(err, &kafkaError) { 1356 c.conn.Close() 1357 } 1358 } 1359 1360 d.unsetConnReadDeadline() 1361 lock.Unlock() 1362 return err 1363 } 1364 1365 func (c *Conn) doRequest(d *connDeadline, write func(time.Time, int32) error) (id int32, err error) { 1366 c.enter() 1367 c.wlock.Lock() 1368 c.correlationID++ 1369 id = c.correlationID 1370 err = write(d.setConnWriteDeadline(c.conn), id) 1371 d.unsetConnWriteDeadline() 1372 1373 if err != nil { 1374 // When an error occurs there's no way to know if the connection is in a 1375 // recoverable state so we're better off just giving up at this point to 1376 // avoid any risk of corrupting the following operations. 1377 c.conn.Close() 1378 c.leave() 1379 } 1380 1381 c.wlock.Unlock() 1382 return 1383 } 1384 1385 func (c *Conn) waitResponse(d *connDeadline, id int32) (deadline time.Time, size int, lock *sync.Mutex, err error) { 1386 for { 1387 var rsz int32 1388 var rid int32 1389 1390 c.rlock.Lock() 1391 deadline = d.setConnReadDeadline(c.conn) 1392 rsz, rid, err = c.peekResponseSizeAndID() 1393 1394 if err != nil { 1395 d.unsetConnReadDeadline() 1396 c.conn.Close() 1397 c.rlock.Unlock() 1398 break 1399 } 1400 1401 if id == rid { 1402 c.skipResponseSizeAndID() 1403 size, lock = int(rsz-4), &c.rlock 1404 // Don't unlock the read mutex to yield ownership to the caller. 1405 break 1406 } 1407 1408 if c.concurrency() == 1 { 1409 // If the goroutine is the only one waiting on this connection it 1410 // should be impossible to read a correlation id different from the 1411 // one it expects. This is a sign that the data we are reading on 1412 // the wire is corrupted and the connection needs to be closed. 1413 err = io.ErrNoProgress 1414 c.rlock.Unlock() 1415 break 1416 } 1417 1418 // Optimistically release the read lock if a response has already 1419 // been received but the current operation is not the target for it. 1420 c.rlock.Unlock() 1421 } 1422 1423 c.leave() 1424 return 1425 } 1426 1427 func (c *Conn) requestHeader(apiKey apiKey, apiVersion apiVersion, correlationID int32) requestHeader { 1428 return requestHeader{ 1429 ApiKey: int16(apiKey), 1430 ApiVersion: int16(apiVersion), 1431 CorrelationID: correlationID, 1432 ClientID: c.clientID, 1433 } 1434 } 1435 1436 func (c *Conn) ApiVersions() ([]ApiVersion, error) { 1437 deadline := &c.rdeadline 1438 1439 if deadline.deadline().IsZero() { 1440 // ApiVersions is called automatically when API version negotiation 1441 // needs to happen, so we are not guaranteed that a read deadline has 1442 // been set yet. Fallback to use the write deadline in case it was 1443 // set, for example when version negotiation is initiated during a 1444 // produce request. 1445 deadline = &c.wdeadline 1446 } 1447 1448 id, err := c.doRequest(deadline, func(_ time.Time, id int32) error { 1449 h := requestHeader{ 1450 ApiKey: int16(apiVersions), 1451 ApiVersion: int16(v0), 1452 CorrelationID: id, 1453 ClientID: c.clientID, 1454 } 1455 h.Size = (h.size() - 4) 1456 h.writeTo(&c.wb) 1457 return c.wbuf.Flush() 1458 }) 1459 if err != nil { 1460 return nil, err 1461 } 1462 1463 _, size, lock, err := c.waitResponse(deadline, id) 1464 if err != nil { 1465 return nil, err 1466 } 1467 defer lock.Unlock() 1468 1469 var errorCode int16 1470 if size, err = readInt16(&c.rbuf, size, &errorCode); err != nil { 1471 return nil, err 1472 } 1473 var arrSize int32 1474 if size, err = readInt32(&c.rbuf, size, &arrSize); err != nil { 1475 return nil, err 1476 } 1477 r := make([]ApiVersion, arrSize) 1478 for i := 0; i < int(arrSize); i++ { 1479 if size, err = readInt16(&c.rbuf, size, &r[i].ApiKey); err != nil { 1480 return nil, err 1481 } 1482 if size, err = readInt16(&c.rbuf, size, &r[i].MinVersion); err != nil { 1483 return nil, err 1484 } 1485 if size, err = readInt16(&c.rbuf, size, &r[i].MaxVersion); err != nil { 1486 return nil, err 1487 } 1488 } 1489 1490 if errorCode != 0 { 1491 return r, Error(errorCode) 1492 } 1493 1494 return r, nil 1495 } 1496 1497 // connDeadline is a helper type to implement read/write deadline management on 1498 // the kafka connection. 1499 type connDeadline struct { 1500 mutex sync.Mutex 1501 value time.Time 1502 rconn net.Conn 1503 wconn net.Conn 1504 } 1505 1506 func (d *connDeadline) deadline() time.Time { 1507 d.mutex.Lock() 1508 t := d.value 1509 d.mutex.Unlock() 1510 return t 1511 } 1512 1513 func (d *connDeadline) setDeadline(t time.Time) { 1514 d.mutex.Lock() 1515 d.value = t 1516 1517 if d.rconn != nil { 1518 d.rconn.SetReadDeadline(t) 1519 } 1520 1521 if d.wconn != nil { 1522 d.wconn.SetWriteDeadline(t) 1523 } 1524 1525 d.mutex.Unlock() 1526 } 1527 1528 func (d *connDeadline) setConnReadDeadline(conn net.Conn) time.Time { 1529 d.mutex.Lock() 1530 deadline := d.value 1531 d.rconn = conn 1532 d.rconn.SetReadDeadline(deadline) 1533 d.mutex.Unlock() 1534 return deadline 1535 } 1536 1537 func (d *connDeadline) setConnWriteDeadline(conn net.Conn) time.Time { 1538 d.mutex.Lock() 1539 deadline := d.value 1540 d.wconn = conn 1541 d.wconn.SetWriteDeadline(deadline) 1542 d.mutex.Unlock() 1543 return deadline 1544 } 1545 1546 func (d *connDeadline) unsetConnReadDeadline() { 1547 d.mutex.Lock() 1548 d.rconn = nil 1549 d.mutex.Unlock() 1550 } 1551 1552 func (d *connDeadline) unsetConnWriteDeadline() { 1553 d.mutex.Lock() 1554 d.wconn = nil 1555 d.mutex.Unlock() 1556 } 1557 1558 // saslHandshake sends the SASL handshake message. This will determine whether 1559 // the Mechanism is supported by the cluster. If it's not, this function will 1560 // error out with UnsupportedSASLMechanism. 1561 // 1562 // If the mechanism is unsupported, the handshake request will reply with the 1563 // list of the cluster's configured mechanisms, which could potentially be used 1564 // to facilitate negotiation. At the moment, we are not negotiating the 1565 // mechanism as we believe that brokers are usually known to the client, and 1566 // therefore the client should already know which mechanisms are supported. 1567 // 1568 // See http://kafka.apache.org/protocol.html#The_Messages_SaslHandshake 1569 func (c *Conn) saslHandshake(mechanism string) error { 1570 // The wire format for V0 and V1 is identical, but the version 1571 // number will affect how the SASL authentication 1572 // challenge/responses are sent 1573 var resp saslHandshakeResponseV0 1574 1575 version, err := c.negotiateVersion(saslHandshake, v0, v1) 1576 if err != nil { 1577 return err 1578 } 1579 1580 err = c.writeOperation( 1581 func(deadline time.Time, id int32) error { 1582 return c.writeRequest(saslHandshake, version, id, &saslHandshakeRequestV0{Mechanism: mechanism}) 1583 }, 1584 func(deadline time.Time, size int) error { 1585 return expectZeroSize(func() (int, error) { 1586 return (&resp).readFrom(&c.rbuf, size) 1587 }()) 1588 }, 1589 ) 1590 if err == nil && resp.ErrorCode != 0 { 1591 err = Error(resp.ErrorCode) 1592 } 1593 return err 1594 } 1595 1596 // saslAuthenticate sends the SASL authenticate message. This function must 1597 // be immediately preceded by a successful saslHandshake. 1598 // 1599 // See http://kafka.apache.org/protocol.html#The_Messages_SaslAuthenticate 1600 func (c *Conn) saslAuthenticate(data []byte) ([]byte, error) { 1601 // if we sent a v1 handshake, then we must encapsulate the authentication 1602 // request in a saslAuthenticateRequest. otherwise, we read and write raw 1603 // bytes. 1604 version, err := c.negotiateVersion(saslHandshake, v0, v1) 1605 if err != nil { 1606 return nil, err 1607 } 1608 if version == v1 { 1609 var request = saslAuthenticateRequestV0{Data: data} 1610 var response saslAuthenticateResponseV0 1611 1612 err := c.writeOperation( 1613 func(deadline time.Time, id int32) error { 1614 return c.writeRequest(saslAuthenticate, v0, id, request) 1615 }, 1616 func(deadline time.Time, size int) error { 1617 return expectZeroSize(func() (remain int, err error) { 1618 return (&response).readFrom(&c.rbuf, size) 1619 }()) 1620 }, 1621 ) 1622 if err == nil && response.ErrorCode != 0 { 1623 err = Error(response.ErrorCode) 1624 } 1625 return response.Data, err 1626 } 1627 1628 // fall back to opaque bytes on the wire. the broker is expecting these if 1629 // it just processed a v0 sasl handshake. 1630 c.wb.writeInt32(int32(len(data))) 1631 if _, err := c.wb.Write(data); err != nil { 1632 return nil, err 1633 } 1634 if err := c.wb.Flush(); err != nil { 1635 return nil, err 1636 } 1637 1638 var respLen int32 1639 if _, err := readInt32(&c.rbuf, 4, &respLen); err != nil { 1640 return nil, err 1641 } 1642 1643 resp, _, err := readNewBytes(&c.rbuf, int(respLen), int(respLen)) 1644 return resp, err 1645 }