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