github.com/segmentio/kafka-go@v0.4.48-0.20240318174348-3f6244eb34fd/reader.go (about) 1 package kafka 2 3 import ( 4 "context" 5 "errors" 6 "fmt" 7 "io" 8 "math" 9 "sort" 10 "strconv" 11 "sync" 12 "sync/atomic" 13 "time" 14 ) 15 16 const ( 17 LastOffset int64 = -1 // The most recent offset available for a partition. 18 FirstOffset int64 = -2 // The least recent offset available for a partition. 19 ) 20 21 const ( 22 // defaultCommitRetries holds the number of commit attempts to make 23 // before giving up. 24 defaultCommitRetries = 3 25 ) 26 27 const ( 28 // defaultFetchMinBytes of 1 byte means that fetch requests are answered as 29 // soon as a single byte of data is available or the fetch request times out 30 // waiting for data to arrive. 31 defaultFetchMinBytes = 1 32 ) 33 34 var ( 35 errOnlyAvailableWithGroup = errors.New("unavailable when GroupID is not set") 36 errNotAvailableWithGroup = errors.New("unavailable when GroupID is set") 37 ) 38 39 const ( 40 // defaultReadBackoffMax/Min sets the boundaries for how long the reader wait before 41 // polling for new messages. 42 defaultReadBackoffMin = 100 * time.Millisecond 43 defaultReadBackoffMax = 1 * time.Second 44 ) 45 46 // Reader provides a high-level API for consuming messages from kafka. 47 // 48 // A Reader automatically manages reconnections to a kafka server, and 49 // blocking methods have context support for asynchronous cancellations. 50 // 51 // Note that it is important to call `Close()` on a `Reader` when a process exits. 52 // The kafka server needs a graceful disconnect to stop it from continuing to 53 // attempt to send messages to the connected clients. The given example will not 54 // call `Close()` if the process is terminated with SIGINT (ctrl-c at the shell) or 55 // SIGTERM (as docker stop or a kubernetes restart does). This can result in a 56 // delay when a new reader on the same topic connects (e.g. new process started 57 // or new container running). Use a `signal.Notify` handler to close the reader on 58 // process shutdown. 59 type Reader struct { 60 // immutable fields of the reader 61 config ReaderConfig 62 63 // communication channels between the parent reader and its subreaders 64 msgs chan readerMessage 65 66 // mutable fields of the reader (synchronized on the mutex) 67 mutex sync.Mutex 68 join sync.WaitGroup 69 cancel context.CancelFunc 70 stop context.CancelFunc 71 done chan struct{} 72 commits chan commitRequest 73 version int64 // version holds the generation of the spawned readers 74 offset int64 75 lag int64 76 closed bool 77 78 // Without a group subscription (when Reader.config.GroupID == ""), 79 // when errors occur, the Reader gets a synthetic readerMessage with 80 // a non-nil err set. With group subscriptions however, when an error 81 // occurs in Reader.run, there's no reader running (sic, cf. reader vs. 82 // Reader) and there's no way to let the high-level methods like 83 // FetchMessage know that an error indeed occurred. If an error in run 84 // occurs, it will be non-block-sent to this unbuffered channel, where 85 // the high-level methods can select{} on it and notify the caller. 86 runError chan error 87 88 // reader stats are all made of atomic values, no need for synchronization. 89 once uint32 90 stctx context.Context 91 // reader stats are all made of atomic values, no need for synchronization. 92 // Use a pointer to ensure 64-bit alignment of the values. 93 stats *readerStats 94 } 95 96 // useConsumerGroup indicates whether the Reader is part of a consumer group. 97 func (r *Reader) useConsumerGroup() bool { return r.config.GroupID != "" } 98 99 func (r *Reader) getTopics() []string { 100 if len(r.config.GroupTopics) > 0 { 101 return r.config.GroupTopics[:] 102 } 103 104 return []string{r.config.Topic} 105 } 106 107 // useSyncCommits indicates whether the Reader is configured to perform sync or 108 // async commits. 109 func (r *Reader) useSyncCommits() bool { return r.config.CommitInterval == 0 } 110 111 func (r *Reader) unsubscribe() { 112 r.cancel() 113 r.join.Wait() 114 // it would be interesting to drain the r.msgs channel at this point since 115 // it will contain buffered messages for partitions that may not be 116 // re-assigned to this reader in the next consumer group generation. 117 // however, draining the channel could race with the client calling 118 // ReadMessage, which could result in messages delivered and/or committed 119 // with gaps in the offset. for now, we will err on the side of caution and 120 // potentially have those messages be reprocessed in the next generation by 121 // another consumer to avoid such a race. 122 } 123 124 func (r *Reader) subscribe(allAssignments map[string][]PartitionAssignment) { 125 offsets := make(map[topicPartition]int64) 126 for topic, assignments := range allAssignments { 127 for _, assignment := range assignments { 128 key := topicPartition{ 129 topic: topic, 130 partition: int32(assignment.ID), 131 } 132 offsets[key] = assignment.Offset 133 } 134 } 135 136 r.mutex.Lock() 137 r.start(offsets) 138 r.mutex.Unlock() 139 140 r.withLogger(func(l Logger) { 141 l.Printf("subscribed to topics and partitions: %+v", offsets) 142 }) 143 } 144 145 // commitOffsetsWithRetry attempts to commit the specified offsets and retries 146 // up to the specified number of times. 147 func (r *Reader) commitOffsetsWithRetry(gen *Generation, offsetStash offsetStash, retries int) (err error) { 148 const ( 149 backoffDelayMin = 100 * time.Millisecond 150 backoffDelayMax = 5 * time.Second 151 ) 152 153 for attempt := 0; attempt < retries; attempt++ { 154 if attempt != 0 { 155 if !sleep(r.stctx, backoff(attempt, backoffDelayMin, backoffDelayMax)) { 156 return 157 } 158 } 159 160 if err = gen.CommitOffsets(offsetStash); err == nil { 161 return 162 } 163 } 164 165 return // err will not be nil 166 } 167 168 // offsetStash holds offsets by topic => partition => offset. 169 type offsetStash map[string]map[int]int64 170 171 // merge updates the offsetStash with the offsets from the provided messages. 172 func (o offsetStash) merge(commits []commit) { 173 for _, c := range commits { 174 offsetsByPartition, ok := o[c.topic] 175 if !ok { 176 offsetsByPartition = map[int]int64{} 177 o[c.topic] = offsetsByPartition 178 } 179 180 if offset, ok := offsetsByPartition[c.partition]; !ok || c.offset > offset { 181 offsetsByPartition[c.partition] = c.offset 182 } 183 } 184 } 185 186 // reset clears the contents of the offsetStash. 187 func (o offsetStash) reset() { 188 for key := range o { 189 delete(o, key) 190 } 191 } 192 193 // commitLoopImmediate handles each commit synchronously. 194 func (r *Reader) commitLoopImmediate(ctx context.Context, gen *Generation) { 195 offsets := offsetStash{} 196 197 for { 198 select { 199 case <-ctx.Done(): 200 // drain the commit channel and prepare a single, final commit. 201 // the commit will combine any outstanding requests and the result 202 // will be sent back to all the callers of CommitMessages so that 203 // they can return. 204 var errchs []chan<- error 205 for hasCommits := true; hasCommits; { 206 select { 207 case req := <-r.commits: 208 offsets.merge(req.commits) 209 errchs = append(errchs, req.errch) 210 default: 211 hasCommits = false 212 } 213 } 214 err := r.commitOffsetsWithRetry(gen, offsets, defaultCommitRetries) 215 for _, errch := range errchs { 216 // NOTE : this will be a buffered channel and will not block. 217 errch <- err 218 } 219 return 220 221 case req := <-r.commits: 222 offsets.merge(req.commits) 223 req.errch <- r.commitOffsetsWithRetry(gen, offsets, defaultCommitRetries) 224 offsets.reset() 225 } 226 } 227 } 228 229 // commitLoopInterval handles each commit asynchronously with a period defined 230 // by ReaderConfig.CommitInterval. 231 func (r *Reader) commitLoopInterval(ctx context.Context, gen *Generation) { 232 ticker := time.NewTicker(r.config.CommitInterval) 233 defer ticker.Stop() 234 235 // the offset stash should not survive rebalances b/c the consumer may 236 // receive new assignments. 237 offsets := offsetStash{} 238 239 commit := func() { 240 if err := r.commitOffsetsWithRetry(gen, offsets, defaultCommitRetries); err != nil { 241 r.withErrorLogger(func(l Logger) { l.Printf("%v", err) }) 242 } else { 243 offsets.reset() 244 } 245 } 246 247 for { 248 select { 249 case <-ctx.Done(): 250 // drain the commit channel in order to prepare the final commit. 251 for hasCommits := true; hasCommits; { 252 select { 253 case req := <-r.commits: 254 offsets.merge(req.commits) 255 default: 256 hasCommits = false 257 } 258 } 259 commit() 260 return 261 262 case <-ticker.C: 263 commit() 264 265 case req := <-r.commits: 266 offsets.merge(req.commits) 267 } 268 } 269 } 270 271 // commitLoop processes commits off the commit chan. 272 func (r *Reader) commitLoop(ctx context.Context, gen *Generation) { 273 r.withLogger(func(l Logger) { 274 l.Printf("started commit for group %s\n", r.config.GroupID) 275 }) 276 defer r.withLogger(func(l Logger) { 277 l.Printf("stopped commit for group %s\n", r.config.GroupID) 278 }) 279 280 if r.useSyncCommits() { 281 r.commitLoopImmediate(ctx, gen) 282 } else { 283 r.commitLoopInterval(ctx, gen) 284 } 285 } 286 287 // run provides the main consumer group management loop. Each iteration performs the 288 // handshake to join the Reader to the consumer group. 289 // 290 // This function is responsible for closing the consumer group upon exit. 291 func (r *Reader) run(cg *ConsumerGroup) { 292 defer close(r.done) 293 defer cg.Close() 294 295 r.withLogger(func(l Logger) { 296 l.Printf("entering loop for consumer group, %v\n", r.config.GroupID) 297 }) 298 299 for { 300 // Limit the number of attempts at waiting for the next 301 // consumer generation. 302 var err error 303 var gen *Generation 304 for attempt := 1; attempt <= r.config.MaxAttempts; attempt++ { 305 gen, err = cg.Next(r.stctx) 306 if err == nil { 307 break 308 } 309 if errors.Is(err, r.stctx.Err()) { 310 return 311 } 312 r.stats.errors.observe(1) 313 r.withErrorLogger(func(l Logger) { 314 l.Printf("%v", err) 315 }) 316 // Continue with next attempt... 317 } 318 if err != nil { 319 // All attempts have failed. 320 select { 321 case r.runError <- err: 322 // If somebody's receiving on the runError, let 323 // them know the error occurred. 324 default: 325 // Otherwise, don't block to allow healing. 326 } 327 continue 328 } 329 330 r.stats.rebalances.observe(1) 331 332 r.subscribe(gen.Assignments) 333 334 gen.Start(func(ctx context.Context) { 335 r.commitLoop(ctx, gen) 336 }) 337 gen.Start(func(ctx context.Context) { 338 // wait for the generation to end and then unsubscribe. 339 select { 340 case <-ctx.Done(): 341 // continue to next generation 342 case <-r.stctx.Done(): 343 // this will be the last loop because the reader is closed. 344 } 345 r.unsubscribe() 346 }) 347 } 348 } 349 350 // ReaderConfig is a configuration object used to create new instances of 351 // Reader. 352 type ReaderConfig struct { 353 // The list of broker addresses used to connect to the kafka cluster. 354 Brokers []string 355 356 // GroupID holds the optional consumer group id. If GroupID is specified, then 357 // Partition should NOT be specified e.g. 0 358 GroupID string 359 360 // GroupTopics allows specifying multiple topics, but can only be used in 361 // combination with GroupID, as it is a consumer-group feature. As such, if 362 // GroupID is set, then either Topic or GroupTopics must be defined. 363 GroupTopics []string 364 365 // The topic to read messages from. 366 Topic string 367 368 // Partition to read messages from. Either Partition or GroupID may 369 // be assigned, but not both 370 Partition int 371 372 // An dialer used to open connections to the kafka server. This field is 373 // optional, if nil, the default dialer is used instead. 374 Dialer *Dialer 375 376 // The capacity of the internal message queue, defaults to 100 if none is 377 // set. 378 QueueCapacity int 379 380 // MinBytes indicates to the broker the minimum batch size that the consumer 381 // will accept. Setting a high minimum when consuming from a low-volume topic 382 // may result in delayed delivery when the broker does not have enough data to 383 // satisfy the defined minimum. 384 // 385 // Default: 1 386 MinBytes int 387 388 // MaxBytes indicates to the broker the maximum batch size that the consumer 389 // will accept. The broker will truncate a message to satisfy this maximum, so 390 // choose a value that is high enough for your largest message size. 391 // 392 // Default: 1MB 393 MaxBytes int 394 395 // Maximum amount of time to wait for new data to come when fetching batches 396 // of messages from kafka. 397 // 398 // Default: 10s 399 MaxWait time.Duration 400 401 // ReadBatchTimeout amount of time to wait to fetch message from kafka messages batch. 402 // 403 // Default: 10s 404 ReadBatchTimeout time.Duration 405 406 // ReadLagInterval sets the frequency at which the reader lag is updated. 407 // Setting this field to a negative value disables lag reporting. 408 ReadLagInterval time.Duration 409 410 // GroupBalancers is the priority-ordered list of client-side consumer group 411 // balancing strategies that will be offered to the coordinator. The first 412 // strategy that all group members support will be chosen by the leader. 413 // 414 // Default: [Range, RoundRobin] 415 // 416 // Only used when GroupID is set 417 GroupBalancers []GroupBalancer 418 419 // HeartbeatInterval sets the optional frequency at which the reader sends the consumer 420 // group heartbeat update. 421 // 422 // Default: 3s 423 // 424 // Only used when GroupID is set 425 HeartbeatInterval time.Duration 426 427 // CommitInterval indicates the interval at which offsets are committed to 428 // the broker. If 0, commits will be handled synchronously. 429 // 430 // Default: 0 431 // 432 // Only used when GroupID is set 433 CommitInterval time.Duration 434 435 // PartitionWatchInterval indicates how often a reader checks for partition changes. 436 // If a reader sees a partition change (such as a partition add) it will rebalance the group 437 // picking up new partitions. 438 // 439 // Default: 5s 440 // 441 // Only used when GroupID is set and WatchPartitionChanges is set. 442 PartitionWatchInterval time.Duration 443 444 // WatchForPartitionChanges is used to inform kafka-go that a consumer group should be 445 // polling the brokers and rebalancing if any partition changes happen to the topic. 446 WatchPartitionChanges bool 447 448 // SessionTimeout optionally sets the length of time that may pass without a heartbeat 449 // before the coordinator considers the consumer dead and initiates a rebalance. 450 // 451 // Default: 30s 452 // 453 // Only used when GroupID is set 454 SessionTimeout time.Duration 455 456 // RebalanceTimeout optionally sets the length of time the coordinator will wait 457 // for members to join as part of a rebalance. For kafka servers under higher 458 // load, it may be useful to set this value higher. 459 // 460 // Default: 30s 461 // 462 // Only used when GroupID is set 463 RebalanceTimeout time.Duration 464 465 // JoinGroupBackoff optionally sets the length of time to wait between re-joining 466 // the consumer group after an error. 467 // 468 // Default: 5s 469 JoinGroupBackoff time.Duration 470 471 // RetentionTime optionally sets the length of time the consumer group will be saved 472 // by the broker. -1 will disable the setting and leave the 473 // retention up to the broker's offsets.retention.minutes property. By 474 // default, that setting is 1 day for kafka < 2.0 and 7 days for kafka >= 2.0. 475 // 476 // Default: -1 477 // 478 // Only used when GroupID is set 479 RetentionTime time.Duration 480 481 // StartOffset determines from whence the consumer group should begin 482 // consuming when it finds a partition without a committed offset. If 483 // non-zero, it must be set to one of FirstOffset or LastOffset. 484 // 485 // Default: FirstOffset 486 // 487 // Only used when GroupID is set 488 StartOffset int64 489 490 // BackoffDelayMin optionally sets the smallest amount of time the reader will wait before 491 // polling for new messages 492 // 493 // Default: 100ms 494 ReadBackoffMin time.Duration 495 496 // BackoffDelayMax optionally sets the maximum amount of time the reader will wait before 497 // polling for new messages 498 // 499 // Default: 1s 500 ReadBackoffMax time.Duration 501 502 // If not nil, specifies a logger used to report internal changes within the 503 // reader. 504 Logger Logger 505 506 // ErrorLogger is the logger used to report errors. If nil, the reader falls 507 // back to using Logger instead. 508 ErrorLogger Logger 509 510 // IsolationLevel controls the visibility of transactional records. 511 // ReadUncommitted makes all records visible. With ReadCommitted only 512 // non-transactional and committed records are visible. 513 IsolationLevel IsolationLevel 514 515 // Limit of how many attempts to connect will be made before returning the error. 516 // 517 // The default is to try 3 times. 518 MaxAttempts int 519 520 // OffsetOutOfRangeError indicates that the reader should return an error in 521 // the event of an OffsetOutOfRange error, rather than retrying indefinitely. 522 // This flag is being added to retain backwards-compatibility, so it will be 523 // removed in a future version of kafka-go. 524 OffsetOutOfRangeError bool 525 } 526 527 // Validate method validates ReaderConfig properties. 528 func (config *ReaderConfig) Validate() error { 529 if len(config.Brokers) == 0 { 530 return errors.New("cannot create a new kafka reader with an empty list of broker addresses") 531 } 532 533 if config.Partition < 0 || config.Partition >= math.MaxInt32 { 534 return fmt.Errorf("partition number out of bounds: %d", config.Partition) 535 } 536 537 if config.MinBytes < 0 { 538 return fmt.Errorf("invalid negative minimum batch size (min = %d)", config.MinBytes) 539 } 540 541 if config.MaxBytes < 0 { 542 return fmt.Errorf("invalid negative maximum batch size (max = %d)", config.MaxBytes) 543 } 544 545 if config.GroupID != "" { 546 if config.Partition != 0 { 547 return errors.New("either Partition or GroupID may be specified, but not both") 548 } 549 550 if len(config.Topic) == 0 && len(config.GroupTopics) == 0 { 551 return errors.New("either Topic or GroupTopics must be specified with GroupID") 552 } 553 } else if len(config.Topic) == 0 { 554 return errors.New("cannot create a new kafka reader with an empty topic") 555 } 556 557 if config.MinBytes > config.MaxBytes { 558 return fmt.Errorf("minimum batch size greater than the maximum (min = %d, max = %d)", config.MinBytes, config.MaxBytes) 559 } 560 561 if config.ReadBackoffMax < 0 { 562 return fmt.Errorf("ReadBackoffMax out of bounds: %d", config.ReadBackoffMax) 563 } 564 565 if config.ReadBackoffMin < 0 { 566 return fmt.Errorf("ReadBackoffMin out of bounds: %d", config.ReadBackoffMin) 567 } 568 569 return nil 570 } 571 572 // ReaderStats is a data structure returned by a call to Reader.Stats that exposes 573 // details about the behavior of the reader. 574 type ReaderStats struct { 575 Dials int64 `metric:"kafka.reader.dial.count" type:"counter"` 576 Fetches int64 `metric:"kafka.reader.fetch.count" type:"counter"` 577 Messages int64 `metric:"kafka.reader.message.count" type:"counter"` 578 Bytes int64 `metric:"kafka.reader.message.bytes" type:"counter"` 579 Rebalances int64 `metric:"kafka.reader.rebalance.count" type:"counter"` 580 Timeouts int64 `metric:"kafka.reader.timeout.count" type:"counter"` 581 Errors int64 `metric:"kafka.reader.error.count" type:"counter"` 582 583 DialTime DurationStats `metric:"kafka.reader.dial.seconds"` 584 ReadTime DurationStats `metric:"kafka.reader.read.seconds"` 585 WaitTime DurationStats `metric:"kafka.reader.wait.seconds"` 586 FetchSize SummaryStats `metric:"kafka.reader.fetch.size"` 587 FetchBytes SummaryStats `metric:"kafka.reader.fetch.bytes"` 588 589 Offset int64 `metric:"kafka.reader.offset" type:"gauge"` 590 Lag int64 `metric:"kafka.reader.lag" type:"gauge"` 591 MinBytes int64 `metric:"kafka.reader.fetch_bytes.min" type:"gauge"` 592 MaxBytes int64 `metric:"kafka.reader.fetch_bytes.max" type:"gauge"` 593 MaxWait time.Duration `metric:"kafka.reader.fetch_wait.max" type:"gauge"` 594 QueueLength int64 `metric:"kafka.reader.queue.length" type:"gauge"` 595 QueueCapacity int64 `metric:"kafka.reader.queue.capacity" type:"gauge"` 596 597 ClientID string `tag:"client_id"` 598 Topic string `tag:"topic"` 599 Partition string `tag:"partition"` 600 601 // The original `Fetches` field had a typo where the metric name was called 602 // "kafak..." instead of "kafka...", in order to offer time to fix monitors 603 // that may be relying on this mistake we are temporarily introducing this 604 // field. 605 DeprecatedFetchesWithTypo int64 `metric:"kafak.reader.fetch.count" type:"counter"` 606 } 607 608 // readerStats is a struct that contains statistics on a reader. 609 type readerStats struct { 610 dials counter 611 fetches counter 612 messages counter 613 bytes counter 614 rebalances counter 615 timeouts counter 616 errors counter 617 dialTime summary 618 readTime summary 619 waitTime summary 620 fetchSize summary 621 fetchBytes summary 622 offset gauge 623 lag gauge 624 partition string 625 } 626 627 // NewReader creates and returns a new Reader configured with config. 628 // The offset is initialized to FirstOffset. 629 func NewReader(config ReaderConfig) *Reader { 630 if err := config.Validate(); err != nil { 631 panic(err) 632 } 633 634 if config.GroupID != "" { 635 if len(config.GroupBalancers) == 0 { 636 config.GroupBalancers = []GroupBalancer{ 637 RangeGroupBalancer{}, 638 RoundRobinGroupBalancer{}, 639 } 640 } 641 } 642 643 if config.Dialer == nil { 644 config.Dialer = DefaultDialer 645 } 646 647 if config.MaxBytes == 0 { 648 config.MaxBytes = 1e6 // 1 MB 649 } 650 651 if config.MinBytes == 0 { 652 config.MinBytes = defaultFetchMinBytes 653 } 654 655 if config.MaxWait == 0 { 656 config.MaxWait = 10 * time.Second 657 } 658 659 if config.ReadBatchTimeout == 0 { 660 config.ReadBatchTimeout = 10 * time.Second 661 } 662 663 if config.ReadLagInterval == 0 { 664 config.ReadLagInterval = 1 * time.Minute 665 } 666 667 if config.ReadBackoffMin == 0 { 668 config.ReadBackoffMin = defaultReadBackoffMin 669 } 670 671 if config.ReadBackoffMax == 0 { 672 config.ReadBackoffMax = defaultReadBackoffMax 673 } 674 675 if config.ReadBackoffMax < config.ReadBackoffMin { 676 panic(fmt.Errorf("ReadBackoffMax %d smaller than ReadBackoffMin %d", config.ReadBackoffMax, config.ReadBackoffMin)) 677 } 678 679 if config.QueueCapacity == 0 { 680 config.QueueCapacity = 100 681 } 682 683 if config.MaxAttempts == 0 { 684 config.MaxAttempts = 3 685 } 686 687 // when configured as a consumer group; stats should report a partition of -1 688 readerStatsPartition := config.Partition 689 if config.GroupID != "" { 690 readerStatsPartition = -1 691 } 692 693 // when configured as a consume group, start version as 1 to ensure that only 694 // the rebalance function will start readers 695 version := int64(0) 696 if config.GroupID != "" { 697 version = 1 698 } 699 700 stctx, stop := context.WithCancel(context.Background()) 701 r := &Reader{ 702 config: config, 703 msgs: make(chan readerMessage, config.QueueCapacity), 704 cancel: func() {}, 705 commits: make(chan commitRequest, config.QueueCapacity), 706 stop: stop, 707 offset: FirstOffset, 708 stctx: stctx, 709 stats: &readerStats{ 710 dialTime: makeSummary(), 711 readTime: makeSummary(), 712 waitTime: makeSummary(), 713 fetchSize: makeSummary(), 714 fetchBytes: makeSummary(), 715 // Generate the string representation of the partition number only 716 // once when the reader is created. 717 partition: strconv.Itoa(readerStatsPartition), 718 }, 719 version: version, 720 } 721 if r.useConsumerGroup() { 722 r.done = make(chan struct{}) 723 r.runError = make(chan error) 724 cg, err := NewConsumerGroup(ConsumerGroupConfig{ 725 ID: r.config.GroupID, 726 Brokers: r.config.Brokers, 727 Dialer: r.config.Dialer, 728 Topics: r.getTopics(), 729 GroupBalancers: r.config.GroupBalancers, 730 HeartbeatInterval: r.config.HeartbeatInterval, 731 PartitionWatchInterval: r.config.PartitionWatchInterval, 732 WatchPartitionChanges: r.config.WatchPartitionChanges, 733 SessionTimeout: r.config.SessionTimeout, 734 RebalanceTimeout: r.config.RebalanceTimeout, 735 JoinGroupBackoff: r.config.JoinGroupBackoff, 736 RetentionTime: r.config.RetentionTime, 737 StartOffset: r.config.StartOffset, 738 Logger: r.config.Logger, 739 ErrorLogger: r.config.ErrorLogger, 740 }) 741 if err != nil { 742 panic(err) 743 } 744 go r.run(cg) 745 } 746 747 return r 748 } 749 750 // Config returns the reader's configuration. 751 func (r *Reader) Config() ReaderConfig { 752 return r.config 753 } 754 755 // Close closes the stream, preventing the program from reading any more 756 // messages from it. 757 func (r *Reader) Close() error { 758 atomic.StoreUint32(&r.once, 1) 759 760 r.mutex.Lock() 761 closed := r.closed 762 r.closed = true 763 r.mutex.Unlock() 764 765 r.cancel() 766 r.stop() 767 r.join.Wait() 768 769 if r.done != nil { 770 <-r.done 771 } 772 773 if !closed { 774 close(r.msgs) 775 } 776 777 return nil 778 } 779 780 // ReadMessage reads and return the next message from the r. The method call 781 // blocks until a message becomes available, or an error occurs. The program 782 // may also specify a context to asynchronously cancel the blocking operation. 783 // 784 // The method returns io.EOF to indicate that the reader has been closed. 785 // 786 // If consumer groups are used, ReadMessage will automatically commit the 787 // offset when called. Note that this could result in an offset being committed 788 // before the message is fully processed. 789 // 790 // If more fine-grained control of when offsets are committed is required, it 791 // is recommended to use FetchMessage with CommitMessages instead. 792 func (r *Reader) ReadMessage(ctx context.Context) (Message, error) { 793 m, err := r.FetchMessage(ctx) 794 if err != nil { 795 return Message{}, fmt.Errorf("fetching message: %w", err) 796 } 797 798 if r.useConsumerGroup() { 799 if err := r.CommitMessages(ctx, m); err != nil { 800 return Message{}, fmt.Errorf("committing message: %w", err) 801 } 802 } 803 804 return m, nil 805 } 806 807 // FetchMessage reads and return the next message from the r. The method call 808 // blocks until a message becomes available, or an error occurs. The program 809 // may also specify a context to asynchronously cancel the blocking operation. 810 // 811 // The method returns io.EOF to indicate that the reader has been closed. 812 // 813 // FetchMessage does not commit offsets automatically when using consumer groups. 814 // Use CommitMessages to commit the offset. 815 func (r *Reader) FetchMessage(ctx context.Context) (Message, error) { 816 r.activateReadLag() 817 818 for { 819 r.mutex.Lock() 820 821 if !r.closed && r.version == 0 { 822 r.start(r.getTopicPartitionOffset()) 823 } 824 825 version := r.version 826 r.mutex.Unlock() 827 828 select { 829 case <-ctx.Done(): 830 return Message{}, ctx.Err() 831 832 case err := <-r.runError: 833 return Message{}, err 834 835 case m, ok := <-r.msgs: 836 if !ok { 837 return Message{}, io.EOF 838 } 839 840 if m.version >= version { 841 r.mutex.Lock() 842 843 switch { 844 case m.error != nil: 845 case version == r.version: 846 r.offset = m.message.Offset + 1 847 r.lag = m.watermark - r.offset 848 } 849 850 r.mutex.Unlock() 851 852 if errors.Is(m.error, io.EOF) { 853 // io.EOF is used as a marker to indicate that the stream 854 // has been closed, in case it was received from the inner 855 // reader we don't want to confuse the program and replace 856 // the error with io.ErrUnexpectedEOF. 857 m.error = io.ErrUnexpectedEOF 858 } 859 860 return m.message, m.error 861 } 862 } 863 } 864 } 865 866 // CommitMessages commits the list of messages passed as argument. The program 867 // may pass a context to asynchronously cancel the commit operation when it was 868 // configured to be blocking. 869 // 870 // Because kafka consumer groups track a single offset per partition, the 871 // highest message offset passed to CommitMessages will cause all previous 872 // messages to be committed. Applications need to account for these Kafka 873 // limitations when committing messages, and maintain message ordering if they 874 // need strong delivery guarantees. This property makes it valid to pass only 875 // the last message seen to CommitMessages in order to move the offset of the 876 // topic/partition it belonged to forward, effectively committing all previous 877 // messages in the partition. 878 func (r *Reader) CommitMessages(ctx context.Context, msgs ...Message) error { 879 if !r.useConsumerGroup() { 880 return errOnlyAvailableWithGroup 881 } 882 883 var errch <-chan error 884 creq := commitRequest{ 885 commits: makeCommits(msgs...), 886 } 887 888 if r.useSyncCommits() { 889 ch := make(chan error, 1) 890 errch, creq.errch = ch, ch 891 } 892 893 select { 894 case r.commits <- creq: 895 case <-ctx.Done(): 896 return ctx.Err() 897 case <-r.stctx.Done(): 898 // This context is used to ensure we don't allow commits after the 899 // reader was closed. 900 return io.ErrClosedPipe 901 } 902 903 if !r.useSyncCommits() { 904 return nil 905 } 906 907 select { 908 case <-ctx.Done(): 909 return ctx.Err() 910 case err := <-errch: 911 return err 912 } 913 } 914 915 // ReadLag returns the current lag of the reader by fetching the last offset of 916 // the topic and partition and computing the difference between that value and 917 // the offset of the last message returned by ReadMessage. 918 // 919 // This method is intended to be used in cases where a program may be unable to 920 // call ReadMessage to update the value returned by Lag, but still needs to get 921 // an up to date estimation of how far behind the reader is. For example when 922 // the consumer is not ready to process the next message. 923 // 924 // The function returns a lag of zero when the reader's current offset is 925 // negative. 926 func (r *Reader) ReadLag(ctx context.Context) (lag int64, err error) { 927 if r.useConsumerGroup() { 928 return 0, errNotAvailableWithGroup 929 } 930 931 type offsets struct { 932 first int64 933 last int64 934 } 935 936 offch := make(chan offsets, 1) 937 errch := make(chan error, 1) 938 939 go func() { 940 var off offsets 941 var err error 942 943 for _, broker := range r.config.Brokers { 944 var conn *Conn 945 946 if conn, err = r.config.Dialer.DialLeader(ctx, "tcp", broker, r.config.Topic, r.config.Partition); err != nil { 947 continue 948 } 949 950 deadline, _ := ctx.Deadline() 951 conn.SetDeadline(deadline) 952 953 off.first, off.last, err = conn.ReadOffsets() 954 conn.Close() 955 956 if err == nil { 957 break 958 } 959 } 960 961 if err != nil { 962 errch <- err 963 } else { 964 offch <- off 965 } 966 }() 967 968 select { 969 case off := <-offch: 970 switch cur := r.Offset(); { 971 case cur == FirstOffset: 972 lag = off.last - off.first 973 974 case cur == LastOffset: 975 lag = 0 976 977 default: 978 lag = off.last - cur 979 } 980 case err = <-errch: 981 case <-ctx.Done(): 982 err = ctx.Err() 983 } 984 985 return 986 } 987 988 // Offset returns the current absolute offset of the reader, or -1 989 // if r is backed by a consumer group. 990 func (r *Reader) Offset() int64 { 991 if r.useConsumerGroup() { 992 return -1 993 } 994 995 r.mutex.Lock() 996 offset := r.offset 997 r.mutex.Unlock() 998 r.withLogger(func(log Logger) { 999 log.Printf("looking up offset of kafka reader for partition %d of %s: %s", r.config.Partition, r.config.Topic, toHumanOffset(offset)) 1000 }) 1001 return offset 1002 } 1003 1004 // Lag returns the lag of the last message returned by ReadMessage, or -1 1005 // if r is backed by a consumer group. 1006 func (r *Reader) Lag() int64 { 1007 if r.useConsumerGroup() { 1008 return -1 1009 } 1010 1011 r.mutex.Lock() 1012 lag := r.lag 1013 r.mutex.Unlock() 1014 return lag 1015 } 1016 1017 // SetOffset changes the offset from which the next batch of messages will be 1018 // read. The method fails with io.ErrClosedPipe if the reader has already been closed. 1019 // 1020 // From version 0.2.0, FirstOffset and LastOffset can be used to indicate the first 1021 // or last available offset in the partition. Please note while -1 and -2 were accepted 1022 // to indicate the first or last offset in previous versions, the meanings of the numbers 1023 // were swapped in 0.2.0 to match the meanings in other libraries and the Kafka protocol 1024 // specification. 1025 func (r *Reader) SetOffset(offset int64) error { 1026 if r.useConsumerGroup() { 1027 return errNotAvailableWithGroup 1028 } 1029 1030 var err error 1031 r.mutex.Lock() 1032 1033 if r.closed { 1034 err = io.ErrClosedPipe 1035 } else if offset != r.offset { 1036 r.withLogger(func(log Logger) { 1037 log.Printf("setting the offset of the kafka reader for partition %d of %s from %s to %s", 1038 r.config.Partition, r.config.Topic, toHumanOffset(r.offset), toHumanOffset(offset)) 1039 }) 1040 r.offset = offset 1041 1042 if r.version != 0 { 1043 r.start(r.getTopicPartitionOffset()) 1044 } 1045 1046 r.activateReadLag() 1047 } 1048 1049 r.mutex.Unlock() 1050 return err 1051 } 1052 1053 // SetOffsetAt changes the offset from which the next batch of messages will be 1054 // read given the timestamp t. 1055 // 1056 // The method fails if the unable to connect partition leader, or unable to read the offset 1057 // given the ts, or if the reader has been closed. 1058 func (r *Reader) SetOffsetAt(ctx context.Context, t time.Time) error { 1059 r.mutex.Lock() 1060 if r.closed { 1061 r.mutex.Unlock() 1062 return io.ErrClosedPipe 1063 } 1064 r.mutex.Unlock() 1065 1066 if len(r.config.Brokers) < 1 { 1067 return errors.New("no brokers in config") 1068 } 1069 var conn *Conn 1070 var err error 1071 for _, broker := range r.config.Brokers { 1072 conn, err = r.config.Dialer.DialLeader(ctx, "tcp", broker, r.config.Topic, r.config.Partition) 1073 if err != nil { 1074 continue 1075 } 1076 deadline, _ := ctx.Deadline() 1077 conn.SetDeadline(deadline) 1078 offset, err := conn.ReadOffset(t) 1079 conn.Close() 1080 if err != nil { 1081 return err 1082 } 1083 1084 return r.SetOffset(offset) 1085 } 1086 return fmt.Errorf("error dialing all brokers, one of the errors: %w", err) 1087 } 1088 1089 // Stats returns a snapshot of the reader stats since the last time the method 1090 // was called, or since the reader was created if it is called for the first 1091 // time. 1092 // 1093 // A typical use of this method is to spawn a goroutine that will periodically 1094 // call Stats on a kafka reader and report the metrics to a stats collection 1095 // system. 1096 func (r *Reader) Stats() ReaderStats { 1097 stats := ReaderStats{ 1098 Dials: r.stats.dials.snapshot(), 1099 Fetches: r.stats.fetches.snapshot(), 1100 Messages: r.stats.messages.snapshot(), 1101 Bytes: r.stats.bytes.snapshot(), 1102 Rebalances: r.stats.rebalances.snapshot(), 1103 Timeouts: r.stats.timeouts.snapshot(), 1104 Errors: r.stats.errors.snapshot(), 1105 DialTime: r.stats.dialTime.snapshotDuration(), 1106 ReadTime: r.stats.readTime.snapshotDuration(), 1107 WaitTime: r.stats.waitTime.snapshotDuration(), 1108 FetchSize: r.stats.fetchSize.snapshot(), 1109 FetchBytes: r.stats.fetchBytes.snapshot(), 1110 Offset: r.stats.offset.snapshot(), 1111 Lag: r.stats.lag.snapshot(), 1112 MinBytes: int64(r.config.MinBytes), 1113 MaxBytes: int64(r.config.MaxBytes), 1114 MaxWait: r.config.MaxWait, 1115 QueueLength: int64(len(r.msgs)), 1116 QueueCapacity: int64(cap(r.msgs)), 1117 ClientID: r.config.Dialer.ClientID, 1118 Topic: r.config.Topic, 1119 Partition: r.stats.partition, 1120 } 1121 // TODO: remove when we get rid of the deprecated field. 1122 stats.DeprecatedFetchesWithTypo = stats.Fetches 1123 return stats 1124 } 1125 1126 func (r *Reader) getTopicPartitionOffset() map[topicPartition]int64 { 1127 key := topicPartition{topic: r.config.Topic, partition: int32(r.config.Partition)} 1128 return map[topicPartition]int64{key: r.offset} 1129 } 1130 1131 func (r *Reader) withLogger(do func(Logger)) { 1132 if r.config.Logger != nil { 1133 do(r.config.Logger) 1134 } 1135 } 1136 1137 func (r *Reader) withErrorLogger(do func(Logger)) { 1138 if r.config.ErrorLogger != nil { 1139 do(r.config.ErrorLogger) 1140 } else { 1141 r.withLogger(do) 1142 } 1143 } 1144 1145 func (r *Reader) activateReadLag() { 1146 if r.config.ReadLagInterval > 0 && atomic.CompareAndSwapUint32(&r.once, 0, 1) { 1147 // read lag will only be calculated when not using consumer groups 1148 // todo discuss how capturing read lag should interact with rebalancing 1149 if !r.useConsumerGroup() { 1150 go r.readLag(r.stctx) 1151 } 1152 } 1153 } 1154 1155 func (r *Reader) readLag(ctx context.Context) { 1156 ticker := time.NewTicker(r.config.ReadLagInterval) 1157 defer ticker.Stop() 1158 1159 for { 1160 timeout, cancel := context.WithTimeout(ctx, r.config.ReadLagInterval/2) 1161 lag, err := r.ReadLag(timeout) 1162 cancel() 1163 1164 if err != nil { 1165 r.stats.errors.observe(1) 1166 r.withErrorLogger(func(log Logger) { 1167 log.Printf("kafka reader failed to read lag of partition %d of %s: %s", r.config.Partition, r.config.Topic, err) 1168 }) 1169 } else { 1170 r.stats.lag.observe(lag) 1171 } 1172 1173 select { 1174 case <-ticker.C: 1175 case <-ctx.Done(): 1176 return 1177 } 1178 } 1179 } 1180 1181 func (r *Reader) start(offsetsByPartition map[topicPartition]int64) { 1182 if r.closed { 1183 // don't start child reader if parent Reader is closed 1184 return 1185 } 1186 1187 ctx, cancel := context.WithCancel(context.Background()) 1188 1189 r.cancel() // always cancel the previous reader 1190 r.cancel = cancel 1191 r.version++ 1192 1193 r.join.Add(len(offsetsByPartition)) 1194 for key, offset := range offsetsByPartition { 1195 go func(ctx context.Context, key topicPartition, offset int64, join *sync.WaitGroup) { 1196 defer join.Done() 1197 1198 (&reader{ 1199 dialer: r.config.Dialer, 1200 logger: r.config.Logger, 1201 errorLogger: r.config.ErrorLogger, 1202 brokers: r.config.Brokers, 1203 topic: key.topic, 1204 partition: int(key.partition), 1205 minBytes: r.config.MinBytes, 1206 maxBytes: r.config.MaxBytes, 1207 maxWait: r.config.MaxWait, 1208 readBatchTimeout: r.config.ReadBatchTimeout, 1209 backoffDelayMin: r.config.ReadBackoffMin, 1210 backoffDelayMax: r.config.ReadBackoffMax, 1211 version: r.version, 1212 msgs: r.msgs, 1213 stats: r.stats, 1214 isolationLevel: r.config.IsolationLevel, 1215 maxAttempts: r.config.MaxAttempts, 1216 1217 // backwards-compatibility flags 1218 offsetOutOfRangeError: r.config.OffsetOutOfRangeError, 1219 }).run(ctx, offset) 1220 }(ctx, key, offset, &r.join) 1221 } 1222 } 1223 1224 // A reader reads messages from kafka and produces them on its channels, it's 1225 // used as a way to asynchronously fetch messages while the main program reads 1226 // them using the high level reader API. 1227 type reader struct { 1228 dialer *Dialer 1229 logger Logger 1230 errorLogger Logger 1231 brokers []string 1232 topic string 1233 partition int 1234 minBytes int 1235 maxBytes int 1236 maxWait time.Duration 1237 readBatchTimeout time.Duration 1238 backoffDelayMin time.Duration 1239 backoffDelayMax time.Duration 1240 version int64 1241 msgs chan<- readerMessage 1242 stats *readerStats 1243 isolationLevel IsolationLevel 1244 maxAttempts int 1245 1246 offsetOutOfRangeError bool 1247 } 1248 1249 type readerMessage struct { 1250 version int64 1251 message Message 1252 watermark int64 1253 error error 1254 } 1255 1256 func (r *reader) run(ctx context.Context, offset int64) { 1257 // This is the reader's main loop, it only ends if the context is canceled 1258 // and will keep attempting to reader messages otherwise. 1259 // 1260 // Retrying indefinitely has the nice side effect of preventing Read calls 1261 // on the parent reader to block if connection to the kafka server fails, 1262 // the reader keeps reporting errors on the error channel which will then 1263 // be surfaced to the program. 1264 // If the reader wasn't retrying then the program would block indefinitely 1265 // on a Read call after reading the first error. 1266 for attempt := 0; true; attempt++ { 1267 if attempt != 0 { 1268 if !sleep(ctx, backoff(attempt, r.backoffDelayMin, r.backoffDelayMax)) { 1269 return 1270 } 1271 } 1272 1273 r.withLogger(func(log Logger) { 1274 log.Printf("initializing kafka reader for partition %d of %s starting at offset %d", r.partition, r.topic, toHumanOffset(offset)) 1275 }) 1276 1277 conn, start, err := r.initialize(ctx, offset) 1278 if err != nil { 1279 if errors.Is(err, OffsetOutOfRange) { 1280 if r.offsetOutOfRangeError { 1281 r.sendError(ctx, err) 1282 return 1283 } 1284 1285 // This would happen if the requested offset is passed the last 1286 // offset on the partition leader. In that case we're just going 1287 // to retry later hoping that enough data has been produced. 1288 r.withErrorLogger(func(log Logger) { 1289 log.Printf("error initializing the kafka reader for partition %d of %s: %s", r.partition, r.topic, err) 1290 }) 1291 1292 continue 1293 } 1294 1295 // Perform a configured number of attempts before 1296 // reporting first errors, this helps mitigate 1297 // situations where the kafka server is temporarily 1298 // unavailable. 1299 if attempt >= r.maxAttempts { 1300 r.sendError(ctx, err) 1301 } else { 1302 r.stats.errors.observe(1) 1303 r.withErrorLogger(func(log Logger) { 1304 log.Printf("error initializing the kafka reader for partition %d of %s: %s", r.partition, r.topic, err) 1305 }) 1306 } 1307 continue 1308 } 1309 1310 // Resetting the attempt counter ensures that if a failure occurs after 1311 // a successful initialization we don't keep increasing the backoff 1312 // timeout. 1313 attempt = 0 1314 1315 // Now we're sure to have an absolute offset number, may anything happen 1316 // to the connection we know we'll want to restart from this offset. 1317 offset = start 1318 1319 errcount := 0 1320 readLoop: 1321 for { 1322 if !sleep(ctx, backoff(errcount, r.backoffDelayMin, r.backoffDelayMax)) { 1323 conn.Close() 1324 return 1325 } 1326 1327 offset, err = r.read(ctx, offset, conn) 1328 switch { 1329 case err == nil: 1330 errcount = 0 1331 continue 1332 1333 case errors.Is(err, io.EOF): 1334 // done with this batch of messages...carry on. note that this 1335 // block relies on the batch repackaging real io.EOF errors as 1336 // io.UnexpectedEOF. otherwise, we would end up swallowing real 1337 // errors here. 1338 errcount = 0 1339 continue 1340 1341 case errors.Is(err, io.ErrNoProgress): 1342 // This error is returned by the Conn when it believes the connection 1343 // has been corrupted, so we need to explicitly close it. Since we are 1344 // explicitly handling it and a retry will pick up, we can suppress the 1345 // error metrics and logs for this case. 1346 conn.Close() 1347 break readLoop 1348 1349 case errors.Is(err, UnknownTopicOrPartition): 1350 r.withErrorLogger(func(log Logger) { 1351 log.Printf("failed to read from current broker %v for partition %d of %s at offset %d: %v", r.brokers, r.partition, r.topic, toHumanOffset(offset), err) 1352 }) 1353 1354 conn.Close() 1355 1356 // The next call to .initialize will re-establish a connection to the proper 1357 // topic/partition broker combo. 1358 r.stats.rebalances.observe(1) 1359 break readLoop 1360 1361 case errors.Is(err, NotLeaderForPartition): 1362 r.withErrorLogger(func(log Logger) { 1363 log.Printf("failed to read from current broker for partition %d of %s at offset %d: %v", r.partition, r.topic, toHumanOffset(offset), err) 1364 }) 1365 1366 conn.Close() 1367 1368 // The next call to .initialize will re-establish a connection to the proper 1369 // partition leader. 1370 r.stats.rebalances.observe(1) 1371 break readLoop 1372 1373 case errors.Is(err, RequestTimedOut): 1374 // Timeout on the kafka side, this can be safely retried. 1375 errcount = 0 1376 r.withLogger(func(log Logger) { 1377 log.Printf("no messages received from kafka within the allocated time for partition %d of %s at offset %d: %v", r.partition, r.topic, toHumanOffset(offset), err) 1378 }) 1379 r.stats.timeouts.observe(1) 1380 continue 1381 1382 case errors.Is(err, OffsetOutOfRange): 1383 first, last, err := r.readOffsets(conn) 1384 if err != nil { 1385 r.withErrorLogger(func(log Logger) { 1386 log.Printf("the kafka reader got an error while attempting to determine whether it was reading before the first offset or after the last offset of partition %d of %s: %s", r.partition, r.topic, err) 1387 }) 1388 conn.Close() 1389 break readLoop 1390 } 1391 1392 switch { 1393 case offset < first: 1394 r.withErrorLogger(func(log Logger) { 1395 log.Printf("the kafka reader is reading before the first offset for partition %d of %s, skipping from offset %d to %d (%d messages)", r.partition, r.topic, toHumanOffset(offset), first, first-offset) 1396 }) 1397 offset, errcount = first, 0 1398 continue // retry immediately so we don't keep falling behind due to the backoff 1399 1400 case offset < last: 1401 errcount = 0 1402 continue // more messages have already become available, retry immediately 1403 1404 default: 1405 // We may be reading past the last offset, will retry later. 1406 r.withErrorLogger(func(log Logger) { 1407 log.Printf("the kafka reader is reading passed the last offset for partition %d of %s at offset %d", r.partition, r.topic, toHumanOffset(offset)) 1408 }) 1409 } 1410 1411 case errors.Is(err, context.Canceled): 1412 // Another reader has taken over, we can safely quit. 1413 conn.Close() 1414 return 1415 1416 case errors.Is(err, errUnknownCodec): 1417 // The compression codec is either unsupported or has not been 1418 // imported. This is a fatal error b/c the reader cannot 1419 // proceed. 1420 r.sendError(ctx, err) 1421 break readLoop 1422 1423 default: 1424 var kafkaError Error 1425 if errors.As(err, &kafkaError) { 1426 r.sendError(ctx, err) 1427 } else { 1428 r.withErrorLogger(func(log Logger) { 1429 log.Printf("the kafka reader got an unknown error reading partition %d of %s at offset %d: %s", r.partition, r.topic, toHumanOffset(offset), err) 1430 }) 1431 r.stats.errors.observe(1) 1432 conn.Close() 1433 break readLoop 1434 } 1435 } 1436 1437 errcount++ 1438 } 1439 } 1440 } 1441 1442 func (r *reader) initialize(ctx context.Context, offset int64) (conn *Conn, start int64, err error) { 1443 for i := 0; i != len(r.brokers) && conn == nil; i++ { 1444 broker := r.brokers[i] 1445 var first, last int64 1446 1447 t0 := time.Now() 1448 conn, err = r.dialer.DialLeader(ctx, "tcp", broker, r.topic, r.partition) 1449 t1 := time.Now() 1450 r.stats.dials.observe(1) 1451 r.stats.dialTime.observeDuration(t1.Sub(t0)) 1452 1453 if err != nil { 1454 continue 1455 } 1456 1457 if first, last, err = r.readOffsets(conn); err != nil { 1458 conn.Close() 1459 conn = nil 1460 break 1461 } 1462 1463 switch { 1464 case offset == FirstOffset: 1465 offset = first 1466 1467 case offset == LastOffset: 1468 offset = last 1469 1470 case offset < first: 1471 offset = first 1472 } 1473 1474 r.withLogger(func(log Logger) { 1475 log.Printf("the kafka reader for partition %d of %s is seeking to offset %d", r.partition, r.topic, toHumanOffset(offset)) 1476 }) 1477 1478 if start, err = conn.Seek(offset, SeekAbsolute); err != nil { 1479 conn.Close() 1480 conn = nil 1481 break 1482 } 1483 1484 conn.SetDeadline(time.Time{}) 1485 } 1486 1487 return 1488 } 1489 1490 func (r *reader) read(ctx context.Context, offset int64, conn *Conn) (int64, error) { 1491 r.stats.fetches.observe(1) 1492 r.stats.offset.observe(offset) 1493 1494 t0 := time.Now() 1495 conn.SetReadDeadline(t0.Add(r.maxWait)) 1496 1497 batch := conn.ReadBatchWith(ReadBatchConfig{ 1498 MinBytes: r.minBytes, 1499 MaxBytes: r.maxBytes, 1500 IsolationLevel: r.isolationLevel, 1501 }) 1502 highWaterMark := batch.HighWaterMark() 1503 1504 t1 := time.Now() 1505 r.stats.waitTime.observeDuration(t1.Sub(t0)) 1506 1507 var msg Message 1508 var err error 1509 var size int64 1510 var bytes int64 1511 1512 for { 1513 conn.SetReadDeadline(time.Now().Add(r.readBatchTimeout)) 1514 1515 if msg, err = batch.ReadMessage(); err != nil { 1516 batch.Close() 1517 break 1518 } 1519 1520 n := int64(len(msg.Key) + len(msg.Value)) 1521 r.stats.messages.observe(1) 1522 r.stats.bytes.observe(n) 1523 1524 if err = r.sendMessage(ctx, msg, highWaterMark); err != nil { 1525 batch.Close() 1526 break 1527 } 1528 1529 offset = msg.Offset + 1 1530 r.stats.offset.observe(offset) 1531 r.stats.lag.observe(highWaterMark - offset) 1532 1533 size++ 1534 bytes += n 1535 } 1536 1537 conn.SetReadDeadline(time.Time{}) 1538 1539 t2 := time.Now() 1540 r.stats.readTime.observeDuration(t2.Sub(t1)) 1541 r.stats.fetchSize.observe(size) 1542 r.stats.fetchBytes.observe(bytes) 1543 return offset, err 1544 } 1545 1546 func (r *reader) readOffsets(conn *Conn) (first, last int64, err error) { 1547 conn.SetDeadline(time.Now().Add(10 * time.Second)) 1548 return conn.ReadOffsets() 1549 } 1550 1551 func (r *reader) sendMessage(ctx context.Context, msg Message, watermark int64) error { 1552 select { 1553 case r.msgs <- readerMessage{version: r.version, message: msg, watermark: watermark}: 1554 return nil 1555 case <-ctx.Done(): 1556 return ctx.Err() 1557 } 1558 } 1559 1560 func (r *reader) sendError(ctx context.Context, err error) error { 1561 select { 1562 case r.msgs <- readerMessage{version: r.version, error: err}: 1563 return nil 1564 case <-ctx.Done(): 1565 return ctx.Err() 1566 } 1567 } 1568 1569 func (r *reader) withLogger(do func(Logger)) { 1570 if r.logger != nil { 1571 do(r.logger) 1572 } 1573 } 1574 1575 func (r *reader) withErrorLogger(do func(Logger)) { 1576 if r.errorLogger != nil { 1577 do(r.errorLogger) 1578 } else { 1579 r.withLogger(do) 1580 } 1581 } 1582 1583 // extractTopics returns the unique list of topics represented by the set of 1584 // provided members. 1585 func extractTopics(members []GroupMember) []string { 1586 visited := map[string]struct{}{} 1587 var topics []string 1588 1589 for _, member := range members { 1590 for _, topic := range member.Topics { 1591 if _, seen := visited[topic]; seen { 1592 continue 1593 } 1594 1595 topics = append(topics, topic) 1596 visited[topic] = struct{}{} 1597 } 1598 } 1599 1600 sort.Strings(topics) 1601 1602 return topics 1603 } 1604 1605 type humanOffset int64 1606 1607 func toHumanOffset(v int64) humanOffset { 1608 return humanOffset(v) 1609 } 1610 1611 func (offset humanOffset) Format(w fmt.State, _ rune) { 1612 v := int64(offset) 1613 switch v { 1614 case FirstOffset: 1615 fmt.Fprint(w, "first offset") 1616 case LastOffset: 1617 fmt.Fprint(w, "last offset") 1618 default: 1619 fmt.Fprint(w, strconv.FormatInt(v, 10)) 1620 } 1621 }