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