-
Notifications
You must be signed in to change notification settings - Fork 0
/
reader.go
139 lines (123 loc) · 3.42 KB
/
reader.go
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
package seqreader
import (
"context"
"sync"
"time"
"cardinalby/seq-kafka-reader/tracking"
"cardinalby/seq-kafka-reader/types"
"github.com/segmentio/kafka-go"
)
type Reader struct {
*kafka.Reader
commitInterval time.Duration
internalCtx context.Context
internalCtxCancel context.CancelFunc
waitGroupCancel context.CancelFunc
commitTracker *tracking.TopicCommitTracker
fetchMu sync.Mutex
readyToCommit sync.Cond
}
func NewReader(config kafka.ReaderConfig) *Reader {
commitInterval := config.CommitInterval
// make underlying kafka-go reader always commit synchronously
config.CommitInterval = 0
reader := &Reader{
Reader: kafka.NewReader(config),
commitInterval: commitInterval,
commitTracker: tracking.NewTopicCommitTracker(),
fetchMu: sync.Mutex{},
}
reader.internalCtx, reader.internalCtxCancel = context.WithCancel(context.Background())
if commitInterval > 0 {
go func() {
reader.runCommitLoop(reader.internalCtx)
}()
}
return reader
}
func (r *Reader) FetchMessage(ctx context.Context) (kafka.Message, error) {
r.fetchMu.Lock()
defer r.fetchMu.Unlock()
msg, err := r.Reader.FetchMessage(ctx)
if err != nil {
return msg, err
}
r.commitTracker.OnFetch(&msg)
return msg, nil
}
func (r *Reader) CommitMessages(ctx context.Context, msgs ...kafka.Message) error {
var commitResSub tracking.MessageResultSubs
if r.useSyncCommits() {
// subscribe first to make sure we get the results anyway
commitResSub = r.commitTracker.SubscribeToCommitResult(msgs)
}
if err := r.commitTracker.OnCommitIntent(msgs); err != nil {
// it shouldn't happen if correct msgs are passed
commitResSub.Unsubscribe()
return err
}
if r.useSyncCommits() {
return <-commitResSub.Aggregate(ctx)
}
return nil
}
func (r *Reader) ReadMessage(ctx context.Context) (kafka.Message, error) {
// copy the origin ReadMessage logic
m, err := r.FetchMessage(ctx)
if err != nil {
return kafka.Message{}, err
}
if r.Reader.Config().GroupID != "" {
if err := r.CommitMessages(ctx, m); err != nil {
return kafka.Message{}, err
}
}
return m, nil
}
func (r *Reader) Close() error {
r.internalCtxCancel()
return r.Reader.Close()
}
func (r *Reader) runCommitLoop(ctx context.Context) {
ticker := time.NewTicker(r.commitInterval)
for {
select {
case <-ctx.Done():
ticker.Stop()
return
case <-ticker.C:
if messages := r.commitTracker.PopNextToCommit(); len(messages) > 0 {
msgErrors := r.commit(ctx, messages)
r.withErrorLogger(func(logger kafka.Logger) {
logger.Printf("commit error: %s", msgErrors.Error())
})
}
}
}
}
func (r *Reader) commit(ctx context.Context, messages []kafka.Message) types.MessageErrors {
var messagesError types.MessageErrors
for i := range messages {
msg := &messages[i]
if err := r.Reader.CommitMessages(ctx, *msg); err != nil {
messagesError = append(messagesError, types.MessageError{Message: msg, Err: err})
continue
}
if err := r.commitTracker.OnCommitResult(msg, nil); err != nil {
r.withErrorLogger(func(logger kafka.Logger) {
logger.Printf("error tracking commit success: %s", err.Error())
})
}
}
return messagesError
}
func (r *Reader) useSyncCommits() bool {
return r.commitInterval == 0
}
func (r *Reader) withErrorLogger(do func(kafka.Logger)) {
if r.Reader.Config().ErrorLogger != nil {
do(r.Reader.Config().ErrorLogger)
} else if r.Reader.Config().Logger != nil {
do(r.Reader.Config().Logger)
}
}