12345678910111213141516171819202122232425262728293031323334353637383940414243444546474849505152535455565758596061626364656667686970717273747576777879808182838485868788899091929394959697989910010110210310410510610710810911011111211311411511611711811912012112212312412512612712812913013113213313413513613713813914014114214314414514614714814915015115215315415515615715815916016116216316416516616716816917017117217317417517617717817918018118218318418518618718818919019119219319419519619719819920020120220320420520620720820921021121221321421521621721821922022122222322422522622722822923023123223323423523623723823924024124224324424524624724824925025125225325425525625725825926026126226326426526626726826927027127227327427527627727827928028128228328428528628728828929029129229329429529629729829930030130230330430530630730830931031131231331431531631731831932032132232332432532632732832933033133233333433533633733833934034134234334434534634734834935035135235335435535635735835936036136236336436536636736836937037137237337437537637737837938038138238338438538638738838939039139239339439539639739839940040140240340440540640740840941041141241341441541641741841942042142242342442542642742842943043143243343443543643743843944044144244344444544644744844945045145245345445545645745845946046146246346446546646746846947047147247347447547647747847948048148248348448548648748848949049149249349449549649749849950050150250350450550650750850951051151251351451551651751851952052152252352452552652752852953053153253353453553653753853954054154254354454554654754854955055155255355455555655755855956056156256356456556656756856957057157257357457557657757857958058158258358458558658758858959059159259359459559659759859960060160260360460560660760860961061161261361461561661761861962062162262362462562662762862963063163263363463563663763863964064164264364464564664764864965065165265365465565665765865966066166266366466566666766866967067167267367467567667767867968068168268368468568668768868969069169269369469569669769869970070170270370470570670770870971071171271371471571671771871972072172272372472572672772872973073173273373473573673773873974074174274374474574674774874975075175275375475575675775875976076176276376476576676776876977077177277377477577677777877978078178278378478578678778878979079179279379479579679779879980080180280380480580680780880981081181281381481581681781881982082182282382482582682782882983083183283383483583683783883984084184284384484584684784884985085185285385485585685785885986086186286386486586686786886987087187287387487587687787887988088188288388488588688788888989089189289389489589689789889990090190290390490590690790890991091191291391491591691791891992092192292392492592692792892993093193293393493593693793893994094194294394494594694794894995095195295395495595695795895996096196296396496596696796896997097197297397497597697797897998098198298398498598698798898999099199299399499599699799899910001001100210031004100510061007100810091010101110121013101410151016101710181019102010211022102310241025102610271028102910301031103210331034103510361037103810391040104110421043104410451046104710481049105010511052105310541055105610571058105910601061106210631064106510661067106810691070107110721073107410751076107710781079108010811082108310841085108610871088108910901091109210931094109510961097109810991100110111021103110411051106110711081109111011111112111311141115111611171118111911201121112211231124112511261127112811291130113111321133113411351136113711381139114011411142114311441145114611471148114911501151115211531154115511561157115811591160116111621163116411651166116711681169117011711172117311741175117611771178117911801181118211831184118511861187118811891190119111921193119411951196119711981199120012011202120312041205120612071208120912101211121212131214121512161217121812191220122112221223122412251226122712281229123012311232123312341235123612371238123912401241124212431244124512461247124812491250125112521253125412551256125712581259126012611262126312641265126612671268126912701271127212731274127512761277127812791280128112821283128412851286128712881289129012911292129312941295129612971298129913001301130213031304130513061307130813091310131113121313131413151316131713181319132013211322132313241325132613271328132913301331133213331334133513361337133813391340134113421343134413451346134713481349135013511352135313541355135613571358135913601361136213631364136513661367136813691370137113721373137413751376137713781379138013811382138313841385138613871388138913901391139213931394139513961397139813991400140114021403140414051406140714081409141014111412141314141415141614171418141914201421142214231424142514261427142814291430143114321433143414351436143714381439144014411442144314441445144614471448144914501451145214531454145514561457145814591460146114621463146414651466146714681469147014711472147314741475147614771478147914801481148214831484148514861487148814891490149114921493149414951496149714981499150015011502 |
- package kafka
- import (
- "context"
- "errors"
- "fmt"
- "io"
- "math"
- "sort"
- "strconv"
- "sync"
- "sync/atomic"
- "time"
- )
- const (
- LastOffset int64 = -1
- FirstOffset int64 = -2
- )
- const (
-
-
- defaultCommitRetries = 3
- )
- const (
-
-
-
- defaultFetchMinBytes = 1
- )
- var (
- errOnlyAvailableWithGroup = errors.New("unavailable when GroupID is not set")
- errNotAvailableWithGroup = errors.New("unavailable when GroupID is set")
- )
- const (
-
-
- defaultReadBackoffMin = 100 * time.Millisecond
- defaultReadBackoffMax = 1 * time.Second
- )
- type Reader struct {
-
- config ReaderConfig
-
- msgs chan readerMessage
-
- mutex sync.Mutex
- join sync.WaitGroup
- cancel context.CancelFunc
- stop context.CancelFunc
- done chan struct{}
- commits chan commitRequest
- version int64
- offset int64
- lag int64
- closed bool
-
-
-
-
-
-
-
-
- runError chan error
-
- once uint32
- stctx context.Context
-
-
- stats *readerStats
- }
- func (r *Reader) useConsumerGroup() bool { return r.config.GroupID != "" }
- func (r *Reader) useSyncCommits() bool { return r.config.CommitInterval == 0 }
- func (r *Reader) unsubscribe() {
- r.cancel()
- r.join.Wait()
-
-
-
-
-
-
-
-
- }
- func (r *Reader) subscribe(assignments []PartitionAssignment) {
- offsetsByPartition := make(map[int]int64)
- for _, assignment := range assignments {
- offsetsByPartition[assignment.ID] = assignment.Offset
- }
- r.mutex.Lock()
- r.start(offsetsByPartition)
- r.mutex.Unlock()
- r.withLogger(func(l Logger) {
- l.Printf("subscribed to partitions: %+v", offsetsByPartition)
- })
- }
- func (r *Reader) waitThrottleTime(throttleTimeMS int32) {
- if throttleTimeMS == 0 {
- return
- }
- t := time.NewTimer(time.Duration(throttleTimeMS) * time.Millisecond)
- defer t.Stop()
- select {
- case <-r.stctx.Done():
- return
- case <-t.C:
- }
- }
- func (r *Reader) commitOffsetsWithRetry(gen *Generation, offsetStash offsetStash, retries int) (err error) {
- const (
- backoffDelayMin = 100 * time.Millisecond
- backoffDelayMax = 5 * time.Second
- )
- for attempt := 0; attempt < retries; attempt++ {
- if attempt != 0 {
- if !sleep(r.stctx, backoff(attempt, backoffDelayMin, backoffDelayMax)) {
- return
- }
- }
- if err = gen.CommitOffsets(offsetStash); err == nil {
- return
- }
- }
- return
- }
- type offsetStash map[string]map[int]int64
- func (o offsetStash) merge(commits []commit) {
- for _, c := range commits {
- offsetsByPartition, ok := o[c.topic]
- if !ok {
- offsetsByPartition = map[int]int64{}
- o[c.topic] = offsetsByPartition
- }
- if offset, ok := offsetsByPartition[c.partition]; !ok || c.offset > offset {
- offsetsByPartition[c.partition] = c.offset
- }
- }
- }
- func (o offsetStash) reset() {
- for key := range o {
- delete(o, key)
- }
- }
- func (r *Reader) commitLoopImmediate(ctx context.Context, gen *Generation) {
- offsets := offsetStash{}
- for {
- select {
- case <-ctx.Done():
- return
- case req := <-r.commits:
- offsets.merge(req.commits)
- req.errch <- r.commitOffsetsWithRetry(gen, offsets, defaultCommitRetries)
- offsets.reset()
- }
- }
- }
- func (r *Reader) commitLoopInterval(ctx context.Context, gen *Generation) {
- ticker := time.NewTicker(r.config.CommitInterval)
- defer ticker.Stop()
-
-
- offsets := offsetStash{}
- commit := func() {
- if err := r.commitOffsetsWithRetry(gen, offsets, defaultCommitRetries); err != nil {
- r.withErrorLogger(func(l Logger) { l.Printf(err.Error()) })
- } else {
- offsets.reset()
- }
- }
- for {
- select {
- case <-ctx.Done():
-
- for hasCommits := true; hasCommits; {
- select {
- case req := <-r.commits:
- offsets.merge(req.commits)
- default:
- hasCommits = false
- }
- }
- commit()
- return
- case <-ticker.C:
- commit()
- case req := <-r.commits:
- offsets.merge(req.commits)
- }
- }
- }
- func (r *Reader) commitLoop(ctx context.Context, gen *Generation) {
- r.withLogger(func(l Logger) {
- l.Printf("started commit for group %s\n", r.config.GroupID)
- })
- defer r.withLogger(func(l Logger) {
- l.Printf("stopped commit for group %s\n", r.config.GroupID)
- })
- if r.config.CommitInterval == 0 {
- r.commitLoopImmediate(ctx, gen)
- } else {
- r.commitLoopInterval(ctx, gen)
- }
- }
- func (r *Reader) run(cg *ConsumerGroup) {
- defer close(r.done)
- defer cg.Close()
- r.withLogger(func(l Logger) {
- l.Printf("entering loop for consumer group, %v\n", r.config.GroupID)
- })
- for {
-
-
- var err error
- var gen *Generation
- for attempt := 1; attempt <= r.config.MaxAttempts; attempt++ {
- gen, err = cg.Next(r.stctx)
- if err == nil {
- break
- }
- if err == r.stctx.Err() {
- return
- }
- r.stats.errors.observe(1)
- r.withErrorLogger(func(l Logger) {
- l.Printf(err.Error())
- })
-
- }
- if err != nil {
-
- select {
- case r.runError <- err:
-
-
- default:
-
- }
- continue
- }
- r.stats.rebalances.observe(1)
- r.subscribe(gen.Assignments[r.config.Topic])
- gen.Start(func(ctx context.Context) {
- r.commitLoop(ctx, gen)
- })
- gen.Start(func(ctx context.Context) {
-
- select {
- case <-ctx.Done():
-
- case <-r.stctx.Done():
-
- }
- r.unsubscribe()
- })
- }
- }
- type ReaderConfig struct {
-
- Brokers []string
-
-
- GroupID string
-
- Topic string
-
-
- Partition int
-
-
- Dialer *Dialer
-
-
- QueueCapacity int
-
- MinBytes int
- MaxBytes int
-
-
- MaxWait time.Duration
-
-
- ReadLagInterval time.Duration
-
-
-
-
-
-
-
- GroupBalancers []GroupBalancer
-
-
-
-
-
-
- HeartbeatInterval time.Duration
-
-
-
-
-
-
- CommitInterval time.Duration
-
-
-
-
-
-
-
- PartitionWatchInterval time.Duration
-
-
- WatchPartitionChanges bool
-
-
-
-
-
-
- SessionTimeout time.Duration
-
-
-
-
-
-
-
- RebalanceTimeout time.Duration
-
-
-
-
- JoinGroupBackoff time.Duration
-
-
-
-
-
-
- RetentionTime time.Duration
-
-
-
-
-
-
-
- StartOffset int64
-
-
-
-
- ReadBackoffMin time.Duration
-
-
-
-
- ReadBackoffMax time.Duration
-
-
- Logger Logger
-
-
- ErrorLogger Logger
-
-
-
- IsolationLevel IsolationLevel
-
-
-
- MaxAttempts int
- }
- func (config *ReaderConfig) Validate() error {
- if len(config.Brokers) == 0 {
- return errors.New("cannot create a new kafka reader with an empty list of broker addresses")
- }
- if len(config.Topic) == 0 {
- return errors.New("cannot create a new kafka reader with an empty topic")
- }
- if config.Partition < 0 || config.Partition >= math.MaxInt32 {
- return errors.New(fmt.Sprintf("partition number out of bounds: %d", config.Partition))
- }
- if config.MinBytes < 0 {
- return errors.New(fmt.Sprintf("invalid negative minimum batch size (min = %d)", config.MinBytes))
- }
- if config.MaxBytes < 0 {
- return errors.New(fmt.Sprintf("invalid negative maximum batch size (max = %d)", config.MaxBytes))
- }
- if config.GroupID != "" && config.Partition != 0 {
- return errors.New("either Partition or GroupID may be specified, but not both")
- }
- if config.MinBytes > config.MaxBytes {
- return errors.New(fmt.Sprintf("minimum batch size greater than the maximum (min = %d, max = %d)", config.MinBytes, config.MaxBytes))
- }
- if config.ReadBackoffMax < 0 {
- return errors.New(fmt.Sprintf("ReadBackoffMax out of bounds: %d", config.ReadBackoffMax))
- }
- if config.ReadBackoffMin < 0 {
- return errors.New(fmt.Sprintf("ReadBackoffMin out of bounds: %d", config.ReadBackoffMin))
- }
- return nil
- }
- type ReaderStats struct {
- Dials int64 `metric:"kafka.reader.dial.count" type:"counter"`
- Fetches int64 `metric:"kafka.reader.fetch.count" type:"counter"`
- Messages int64 `metric:"kafka.reader.message.count" type:"counter"`
- Bytes int64 `metric:"kafka.reader.message.bytes" type:"counter"`
- Rebalances int64 `metric:"kafka.reader.rebalance.count" type:"counter"`
- Timeouts int64 `metric:"kafka.reader.timeout.count" type:"counter"`
- Errors int64 `metric:"kafka.reader.error.count" type:"counter"`
- DialTime DurationStats `metric:"kafka.reader.dial.seconds"`
- ReadTime DurationStats `metric:"kafka.reader.read.seconds"`
- WaitTime DurationStats `metric:"kafka.reader.wait.seconds"`
- FetchSize SummaryStats `metric:"kafka.reader.fetch.size"`
- FetchBytes SummaryStats `metric:"kafka.reader.fetch.bytes"`
- Offset int64 `metric:"kafka.reader.offset" type:"gauge"`
- Lag int64 `metric:"kafka.reader.lag" type:"gauge"`
- MinBytes int64 `metric:"kafka.reader.fetch_bytes.min" type:"gauge"`
- MaxBytes int64 `metric:"kafka.reader.fetch_bytes.max" type:"gauge"`
- MaxWait time.Duration `metric:"kafka.reader.fetch_wait.max" type:"gauge"`
- QueueLength int64 `metric:"kafka.reader.queue.length" type:"gauge"`
- QueueCapacity int64 `metric:"kafka.reader.queue.capacity" type:"gauge"`
- ClientID string `tag:"client_id"`
- Topic string `tag:"topic"`
- Partition string `tag:"partition"`
-
-
-
-
- DeprecatedFetchesWithTypo int64 `metric:"kafak.reader.fetch.count" type:"counter"`
- }
- type readerStats struct {
- dials counter
- fetches counter
- messages counter
- bytes counter
- rebalances counter
- timeouts counter
- errors counter
- dialTime summary
- readTime summary
- waitTime summary
- fetchSize summary
- fetchBytes summary
- offset gauge
- lag gauge
- partition string
- }
- func NewReader(config ReaderConfig) *Reader {
- if err := config.Validate(); err != nil {
- panic(err)
- }
- if config.GroupID != "" {
- if len(config.GroupBalancers) == 0 {
- config.GroupBalancers = []GroupBalancer{
- RangeGroupBalancer{},
- RoundRobinGroupBalancer{},
- }
- }
- }
- if config.Dialer == nil {
- config.Dialer = DefaultDialer
- }
- if config.MaxBytes == 0 {
- config.MaxBytes = 1e6
- }
- if config.MinBytes == 0 {
- config.MinBytes = defaultFetchMinBytes
- }
- if config.MaxWait == 0 {
- config.MaxWait = 10 * time.Second
- }
- if config.ReadLagInterval == 0 {
- config.ReadLagInterval = 1 * time.Minute
- }
- if config.ReadBackoffMin == 0 {
- config.ReadBackoffMin = defaultReadBackoffMin
- }
- if config.ReadBackoffMax == 0 {
- config.ReadBackoffMax = defaultReadBackoffMax
- }
- if config.ReadBackoffMax < config.ReadBackoffMin {
- panic(fmt.Errorf("ReadBackoffMax %d smaller than ReadBackoffMin %d", config.ReadBackoffMax, config.ReadBackoffMin))
- }
- if config.QueueCapacity == 0 {
- config.QueueCapacity = 100
- }
- if config.MaxAttempts == 0 {
- config.MaxAttempts = 3
- }
-
- readerStatsPartition := config.Partition
- if config.GroupID != "" {
- readerStatsPartition = -1
- }
-
-
- version := int64(0)
- if config.GroupID != "" {
- version = 1
- }
- stctx, stop := context.WithCancel(context.Background())
- r := &Reader{
- config: config,
- msgs: make(chan readerMessage, config.QueueCapacity),
- cancel: func() {},
- commits: make(chan commitRequest, config.QueueCapacity),
- stop: stop,
- offset: FirstOffset,
- stctx: stctx,
- stats: &readerStats{
- dialTime: makeSummary(),
- readTime: makeSummary(),
- waitTime: makeSummary(),
- fetchSize: makeSummary(),
- fetchBytes: makeSummary(),
-
-
- partition: strconv.Itoa(readerStatsPartition),
- },
- version: version,
- }
- if r.useConsumerGroup() {
- r.done = make(chan struct{})
- r.runError = make(chan error)
- cg, err := NewConsumerGroup(ConsumerGroupConfig{
- ID: r.config.GroupID,
- Brokers: r.config.Brokers,
- Dialer: r.config.Dialer,
- Topics: []string{r.config.Topic},
- GroupBalancers: r.config.GroupBalancers,
- HeartbeatInterval: r.config.HeartbeatInterval,
- PartitionWatchInterval: r.config.PartitionWatchInterval,
- WatchPartitionChanges: r.config.WatchPartitionChanges,
- SessionTimeout: r.config.SessionTimeout,
- RebalanceTimeout: r.config.RebalanceTimeout,
- JoinGroupBackoff: r.config.JoinGroupBackoff,
- RetentionTime: r.config.RetentionTime,
- StartOffset: r.config.StartOffset,
- Logger: r.config.Logger,
- ErrorLogger: r.config.ErrorLogger,
- })
- if err != nil {
- panic(err)
- }
- go r.run(cg)
- }
- return r
- }
- func (r *Reader) Config() ReaderConfig {
- return r.config
- }
- func (r *Reader) Close() error {
- atomic.StoreUint32(&r.once, 1)
- r.mutex.Lock()
- closed := r.closed
- r.closed = true
- r.mutex.Unlock()
- r.cancel()
- r.stop()
- r.join.Wait()
- if r.done != nil {
- <-r.done
- }
- if !closed {
- close(r.msgs)
- }
- return nil
- }
- func (r *Reader) ReadMessage(ctx context.Context) (Message, error) {
- m, err := r.FetchMessage(ctx)
- if err != nil {
- return Message{}, err
- }
- if r.useConsumerGroup() {
- if err := r.CommitMessages(ctx, m); err != nil {
- return Message{}, err
- }
- }
- return m, nil
- }
- func (r *Reader) FetchMessage(ctx context.Context) (Message, error) {
- r.activateReadLag()
- for {
- r.mutex.Lock()
- if !r.closed && r.version == 0 {
- r.start(map[int]int64{r.config.Partition: r.offset})
- }
- version := r.version
- r.mutex.Unlock()
- select {
- case <-ctx.Done():
- return Message{}, ctx.Err()
- case err := <-r.runError:
- return Message{}, err
- case m, ok := <-r.msgs:
- if !ok {
- return Message{}, io.EOF
- }
- if m.version >= version {
- r.mutex.Lock()
- switch {
- case m.error != nil:
- case version == r.version:
- r.offset = m.message.Offset + 1
- r.lag = m.watermark - r.offset
- }
- r.mutex.Unlock()
- switch m.error {
- case nil:
- case io.EOF:
-
-
-
-
- m.error = io.ErrUnexpectedEOF
- }
- return m.message, m.error
- }
- }
- }
- }
- func (r *Reader) CommitMessages(ctx context.Context, msgs ...Message) error {
- if !r.useConsumerGroup() {
- return errOnlyAvailableWithGroup
- }
- var errch <-chan error
- var creq = commitRequest{
- commits: makeCommits(msgs...),
- }
- if r.useSyncCommits() {
- ch := make(chan error, 1)
- errch, creq.errch = ch, ch
- }
- select {
- case r.commits <- creq:
- case <-ctx.Done():
- return ctx.Err()
- case <-r.stctx.Done():
-
-
- return io.ErrClosedPipe
- }
- if !r.useSyncCommits() {
- return nil
- }
- select {
- case <-ctx.Done():
- return ctx.Err()
- case err := <-errch:
- return err
- }
- }
- func (r *Reader) ReadLag(ctx context.Context) (lag int64, err error) {
- if r.useConsumerGroup() {
- return 0, errNotAvailableWithGroup
- }
- type offsets struct {
- first int64
- last int64
- }
- offch := make(chan offsets, 1)
- errch := make(chan error, 1)
- go func() {
- var off offsets
- var err error
- for _, broker := range r.config.Brokers {
- var conn *Conn
- if conn, err = r.config.Dialer.DialLeader(ctx, "tcp", broker, r.config.Topic, r.config.Partition); err != nil {
- continue
- }
- deadline, _ := ctx.Deadline()
- conn.SetDeadline(deadline)
- off.first, off.last, err = conn.ReadOffsets()
- conn.Close()
- if err == nil {
- break
- }
- }
- if err != nil {
- errch <- err
- } else {
- offch <- off
- }
- }()
- select {
- case off := <-offch:
- switch cur := r.Offset(); {
- case cur == FirstOffset:
- lag = off.last - off.first
- case cur == LastOffset:
- lag = 0
- default:
- lag = off.last - cur
- }
- case err = <-errch:
- case <-ctx.Done():
- err = ctx.Err()
- }
- return
- }
- func (r *Reader) Offset() int64 {
- if r.useConsumerGroup() {
- return -1
- }
- r.mutex.Lock()
- offset := r.offset
- r.mutex.Unlock()
- r.withLogger(func(log Logger) {
- log.Printf("looking up offset of kafka reader for partition %d of %s: %d", r.config.Partition, r.config.Topic, offset)
- })
- return offset
- }
- func (r *Reader) Lag() int64 {
- if r.useConsumerGroup() {
- return -1
- }
- r.mutex.Lock()
- lag := r.lag
- r.mutex.Unlock()
- return lag
- }
- func (r *Reader) SetOffset(offset int64) error {
- if r.useConsumerGroup() {
- return errNotAvailableWithGroup
- }
- var err error
- r.mutex.Lock()
- if r.closed {
- err = io.ErrClosedPipe
- } else if offset != r.offset {
- r.withLogger(func(log Logger) {
- log.Printf("setting the offset of the kafka reader for partition %d of %s from %d to %d",
- r.config.Partition, r.config.Topic, r.offset, offset)
- })
- r.offset = offset
- if r.version != 0 {
- r.start(map[int]int64{r.config.Partition: r.offset})
- }
- r.activateReadLag()
- }
- r.mutex.Unlock()
- return err
- }
- func (r *Reader) SetOffsetAt(ctx context.Context, t time.Time) error {
- r.mutex.Lock()
- if r.closed {
- r.mutex.Unlock()
- return io.ErrClosedPipe
- }
- r.mutex.Unlock()
- for _, broker := range r.config.Brokers {
- conn, err := r.config.Dialer.DialLeader(ctx, "tcp", broker, r.config.Topic, r.config.Partition)
- if err != nil {
- continue
- }
- deadline, _ := ctx.Deadline()
- conn.SetDeadline(deadline)
- offset, err := conn.ReadOffset(t)
- conn.Close()
- if err != nil {
- return err
- }
- return r.SetOffset(offset)
- }
- return fmt.Errorf("error setting offset for timestamp %+v", t)
- }
- func (r *Reader) Stats() ReaderStats {
- stats := ReaderStats{
- Dials: r.stats.dials.snapshot(),
- Fetches: r.stats.fetches.snapshot(),
- Messages: r.stats.messages.snapshot(),
- Bytes: r.stats.bytes.snapshot(),
- Rebalances: r.stats.rebalances.snapshot(),
- Timeouts: r.stats.timeouts.snapshot(),
- Errors: r.stats.errors.snapshot(),
- DialTime: r.stats.dialTime.snapshotDuration(),
- ReadTime: r.stats.readTime.snapshotDuration(),
- WaitTime: r.stats.waitTime.snapshotDuration(),
- FetchSize: r.stats.fetchSize.snapshot(),
- FetchBytes: r.stats.fetchBytes.snapshot(),
- Offset: r.stats.offset.snapshot(),
- Lag: r.stats.lag.snapshot(),
- MinBytes: int64(r.config.MinBytes),
- MaxBytes: int64(r.config.MaxBytes),
- MaxWait: r.config.MaxWait,
- QueueLength: int64(len(r.msgs)),
- QueueCapacity: int64(cap(r.msgs)),
- ClientID: r.config.Dialer.ClientID,
- Topic: r.config.Topic,
- Partition: r.stats.partition,
- }
-
- stats.DeprecatedFetchesWithTypo = stats.Fetches
- return stats
- }
- func (r *Reader) withLogger(do func(Logger)) {
- if r.config.Logger != nil {
- do(r.config.Logger)
- }
- }
- func (r *Reader) withErrorLogger(do func(Logger)) {
- if r.config.ErrorLogger != nil {
- do(r.config.ErrorLogger)
- } else {
- r.withLogger(do)
- }
- }
- func (r *Reader) activateReadLag() {
- if r.config.ReadLagInterval > 0 && atomic.CompareAndSwapUint32(&r.once, 0, 1) {
-
-
- if !r.useConsumerGroup() {
- go r.readLag(r.stctx)
- }
- }
- }
- func (r *Reader) readLag(ctx context.Context) {
- ticker := time.NewTicker(r.config.ReadLagInterval)
- defer ticker.Stop()
- for {
- timeout, cancel := context.WithTimeout(ctx, r.config.ReadLagInterval/2)
- lag, err := r.ReadLag(timeout)
- cancel()
- if err != nil {
- r.stats.errors.observe(1)
- r.withErrorLogger(func(log Logger) {
- log.Printf("kafka reader failed to read lag of partition %d of %s: %s", r.config.Partition, r.config.Topic, err)
- })
- } else {
- r.stats.lag.observe(lag)
- }
- select {
- case <-ticker.C:
- case <-ctx.Done():
- return
- }
- }
- }
- func (r *Reader) start(offsetsByPartition map[int]int64) {
- if r.closed {
-
- return
- }
- ctx, cancel := context.WithCancel(context.Background())
- r.cancel()
- r.cancel = cancel
- r.version++
- r.join.Add(len(offsetsByPartition))
- for partition, offset := range offsetsByPartition {
- go func(ctx context.Context, partition int, offset int64, join *sync.WaitGroup) {
- defer join.Done()
- (&reader{
- dialer: r.config.Dialer,
- logger: r.config.Logger,
- errorLogger: r.config.ErrorLogger,
- brokers: r.config.Brokers,
- topic: r.config.Topic,
- partition: partition,
- minBytes: r.config.MinBytes,
- maxBytes: r.config.MaxBytes,
- maxWait: r.config.MaxWait,
- backoffDelayMin: r.config.ReadBackoffMin,
- backoffDelayMax: r.config.ReadBackoffMax,
- version: r.version,
- msgs: r.msgs,
- stats: r.stats,
- isolationLevel: r.config.IsolationLevel,
- maxAttempts: r.config.MaxAttempts,
- }).run(ctx, offset)
- }(ctx, partition, offset, &r.join)
- }
- }
- type reader struct {
- dialer *Dialer
- logger Logger
- errorLogger Logger
- brokers []string
- topic string
- partition int
- minBytes int
- maxBytes int
- maxWait time.Duration
- backoffDelayMin time.Duration
- backoffDelayMax time.Duration
- version int64
- msgs chan<- readerMessage
- stats *readerStats
- isolationLevel IsolationLevel
- maxAttempts int
- }
- type readerMessage struct {
- version int64
- message Message
- watermark int64
- error error
- }
- func (r *reader) run(ctx context.Context, offset int64) {
-
-
-
-
-
-
-
-
-
- for attempt := 0; true; attempt++ {
- if attempt != 0 {
- if !sleep(ctx, backoff(attempt, r.backoffDelayMin, r.backoffDelayMax)) {
- return
- }
- }
- r.withLogger(func(log Logger) {
- log.Printf("initializing kafka reader for partition %d of %s starting at offset %d", r.partition, r.topic, offset)
- })
- conn, start, err := r.initialize(ctx, offset)
- switch err {
- case nil:
- case OffsetOutOfRange:
-
-
-
- r.withErrorLogger(func(log Logger) {
- log.Printf("error initializing the kafka reader for partition %d of %s: %s", r.partition, r.topic, OffsetOutOfRange)
- })
- continue
- default:
-
-
-
-
- if attempt >= r.maxAttempts {
- r.sendError(ctx, err)
- } else {
- r.stats.errors.observe(1)
- r.withErrorLogger(func(log Logger) {
- log.Printf("error initializing the kafka reader for partition %d of %s: %s", r.partition, r.topic, err)
- })
- }
- continue
- }
-
-
-
- attempt = 0
-
-
- offset = start
- errcount := 0
- readLoop:
- for {
- if !sleep(ctx, backoff(errcount, r.backoffDelayMin, r.backoffDelayMax)) {
- conn.Close()
- return
- }
- switch offset, err = r.read(ctx, offset, conn); err {
- case nil:
- errcount = 0
- case io.EOF:
-
-
-
-
- errcount = 0
- case UnknownTopicOrPartition:
- r.withErrorLogger(func(log Logger) {
- 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)
- })
- conn.Close()
-
-
- r.stats.rebalances.observe(1)
- break readLoop
- case NotLeaderForPartition:
- r.withErrorLogger(func(log Logger) {
- log.Printf("failed to read from current broker for partition %d of %s at offset %d, not the leader", r.partition, r.topic, offset)
- })
- conn.Close()
-
-
- r.stats.rebalances.observe(1)
- break readLoop
- case RequestTimedOut:
-
- errcount = 0
- r.withLogger(func(log Logger) {
- log.Printf("no messages received from kafka within the allocated time for partition %d of %s at offset %d", r.partition, r.topic, offset)
- })
- r.stats.timeouts.observe(1)
- continue
- case OffsetOutOfRange:
- first, last, err := r.readOffsets(conn)
- if err != nil {
- r.withErrorLogger(func(log Logger) {
- 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)
- })
- conn.Close()
- break readLoop
- }
- switch {
- case offset < first:
- r.withErrorLogger(func(log Logger) {
- 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)
- })
- offset, errcount = first, 0
- continue
- case offset < last:
- errcount = 0
- continue
- default:
-
- r.withErrorLogger(func(log Logger) {
- log.Printf("the kafka reader is reading passed the last offset for partition %d of %s at offset %d", r.partition, r.topic, offset)
- })
- }
- case context.Canceled:
-
- conn.Close()
- return
- case errUnknownCodec:
-
-
-
- r.sendError(ctx, err)
- break readLoop
- default:
- if _, ok := err.(Error); ok {
- r.sendError(ctx, err)
- } else {
- r.withErrorLogger(func(log Logger) {
- log.Printf("the kafka reader got an unknown error reading partition %d of %s at offset %d: %s", r.partition, r.topic, offset, err)
- })
- r.stats.errors.observe(1)
- conn.Close()
- break readLoop
- }
- }
- errcount++
- }
- }
- }
- func (r *reader) initialize(ctx context.Context, offset int64) (conn *Conn, start int64, err error) {
- for i := 0; i != len(r.brokers) && conn == nil; i++ {
- var broker = r.brokers[i]
- var first, last int64
- t0 := time.Now()
- conn, err = r.dialer.DialLeader(ctx, "tcp", broker, r.topic, r.partition)
- t1 := time.Now()
- r.stats.dials.observe(1)
- r.stats.dialTime.observeDuration(t1.Sub(t0))
- if err != nil {
- continue
- }
- if first, last, err = r.readOffsets(conn); err != nil {
- conn.Close()
- conn = nil
- break
- }
- switch {
- case offset == FirstOffset:
- offset = first
- case offset == LastOffset:
- offset = last
- case offset < first:
- offset = first
- }
- r.withLogger(func(log Logger) {
- log.Printf("the kafka reader for partition %d of %s is seeking to offset %d", r.partition, r.topic, offset)
- })
- if start, err = conn.Seek(offset, SeekAbsolute); err != nil {
- conn.Close()
- conn = nil
- break
- }
- conn.SetDeadline(time.Time{})
- }
- return
- }
- func (r *reader) read(ctx context.Context, offset int64, conn *Conn) (int64, error) {
- r.stats.fetches.observe(1)
- r.stats.offset.observe(offset)
- t0 := time.Now()
- conn.SetReadDeadline(t0.Add(r.maxWait))
- batch := conn.ReadBatchWith(ReadBatchConfig{
- MinBytes: r.minBytes,
- MaxBytes: r.maxBytes,
- IsolationLevel: r.isolationLevel,
- })
- highWaterMark := batch.HighWaterMark()
- t1 := time.Now()
- r.stats.waitTime.observeDuration(t1.Sub(t0))
- var msg Message
- var err error
- var size int64
- var bytes int64
- const safetyTimeout = 10 * time.Second
- deadline := time.Now().Add(safetyTimeout)
- conn.SetReadDeadline(deadline)
- for {
- if now := time.Now(); deadline.Sub(now) < (safetyTimeout / 2) {
- deadline = now.Add(safetyTimeout)
- conn.SetReadDeadline(deadline)
- }
- if msg, err = batch.ReadMessage(); err != nil {
- batch.Close()
- break
- }
- n := int64(len(msg.Key) + len(msg.Value))
- r.stats.messages.observe(1)
- r.stats.bytes.observe(n)
- if err = r.sendMessage(ctx, msg, highWaterMark); err != nil {
- batch.Close()
- break
- }
- offset = msg.Offset + 1
- r.stats.offset.observe(offset)
- r.stats.lag.observe(highWaterMark - offset)
- size++
- bytes += n
- }
- conn.SetReadDeadline(time.Time{})
- t2 := time.Now()
- r.stats.readTime.observeDuration(t2.Sub(t1))
- r.stats.fetchSize.observe(size)
- r.stats.fetchBytes.observe(bytes)
- return offset, err
- }
- func (r *reader) readOffsets(conn *Conn) (first, last int64, err error) {
- conn.SetDeadline(time.Now().Add(10 * time.Second))
- return conn.ReadOffsets()
- }
- func (r *reader) sendMessage(ctx context.Context, msg Message, watermark int64) error {
- select {
- case r.msgs <- readerMessage{version: r.version, message: msg, watermark: watermark}:
- return nil
- case <-ctx.Done():
- return ctx.Err()
- }
- }
- func (r *reader) sendError(ctx context.Context, err error) error {
- select {
- case r.msgs <- readerMessage{version: r.version, error: err}:
- return nil
- case <-ctx.Done():
- return ctx.Err()
- }
- }
- func (r *reader) withLogger(do func(Logger)) {
- if r.logger != nil {
- do(r.logger)
- }
- }
- func (r *reader) withErrorLogger(do func(Logger)) {
- if r.errorLogger != nil {
- do(r.errorLogger)
- } else {
- r.withLogger(do)
- }
- }
- func extractTopics(members []GroupMember) []string {
- var visited = map[string]struct{}{}
- var topics []string
- for _, member := range members {
- for _, topic := range member.Topics {
- if _, seen := visited[topic]; seen {
- continue
- }
- topics = append(topics, topic)
- visited[topic] = struct{}{}
- }
- }
- sort.Strings(topics)
- return topics
- }
|