partitions.go
5.9 KB
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
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
package cluster
import (
"sort"
"sync"
"time"
"github.com/Shopify/sarama"
)
// PartitionConsumer allows code to consume individual partitions from the cluster.
//
// See docs for Consumer.Partitions() for more on how to implement this.
type PartitionConsumer interface {
sarama.PartitionConsumer
// Topic returns the consumed topic name
Topic() string
// Partition returns the consumed partition
Partition() int32
// InitialOffset returns the offset used for creating the PartitionConsumer instance.
// The returned offset can be a literal offset, or OffsetNewest, or OffsetOldest
InitialOffset() int64
// MarkOffset marks the offset of a message as preocessed.
MarkOffset(offset int64, metadata string)
// ResetOffset resets the offset to a previously processed message.
ResetOffset(offset int64, metadata string)
}
type partitionConsumer struct {
sarama.PartitionConsumer
state partitionState
mu sync.Mutex
topic string
partition int32
initialOffset int64
closeOnce sync.Once
closeErr error
dying, dead chan none
}
func newPartitionConsumer(manager sarama.Consumer, topic string, partition int32, info offsetInfo, defaultOffset int64) (*partitionConsumer, error) {
offset := info.NextOffset(defaultOffset)
pcm, err := manager.ConsumePartition(topic, partition, offset)
// Resume from default offset, if requested offset is out-of-range
if err == sarama.ErrOffsetOutOfRange {
info.Offset = -1
offset = defaultOffset
pcm, err = manager.ConsumePartition(topic, partition, offset)
}
if err != nil {
return nil, err
}
return &partitionConsumer{
PartitionConsumer: pcm,
state: partitionState{Info: info},
topic: topic,
partition: partition,
initialOffset: offset,
dying: make(chan none),
dead: make(chan none),
}, nil
}
// Topic implements PartitionConsumer
func (c *partitionConsumer) Topic() string { return c.topic }
// Partition implements PartitionConsumer
func (c *partitionConsumer) Partition() int32 { return c.partition }
// InitialOffset implements PartitionConsumer
func (c *partitionConsumer) InitialOffset() int64 { return c.initialOffset }
// AsyncClose implements PartitionConsumer
func (c *partitionConsumer) AsyncClose() {
c.closeOnce.Do(func() {
c.closeErr = c.PartitionConsumer.Close()
close(c.dying)
})
}
// Close implements PartitionConsumer
func (c *partitionConsumer) Close() error {
c.AsyncClose()
<-c.dead
return c.closeErr
}
func (c *partitionConsumer) waitFor(stopper <-chan none, errors chan<- error) {
defer close(c.dead)
for {
select {
case err, ok := <-c.Errors():
if !ok {
return
}
select {
case errors <- err:
case <-stopper:
return
case <-c.dying:
return
}
case <-stopper:
return
case <-c.dying:
return
}
}
}
func (c *partitionConsumer) multiplex(stopper <-chan none, messages chan<- *sarama.ConsumerMessage, errors chan<- error) {
defer close(c.dead)
for {
select {
case msg, ok := <-c.Messages():
if !ok {
return
}
select {
case messages <- msg:
case <-stopper:
return
case <-c.dying:
return
}
case err, ok := <-c.Errors():
if !ok {
return
}
select {
case errors <- err:
case <-stopper:
return
case <-c.dying:
return
}
case <-stopper:
return
case <-c.dying:
return
}
}
}
func (c *partitionConsumer) getState() partitionState {
c.mu.Lock()
state := c.state
c.mu.Unlock()
return state
}
func (c *partitionConsumer) markCommitted(offset int64) {
c.mu.Lock()
if offset == c.state.Info.Offset {
c.state.Dirty = false
}
c.mu.Unlock()
}
// MarkOffset implements PartitionConsumer
func (c *partitionConsumer) MarkOffset(offset int64, metadata string) {
c.mu.Lock()
if next := offset + 1; next > c.state.Info.Offset {
c.state.Info.Offset = next
c.state.Info.Metadata = metadata
c.state.Dirty = true
}
c.mu.Unlock()
}
// ResetOffset implements PartitionConsumer
func (c *partitionConsumer) ResetOffset(offset int64, metadata string) {
c.mu.Lock()
if next := offset + 1; next <= c.state.Info.Offset {
c.state.Info.Offset = next
c.state.Info.Metadata = metadata
c.state.Dirty = true
}
c.mu.Unlock()
}
// --------------------------------------------------------------------
type partitionState struct {
Info offsetInfo
Dirty bool
LastCommit time.Time
}
// --------------------------------------------------------------------
type partitionMap struct {
data map[topicPartition]*partitionConsumer
mu sync.RWMutex
}
func newPartitionMap() *partitionMap {
return &partitionMap{
data: make(map[topicPartition]*partitionConsumer),
}
}
func (m *partitionMap) IsSubscribedTo(topic string) bool {
m.mu.RLock()
defer m.mu.RUnlock()
for tp := range m.data {
if tp.Topic == topic {
return true
}
}
return false
}
func (m *partitionMap) Fetch(topic string, partition int32) *partitionConsumer {
m.mu.RLock()
pc, _ := m.data[topicPartition{topic, partition}]
m.mu.RUnlock()
return pc
}
func (m *partitionMap) Store(topic string, partition int32, pc *partitionConsumer) {
m.mu.Lock()
m.data[topicPartition{topic, partition}] = pc
m.mu.Unlock()
}
func (m *partitionMap) Snapshot() map[topicPartition]partitionState {
m.mu.RLock()
defer m.mu.RUnlock()
snap := make(map[topicPartition]partitionState, len(m.data))
for tp, pc := range m.data {
snap[tp] = pc.getState()
}
return snap
}
func (m *partitionMap) Stop() {
m.mu.RLock()
defer m.mu.RUnlock()
var wg sync.WaitGroup
for tp := range m.data {
wg.Add(1)
go func(p *partitionConsumer) {
_ = p.Close()
wg.Done()
}(m.data[tp])
}
wg.Wait()
}
func (m *partitionMap) Clear() {
m.mu.Lock()
for tp := range m.data {
delete(m.data, tp)
}
m.mu.Unlock()
}
func (m *partitionMap) Info() map[string][]int32 {
info := make(map[string][]int32)
m.mu.RLock()
for tp := range m.data {
info[tp.Topic] = append(info[tp.Topic], tp.Partition)
}
m.mu.RUnlock()
for topic := range info {
sort.Sort(int32Slice(info[topic]))
}
return info
}