-
Notifications
You must be signed in to change notification settings - Fork 450
/
consumer_service_writer.go
360 lines (323 loc) · 9.7 KB
/
consumer_service_writer.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
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
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
// Copyright (c) 2018 Uber Technologies, Inc.
//
// Permission is hereby granted, free of charge, to any person obtaining a copy
// of this software and associated documentation files (the "Software"), to deal
// in the Software without restriction, including without limitation the rights
// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
// copies of the Software, and to permit persons to whom the Software is
// furnished to do so, subject to the following conditions:
//
// The above copyright notice and this permission notice shall be included in
// all copies or substantial portions of the Software.
//
// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
// THE SOFTWARE.
package writer
import (
"errors"
"fmt"
"sync"
"time"
"github.com/m3db/m3/src/cluster/placement"
"github.com/m3db/m3/src/msg/producer"
"github.com/m3db/m3/src/msg/topic"
"github.com/m3db/m3/src/x/watch"
"github.com/uber-go/tally"
"go.uber.org/zap"
)
var (
acceptAllFilter = producer.FilterFunc(
func(m producer.Message) bool {
return true
},
)
errUnknownConsumptionType = errors.New("unknown consumption type")
)
type initType int
const (
// failOnError will fail the initialization when any error is encountered.
failOnError initType = iota
// allowInitValueError will not fail the initialization when the initial
// value could not be obtained within timeout.
// This could be used to initialize a new consumer service writer during
// runtime so it allows the consumer service writer to continue waiting
// for the placement update in the background.
allowInitValueError
)
type consumerServiceWriter interface {
// Write writes a message.
Write(rm *producer.RefCountedMessage)
// Init will initialize the consumer service writer.
Init(initType) error
// Close closes the writer and the background watch thread.
Close()
// SetMessageTTLNanos sets the message ttl nanoseconds.
SetMessageTTLNanos(value int64)
// RegisterFilter registers a filter for the consumer service.
RegisterFilter(fn producer.FilterFunc)
// UnregisterFilter unregisters the filter for the consumer service.
UnregisterFilter()
}
type consumerServiceWriterMetrics struct {
placementError tally.Counter
placementUpdate tally.Counter
filterAccepted tally.Counter
filterNotAccepted tally.Counter
queueSize tally.Gauge
}
func newConsumerServiceWriterMetrics(scope tally.Scope) consumerServiceWriterMetrics {
return consumerServiceWriterMetrics{
placementUpdate: scope.Counter("placement-update"),
placementError: scope.Counter("placement-error"),
filterAccepted: scope.Counter("filter-accepted"),
filterNotAccepted: scope.Counter("filter-not-accepted"),
queueSize: scope.Gauge("queue-size"),
}
}
type consumerServiceWriterImpl struct {
sync.Mutex
cs topic.ConsumerService
ps placement.Service
shardWriters []shardWriter
opts Options
logger *zap.Logger
value watch.Value
dataFilter producer.FilterFunc
router ackRouter
consumerWriters map[string]consumerWriter
closed bool
doneCh chan struct{}
wg sync.WaitGroup
m consumerServiceWriterMetrics
cm consumerWriterMetrics
processFn watch.ProcessFn
}
func newConsumerServiceWriter(
cs topic.ConsumerService,
numShards uint32,
opts Options,
) (consumerServiceWriter, error) {
ps, err := opts.ServiceDiscovery().
PlacementService(cs.ServiceID(), opts.PlacementOptions())
if err != nil {
return nil, err
}
ct := cs.ConsumptionType()
if ct == topic.Unknown {
return nil, errUnknownConsumptionType
}
router := newAckRouter(int(numShards))
w := &consumerServiceWriterImpl{
cs: cs,
ps: ps,
shardWriters: initShardWriters(router, ct, numShards, opts),
opts: opts,
logger: opts.InstrumentOptions().Logger(),
dataFilter: acceptAllFilter,
router: router,
consumerWriters: make(map[string]consumerWriter),
closed: false,
doneCh: make(chan struct{}),
m: newConsumerServiceWriterMetrics(opts.InstrumentOptions().MetricsScope()),
cm: newConsumerWriterMetrics(opts.InstrumentOptions().MetricsScope()),
}
w.processFn = w.process
return w, nil
}
func initShardWriters(
router ackRouter,
ct topic.ConsumptionType,
numberOfShards uint32,
opts Options,
) []shardWriter {
var (
sws = make([]shardWriter, numberOfShards)
m = newMessageWriterMetrics(
opts.InstrumentOptions().MetricsScope(),
opts.InstrumentOptions().TimerOptions(),
)
mPool messagePool
)
if opts.MessagePoolOptions() != nil {
mPool = newMessagePool(opts.MessagePoolOptions())
mPool.Init()
}
for i := range sws {
switch ct {
case topic.Shared:
sws[i] = newSharedShardWriter(uint32(i), router, mPool, opts, m)
case topic.Replicated:
sws[i] = newReplicatedShardWriter(uint32(i), numberOfShards, router, mPool, opts, m)
}
}
return sws
}
func (w *consumerServiceWriterImpl) Write(rm *producer.RefCountedMessage) {
if rm.Accept(w.dataFilter) {
w.shardWriters[rm.Shard()].Write(rm)
w.m.filterAccepted.Inc(1)
return
}
// It is not an error if the message does not pass the filter.
w.m.filterNotAccepted.Inc(1)
}
func (w *consumerServiceWriterImpl) Init(t initType) error {
w.wg.Add(1)
go func() {
w.reportMetrics()
w.wg.Done()
}()
updatableFn := func() (watch.Updatable, error) {
return w.ps.Watch()
}
getFn := func(updatable watch.Updatable) (interface{}, error) {
update, err := updatable.(placement.Watch).Get()
if err != nil {
w.m.placementError.Inc(1)
w.logger.Error("invalid placement update from kv", zap.Error(err))
return nil, err
}
w.m.placementUpdate.Inc(1)
return update, nil
}
vOptions := watch.NewOptions().
SetInitWatchTimeout(w.opts.PlacementWatchInitTimeout()).
SetInstrumentOptions(w.opts.InstrumentOptions()).
SetNewUpdatableFn(updatableFn).
SetGetUpdateFn(getFn).
SetProcessFn(w.processFn).
SetKey(w.opts.TopicName())
w.value = watch.NewValue(vOptions)
err := w.value.Watch()
if err == nil {
return nil
}
if t == allowInitValueError {
if _, ok := err.(watch.InitValueError); ok {
w.logger.Warn("invalid placement update, continue to watch for placement updates",
zap.Error(err))
return nil
}
}
return fmt.Errorf("consumer service writer init error: %v", err)
}
func (w *consumerServiceWriterImpl) process(update interface{}) error {
var (
p = update.(placement.Placement)
isSharded = p.IsSharded()
)
// Non sharded placement is only allowed for Shared consumption type.
if w.cs.ConsumptionType() == topic.Replicated && !isSharded {
return fmt.Errorf("non-sharded placement for replicated consumer %s", w.cs.String())
}
// NB(cw): Lock can be removed as w.consumerWriters is only accessed in this thread.
w.Lock()
newConsumerWriters, tobeDeleted := w.diffPlacementWithLock(p)
for i, sw := range w.shardWriters {
if isSharded {
sw.UpdateInstances(p.InstancesForShard(uint32(i)), newConsumerWriters)
continue
}
sw.UpdateInstances(p.Instances(), newConsumerWriters)
}
oldConsumerWriters := w.consumerWriters
w.consumerWriters = newConsumerWriters
w.Unlock()
go func() {
for _, addr := range tobeDeleted {
cw, ok := oldConsumerWriters[addr]
if ok {
cw.Close()
}
}
}()
return nil
}
func (w *consumerServiceWriterImpl) diffPlacementWithLock(newPlacement placement.Placement) (map[string]consumerWriter, []string) {
var (
newInstances = newPlacement.Instances()
newConsumerWriters = make(map[string]consumerWriter, len(newInstances))
toBeDeleted []string
)
for _, instance := range newInstances {
id := instance.Endpoint()
cw, ok := w.consumerWriters[id]
if ok {
newConsumerWriters[id] = cw
continue
}
cw = newConsumerWriter(instance.Endpoint(), w.router, w.opts, w.cm)
cw.Init()
newConsumerWriters[id] = cw
}
for id := range w.consumerWriters {
if _, ok := newConsumerWriters[id]; !ok {
toBeDeleted = append(toBeDeleted, id)
}
}
return newConsumerWriters, toBeDeleted
}
func (w *consumerServiceWriterImpl) Close() {
w.Lock()
if w.closed {
w.Unlock()
return
}
w.closed = true
w.Unlock()
w.logger.Info("closing consumer service writer", zap.String("writer", w.cs.String()))
close(w.doneCh)
// Blocks until all messages consuemd.
var shardWriterWG sync.WaitGroup
for _, sw := range w.shardWriters {
sw := sw
shardWriterWG.Add(1)
go func() {
sw.Close()
shardWriterWG.Done()
}()
}
shardWriterWG.Wait()
w.value.Unwatch()
for _, cw := range w.consumerWriters {
cw.Close()
}
w.wg.Wait()
w.logger.Info("closed consumer service writer", zap.String("writer", w.cs.String()))
}
func (w *consumerServiceWriterImpl) SetMessageTTLNanos(value int64) {
for _, sw := range w.shardWriters {
sw.SetMessageTTLNanos(value)
}
}
func (w *consumerServiceWriterImpl) RegisterFilter(filter producer.FilterFunc) {
w.Lock()
w.dataFilter = filter
w.Unlock()
}
func (w *consumerServiceWriterImpl) UnregisterFilter() {
w.Lock()
w.dataFilter = acceptAllFilter
w.Unlock()
}
func (w *consumerServiceWriterImpl) reportMetrics() {
t := time.NewTicker(w.opts.InstrumentOptions().ReportInterval())
defer t.Stop()
for {
select {
case <-w.doneCh:
return
case <-t.C:
var l int
for _, sw := range w.shardWriters {
l += sw.QueueSize()
}
w.m.queueSize.Update(float64(l))
}
}
}