-
Notifications
You must be signed in to change notification settings - Fork 451
/
commit_log.go
446 lines (359 loc) · 9.73 KB
/
commit_log.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
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
// Copyright (c) 2016 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 commitlog
import (
"errors"
"sync"
"time"
"github.com/m3db/m3/src/dbnode/clock"
"github.com/m3db/m3/src/dbnode/ts"
"github.com/m3db/m3x/context"
xlog "github.com/m3db/m3x/log"
xtime "github.com/m3db/m3x/time"
"github.com/uber-go/tally"
)
var (
// ErrCommitLogQueueFull is raised when trying to write to the commit log
// when the queue is full
ErrCommitLogQueueFull = errors.New("commit log queue is full")
errCommitLogClosed = errors.New("commit log is closed")
timeZero = time.Time{}
)
type newCommitLogWriterFn func(
flushFn flushFn,
opts Options,
) commitLogWriter
type writeCommitLogFn func(
ctx context.Context,
series Series,
datapoint ts.Datapoint,
unit xtime.Unit,
annotation ts.Annotation,
) error
type commitLogFailFn func(err error)
type completionFn func(err error)
type commitLog struct {
sync.RWMutex
opts Options
nowFn clock.NowFn
log xlog.Logger
newCommitLogWriterFn newCommitLogWriterFn
writeFn writeCommitLogFn
commitLogFailFn commitLogFailFn
writer commitLogWriter
// TODO(r): replace buffered channel with concurrent striped
// circular buffer to avoid central write lock contention
writes chan commitLogWrite
flushMutex sync.RWMutex
lastFlushAt time.Time
pendingFlushFns []completionFn
writerExpireAt time.Time
closed bool
closeErr chan error
metrics commitLogMetrics
}
type commitLogMetrics struct {
queued tally.Gauge
success tally.Counter
errors tally.Counter
openErrors tally.Counter
closeErrors tally.Counter
flushErrors tally.Counter
flushDone tally.Counter
}
type valueType int
// nolint: varcheck, unused
const (
writeValueType valueType = iota
flushValueType
)
type commitLogWrite struct {
valueType valueType
series Series
datapoint ts.Datapoint
unit xtime.Unit
annotation ts.Annotation
completionFn completionFn
}
// NewCommitLog creates a new commit log
func NewCommitLog(opts Options) (CommitLog, error) {
if err := opts.Validate(); err != nil {
return nil, err
}
iopts := opts.InstrumentOptions().SetMetricsScope(
opts.InstrumentOptions().MetricsScope().SubScope("commitlog"))
scope := iopts.MetricsScope()
commitLog := &commitLog{
opts: opts,
nowFn: opts.ClockOptions().NowFn(),
log: iopts.Logger(),
newCommitLogWriterFn: newCommitLogWriter,
writes: make(chan commitLogWrite, opts.BacklogQueueSize()),
closeErr: make(chan error),
metrics: commitLogMetrics{
queued: scope.Gauge("writes.queued"),
success: scope.Counter("writes.success"),
errors: scope.Counter("writes.errors"),
openErrors: scope.Counter("writes.open-errors"),
closeErrors: scope.Counter("writes.close-errors"),
flushErrors: scope.Counter("writes.flush-errors"),
flushDone: scope.Counter("writes.flush-done"),
},
}
switch opts.Strategy() {
case StrategyWriteWait:
commitLog.writeFn = commitLog.writeWait
default:
commitLog.writeFn = commitLog.writeBehind
}
return commitLog, nil
}
func (l *commitLog) Open() error {
// Open the buffered commit log writer
if err := l.openWriter(l.nowFn()); err != nil {
return err
}
// Flush the info header to ensure we can write to disk
if err := l.writer.Flush(); err != nil {
return err
}
// NB(r): In the future we can introduce a commit log failure policy
// similar to Cassandra's "stop", for example see:
// https://github.com/apache/cassandra/blob/6dfc1e7eeba539774784dfd650d3e1de6785c938/conf/cassandra.yaml#L232
// Right now it is a large amount of coordination to implement something similar.
l.commitLogFailFn = func(err error) {
l.log.Fatalf("fatal commit log error: %v", err)
}
// Asynchronously write
go l.write()
if flushInterval := l.opts.FlushInterval(); flushInterval > 0 {
// Continually flush the commit log at given interval if set
go l.flushEvery(flushInterval)
}
return nil
}
func (l *commitLog) flushEvery(interval time.Duration) {
// Periodically flush the underlying commit log writer to cover
// the case when writes stall for a considerable time
var sleepForOverride time.Duration
for {
l.metrics.queued.Update(float64(len(l.writes)))
sleepFor := interval
if sleepForOverride > 0 {
sleepFor = sleepForOverride
sleepForOverride = 0
}
time.Sleep(sleepFor)
l.flushMutex.RLock()
lastFlushAt := l.lastFlushAt
l.flushMutex.RUnlock()
if sinceFlush := l.nowFn().Sub(lastFlushAt); sinceFlush < interval {
// Flushed already recently, sleep until we would next consider flushing
sleepForOverride = interval - sinceFlush
continue
}
// Request a flush
l.RLock()
if l.closed {
l.RUnlock()
return
}
l.writes <- commitLogWrite{valueType: flushValueType}
l.RUnlock()
}
}
func (l *commitLog) write() {
for write := range l.writes {
// For writes requiring acks add to pending acks
if write.completionFn != nil {
l.pendingFlushFns = append(l.pendingFlushFns, write.completionFn)
}
if write.valueType == flushValueType {
l.writer.Flush()
continue
}
if now := l.nowFn(); !now.Before(l.writerExpireAt) {
if err := l.openWriter(now); err != nil {
l.metrics.errors.Inc(1)
l.metrics.openErrors.Inc(1)
l.log.Errorf("failed to open commit log: %v", err)
if l.commitLogFailFn != nil {
l.commitLogFailFn(err)
}
continue
}
}
err := l.writer.Write(write.series,
write.datapoint, write.unit, write.annotation)
if err != nil {
l.metrics.errors.Inc(1)
l.log.Errorf("failed to write to commit log: %v", err)
if l.commitLogFailFn != nil {
l.commitLogFailFn(err)
}
continue
}
l.metrics.success.Inc(1)
}
l.Lock()
defer l.Unlock()
writer := l.writer
l.writer = nil
l.closeErr <- writer.Close()
}
func (l *commitLog) onFlush(err error) {
l.flushMutex.Lock()
l.lastFlushAt = l.nowFn()
l.flushMutex.Unlock()
if err != nil {
l.metrics.errors.Inc(1)
l.metrics.flushErrors.Inc(1)
l.log.Errorf("failed to flush commit log: %v", err)
if l.commitLogFailFn != nil {
l.commitLogFailFn(err)
}
}
// onFlush only ever called by "write()" and "openWriter" or
// before "write()" begins on "Open()" and there are no other
// accessors of "pendingFlushFns" so it is safe to read and mutate
// without a lock here
if len(l.pendingFlushFns) == 0 {
l.metrics.flushDone.Inc(1)
return
}
for i := range l.pendingFlushFns {
l.pendingFlushFns[i](err)
l.pendingFlushFns[i] = nil
}
l.pendingFlushFns = l.pendingFlushFns[:0]
l.metrics.flushDone.Inc(1)
}
func (l *commitLog) openWriter(now time.Time) error {
if l.writer != nil {
if err := l.writer.Close(); err != nil {
l.metrics.closeErrors.Inc(1)
l.log.Errorf("failed to close commit log: %v", err)
// If we failed to close then create a new commit log writer
l.writer = nil
}
}
if l.writer == nil {
l.writer = l.newCommitLogWriterFn(l.onFlush, l.opts)
}
blockSize := l.opts.BlockSize()
start := now.Truncate(blockSize)
if err := l.writer.Open(start, blockSize); err != nil {
return err
}
l.writerExpireAt = start.Add(blockSize)
return nil
}
func (l *commitLog) Write(
ctx context.Context,
series Series,
datapoint ts.Datapoint,
unit xtime.Unit,
annotation ts.Annotation,
) error {
return l.writeFn(ctx, series, datapoint, unit, annotation)
}
func (l *commitLog) writeWait(
ctx context.Context,
series Series,
datapoint ts.Datapoint,
unit xtime.Unit,
annotation ts.Annotation,
) error {
l.RLock()
if l.closed {
l.RUnlock()
return errCommitLogClosed
}
var (
wg sync.WaitGroup
result error
)
wg.Add(1)
completion := func(err error) {
result = err
wg.Done()
}
write := commitLogWrite{
series: series,
datapoint: datapoint,
unit: unit,
annotation: annotation,
completionFn: completion,
}
enqueued := false
select {
case l.writes <- write:
enqueued = true
default:
}
l.RUnlock()
if !enqueued {
return ErrCommitLogQueueFull
}
wg.Wait()
return result
}
func (l *commitLog) writeBehind(
ctx context.Context,
series Series,
datapoint ts.Datapoint,
unit xtime.Unit,
annotation ts.Annotation,
) error {
l.RLock()
if l.closed {
l.RUnlock()
return errCommitLogClosed
}
write := commitLogWrite{
series: series,
datapoint: datapoint,
unit: unit,
annotation: annotation,
}
enqueued := false
select {
case l.writes <- write:
enqueued = true
default:
}
l.RUnlock()
if !enqueued {
return ErrCommitLogQueueFull
}
return nil
}
func (l *commitLog) Close() error {
l.Lock()
if l.closed {
l.Unlock()
return nil
}
l.closed = true
close(l.writes)
l.Unlock()
// Receive the result of closing the writer from asynchronous writer
return <-l.closeErr
}