/
commitlog.go
297 lines (273 loc) · 7.3 KB
/
commitlog.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
package commitlog
import (
"fmt"
"io"
"os"
"path/filepath"
"sort"
"strconv"
"strings"
"sync"
"sync/atomic"
"github.com/pkg/errors"
)
var (
ErrCorruptedLog = errors.New("corrupted commitlog")
)
type commitLog struct {
datadir string
mtx sync.Mutex
activeSegment Segment
segments []Segment
segmentMaxRecordCount uint64
currentOffset uint64
maxSegmentCount int
}
type CommitLog interface {
io.Closer
// WriteEntry appends a binary payload to the commitlog.
WriteEntry(ts uint64, value []byte) (uint64, error)
// Delete closes then deletes the commitlog from disk.
Delete() error
// Reader returns a seekable reader
Reader() Cursor
// Offset returns the offset of the next record the be written.
Offset() uint64
Datadir() string
//LookupTimestamp returns the seekable offset of the first record writen after the provided timestamp.
LookupTimestamp(ts uint64) uint64
//Latest returns the timestamp of the most recent record
Latest() uint64
GetStatistics() Statistics
TruncateAfter(offset uint64) error
// TruncateBefore delete segments before the one containing the provided offset
TruncateBefore(offset uint64) error
}
func logFiles(datadir string) []uint64 {
matches, err := filepath.Glob(fmt.Sprintf("%s/*.log", datadir))
if err != nil {
return nil
}
out := make([]uint64, 0)
for idx := range matches {
offsetStr := strings.TrimSuffix(filepath.Base(matches[idx]), ".log")
offset, err := strconv.ParseUint(offsetStr, 10, 64)
if err == nil {
out = append(out, offset)
}
}
sort.Slice(out, func(i, j int) bool { return out[i] < out[j] })
return out
}
type createOpt func(*commitLog)
func WithMaxSegmentCount(i int) createOpt {
return func(c *commitLog) { c.maxSegmentCount = i }
}
func Open(datadir string, segmentMaxRecordCount uint64, opts ...createOpt) (CommitLog, error) {
files := logFiles(datadir)
if len(files) > 0 {
return open(datadir, segmentMaxRecordCount, opts...)
}
err := os.MkdirAll(datadir, 0750)
if err != nil {
return nil, err
}
return create(datadir, segmentMaxRecordCount, opts...)
}
func newLog(datadir string, segmentMaxRecordCount uint64, opts ...createOpt) *commitLog {
c := &commitLog{
datadir: datadir,
segmentMaxRecordCount: segmentMaxRecordCount,
}
for _, opt := range opts {
opt(c)
}
return c
}
func create(datadir string, segmentMaxRecordCount uint64, opts ...createOpt) (CommitLog, error) {
l := newLog(datadir, segmentMaxRecordCount, opts...)
return l, l.appendSegment()
}
func open(datadir string, segmentMaxRecordCount uint64, opts ...createOpt) (CommitLog, error) {
l := newLog(datadir, segmentMaxRecordCount, opts...)
files := logFiles(datadir)
var offset uint64 = files[0]
for {
segment, err := openSegment(datadir, offset, segmentMaxRecordCount, true)
if err != nil {
if err == ErrSegmentDoesNotExist {
break
}
return nil, ErrCorruptedLog
}
l.segments = append(l.segments, segment)
if l.activeSegment != nil {
if l.activeSegment.BaseOffset() < segment.BaseOffset() {
l.activeSegment = segment
}
} else {
l.activeSegment = segment
}
offset += uint64(segmentMaxRecordCount)
}
l.trimSegments()
atomic.StoreUint64(&l.currentOffset, l.activeSegment.CurrentOffset()+l.activeSegment.BaseOffset())
return l, nil
}
func (e *commitLog) Offset() uint64 {
return atomic.LoadUint64(&e.currentOffset)
}
func (e *commitLog) Latest() uint64 {
e.mtx.Lock()
defer e.mtx.Unlock()
return e.activeSegment.Latest()
}
func (e *commitLog) Close() error {
e.mtx.Lock()
defer e.mtx.Unlock()
for _, segment := range e.segments {
segment.Close()
}
return nil
}
func (e *commitLog) Datadir() string {
return e.datadir
}
func (e *commitLog) Delete() error {
e.mtx.Lock()
defer e.mtx.Unlock()
for _, segment := range e.segments {
err := segment.Delete()
if err != nil {
return err
}
}
return nil
}
func (e *commitLog) appendSegment() error {
var nextOffset uint64
if len(e.segments) > 0 {
nextOffset = e.activeSegment.CurrentOffset() + e.activeSegment.BaseOffset()
}
segment, err := createSegment(e.datadir, nextOffset, e.segmentMaxRecordCount)
if err != nil {
return errors.Wrap(err, fmt.Sprintf("failed to create new segment with offset %d", nextOffset))
}
e.segments = append(e.segments, segment)
e.activeSegment = segment
return nil
}
// trimSegments will delete segments acording to e.maxSegmentCount
func (e *commitLog) trimSegments() {
if e.maxSegmentCount <= 0 || len(e.segments) < e.maxSegmentCount {
return
}
count := len(e.segments)
for _, segment := range e.segments[0 : count-e.maxSegmentCount] {
segment.Delete()
}
e.segments = e.segments[count-e.maxSegmentCount:]
}
// LookupOffsetSegment returns the segment containing the provided offset
func (e *commitLog) lookupOffset(offset uint64) Segment {
e.mtx.Lock()
defer e.mtx.Unlock()
idx := e.lookupOffsetSegment(offset)
return e.segments[idx]
}
// LookupOffsetSegment returns the segment index of the segment containing the provided offset
func (e *commitLog) LookupOffsetSegment(offset uint64) int {
e.mtx.Lock()
defer e.mtx.Unlock()
return e.lookupOffsetSegment(offset)
}
func (e *commitLog) lookupOffsetSegment(offset uint64) int {
count := len(e.segments)
idx := sort.Search(count, func(i int) bool {
return e.segments[i].BaseOffset() > offset
})
if idx == 0 {
return 0
}
return idx - 1
}
func (e *commitLog) LookupTimestamp(ts uint64) uint64 {
e.mtx.Lock()
defer e.mtx.Unlock()
count := len(e.segments)
idx := sort.Search(count, func(i int) bool {
seg := e.segments[i]
return seg.Earliest() > ts
})
if idx <= 0 {
return e.segments[0].BaseOffset()
}
seg := e.segments[idx-1]
return seg.LookupTimestamp(ts)
}
func (e *commitLog) Reader() Cursor {
e.mtx.Lock()
defer e.mtx.Unlock()
return &cursor{
log: e,
currentSegment: e.segments[0],
pos: 0,
}
}
func (e *commitLog) TruncateBefore(offset uint64) error {
e.mtx.Lock()
defer e.mtx.Unlock()
idx := e.lookupOffsetSegment(offset)
if idx == 0 {
return nil
}
for _, segment := range e.segments[:idx] {
err := segment.Delete()
if err != nil {
return errors.Wrap(err, "failed to truncate log")
}
}
e.segments = e.segments[idx:]
return nil
}
// Truncate the log *after* the given offset. You must ensure no one is reading the log before truncating it.
func (e *commitLog) TruncateAfter(offset uint64) error {
e.mtx.Lock()
defer e.mtx.Unlock()
segmentIdx := e.lookupOffsetSegment(offset)
var segment Segment
var err error
if segmentIdx == len(e.segments)-1 {
segment = e.activeSegment
} else {
segment = e.segments[segmentIdx]
}
err = segment.TruncateAfter(offset)
if err != nil {
panic(err)
}
e.activeSegment = segment
for i := segmentIdx + 1; i < len(e.segments); i++ {
segment = e.segments[i]
err = segment.Delete()
if err != nil {
panic(err)
}
}
e.segments = e.segments[:segmentIdx+1]
return nil
}
func (e *commitLog) WriteEntry(ts uint64, value []byte) (uint64, error) {
e.mtx.Lock()
defer e.mtx.Unlock()
if segmentEntryCount := e.activeSegment.CurrentOffset(); segmentEntryCount >= e.segmentMaxRecordCount {
err := e.appendSegment()
if err != nil {
return 0, errors.Wrap(err, "failed to extend log")
}
e.trimSegments()
}
_, err := e.activeSegment.WriteEntry(ts, value)
offset := atomic.AddUint64(&e.currentOffset, 1)
return offset - 1, err
}