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