-
Notifications
You must be signed in to change notification settings - Fork 277
/
queue.go
500 lines (414 loc) · 14.8 KB
/
queue.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
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
// Licensed to LinDB under one or more contributor
// license agreements. See the NOTICE file distributed with
// this work for additional information regarding copyright
// ownership. LinDB licenses this file to you under
// the Apache License, Version 2.0 (the "License"); you may
// not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
package queue
import (
"errors"
"fmt"
"path/filepath"
"sync"
"go.uber.org/atomic"
"github.com/lindb/lindb/pkg/fileutil"
"github.com/lindb/lindb/pkg/logger"
"github.com/lindb/lindb/pkg/queue/page"
)
//go:generate mockgen -source ./queue.go -destination ./queue_mock.go -package queue
// for testing
var (
mkDirFunc = fileutil.MkDirIfNotExist
newPageFactoryFunc = page.NewFactory
existFunc = fileutil.Exist
)
var (
// ErrExceedingMessageSizeLimit returns when appending message exceeds the max size limit.
ErrExceedingMessageSizeLimit = errors.New("message exceeds the max page size limit")
// ErrOutOfSequenceRange returns sequence out of range.
ErrOutOfSequenceRange = errors.New("out of sequence range")
// ErrExceedingTotalSizeLimit returns total size limit.
ErrExceedingTotalSizeLimit = errors.New("queue data size exceeds the max size limit")
// ErrMsgNotFound returns message not found.
ErrMsgNotFound = errors.New("message not found")
)
var queueLogger = logger.GetLogger("Queue", "FanOutQueue")
// Queue represents a sequence of segments, new data is appended at append sequence.
// Segments with all message will be removed by gc which sequence < acknowledged sequence.
type Queue interface {
// Put puts data to the end of the queue, if puts failure return err.
Put(message []byte) error
// Get gets the message data at specific index.
Get(sequence int64) (message []byte, err error)
// AppendedSeq returns the written sequence which stands for the latest write barrier.
// New message is appended at append sequence.
AppendedSeq() int64
// SetAppendedSeq sets appended sequence.
SetAppendedSeq(seq int64)
// AcknowledgedSeq returns the acknowledged sequence which stands for the oldest read barrier.
// Message with req less than acknowledged sequence would be deleted at some point.
AcknowledgedSeq() int64
// SetAcknowledgedSeq sets acknowledged sequence.
SetAcknowledgedSeq(seq int64)
// NotEmpty checks queue if empty, waiting until new data written.
NotEmpty(consumeHead int64, checkClosed func() bool) bool
// Signal signals waiting consumers.
Signal()
// GC removes all message which sequence <= acknowledged sequence.
GC()
// Close closes the queue.
Close()
}
// queue implements queue.
type queue struct {
dirPath string // dirPath for queue file
dataSizeLimit int64 // the max size limit in bytes for data file
indexPageFct page.Factory // index page factory
dataPageFct page.Factory // data page factory
metaPageFct page.Factory // meta page factory
// queue meta with headSeq and tailSeq
metaPage page.MappedPage // meta buffer
appendedSeq atomic.Int64 // current written sequence
acknowledgedSeq atomic.Int64 // acknowledged sequence
indexPage page.MappedPage // index buffer
indexPageIndex int64
// message data write context
dataPageIndex int64
dataPage page.MappedPage
messageOffset int
closed atomic.Bool
rwMutex *sync.RWMutex
notEmpty *sync.Cond // not empty condition
}
// NewQueue returns Queue based on dirPath, dataSizeLimit is used to limit the total data/index size,
func NewQueue(dirPath string, dataSizeLimit int64) (Queue, error) {
if err := mkDirFunc(dirPath); err != nil {
return nil, err
}
lock := &sync.RWMutex{}
q := &queue{
dirPath: dirPath,
dataSizeLimit: dataSizeLimit,
rwMutex: lock,
notEmpty: sync.NewCond(lock),
}
// if data size limit < default limit, need reset
if q.dataSizeLimit < defaultDataSizeLimit {
q.dataSizeLimit = defaultDataSizeLimit
}
var err error
defer func() {
// if init queue failure, need release resource(like file/map file etc.)
if err != nil {
q.Close()
}
}()
// init data page factory
var dataPageFct page.Factory
dataPageFct, err = newPageFactoryFunc(filepath.Join(dirPath, dataPath), dataPageSize)
if err != nil {
return nil, err
}
q.dataPageFct = dataPageFct
// init index page factory
var indexPageFct page.Factory
indexPageFct, err = newPageFactoryFunc(filepath.Join(dirPath, indexPath), indexPageSize)
if err != nil {
return nil, err
}
q.indexPageFct = indexPageFct
hasMeta := fileutil.Exist(filepath.Join(dirPath, metaPath, fmt.Sprintf("%d.bat", metaPageIndex)))
// init meta page factory
var metaPageFct page.Factory
metaPageFct, err = newPageFactoryFunc(filepath.Join(dirPath, metaPath), metaPageSize)
if err != nil {
return nil, err
}
q.metaPageFct = metaPageFct
q.metaPage, err = q.metaPageFct.AcquirePage(metaPageIndex)
if err != nil {
return nil, err
}
if hasMeta {
// initialize sequence
q.initSequence()
} else {
q.appendedSeq.Store(SeqNoNewMessageAvailable)
q.acknowledgedSeq.Store(SeqNoNewMessageAvailable)
// persist metadata
q.metaPage.PutUint64(uint64(q.AppendedSeq()), queueAppendedSeqOffset)
q.metaPage.PutUint64(uint64(q.AcknowledgedSeq()), queueAcknowledgedSeqOffset)
err = q.metaPage.Sync()
if err != nil {
return nil, err
}
}
// initialize data page indexes
err = q.initDataPageIndex()
if err != nil {
return nil, err
}
return q, nil
}
// Put puts data to the end of the queue, if puts failure return err
func (q *queue) Put(data []byte) error {
dataLength := len(data)
if dataLength > dataPageSize {
// if message size > data page size, return err
return ErrExceedingMessageSizeLimit
}
dataPageIndex, dataPage, offset, err := q.alloc(dataLength)
if err != nil {
return err
}
// write message data
dataPage.WriteBytes(data, offset)
// persist metadata of message after write data
return q.persistMetaOfMessage(dataPageIndex, dataLength, offset)
}
// Get gets the message data at specific index
func (q *queue) Get(sequence int64) (data []byte, err error) {
if err = q.validateSequence(sequence); err != nil {
return
}
indexPageID := sequence / indexItemsPerPage
indexPage, ok := q.indexPageFct.GetPage(indexPageID)
if !ok {
return nil, ErrMsgNotFound
}
// calculate index offset of previous sequence
indexOffset := int((sequence % indexItemsPerPage) * indexItemLength)
dataPageID := int64(indexPage.ReadUint64(indexOffset + queueDataPageIndexOffset))
dataPage, ok := q.dataPageFct.GetPage(dataPageID)
if !ok {
return nil, ErrMsgNotFound
}
messageOffset := int(indexPage.ReadUint32(indexOffset + messageOffsetOffset))
messageLength := int(indexPage.ReadUint32(indexOffset + messageLengthOffset))
return dataPage.ReadBytes(messageOffset, messageLength), nil
}
// AppendedSeq returns the written sequence which stands for the latest write barrier.
// New message is appended at append sequence.
func (q *queue) AppendedSeq() int64 {
q.rwMutex.RLock()
defer q.rwMutex.RUnlock()
return q.appendedSeq.Load()
}
// SetAppendedSeq sets appended sequence.
func (q *queue) SetAppendedSeq(seq int64) {
q.rwMutex.Lock()
defer q.rwMutex.Unlock()
q.appendedSeq.Store(seq)
q.acknowledgedSeq.Store(seq)
q.metaPage.PutUint64(uint64(q.appendedSeq.Load()), queueAppendedSeqOffset)
q.metaPage.PutUint64(uint64(q.acknowledgedSeq.Load()), queueAcknowledgedSeqOffset)
if err := q.metaPage.Sync(); err != nil {
queueLogger.Error("sync queue meta page error, when set append seq",
logger.String("path", q.dirPath), logger.Error(err))
}
}
// AcknowledgedSeq returns the acknowledged sequence which stands for the oldest read barrier.
// Message with req less than acknowledged sequence would be deleted at some point.
func (q *queue) AcknowledgedSeq() int64 {
q.rwMutex.RLock()
defer q.rwMutex.RUnlock()
return q.acknowledgedSeq.Load()
}
// SetAcknowledgedSeq sets acknowledged sequence.
func (q *queue) SetAcknowledgedSeq(seq int64) {
q.rwMutex.Lock()
defer q.rwMutex.Unlock()
if seq > q.acknowledgedSeq.Load() && seq <= q.appendedSeq.Load() {
q.acknowledgedSeq.Store(seq)
q.metaPage.PutUint64(uint64(seq), queueAcknowledgedSeqOffset)
if err := q.metaPage.Sync(); err != nil {
queueLogger.Error("sync queue meta page error, when ack seq",
logger.String("path", q.dirPath), logger.Error(err))
}
}
}
// NotEmpty checks queue if empty, waiting until new data written.
func (q *queue) NotEmpty(consumeHead int64, checkClosed func() bool) bool {
q.notEmpty.L.Lock()
for consumeHead > q.appendedSeq.Load() && !q.closed.Load() && !checkClosed() {
q.notEmpty.Wait()
}
q.notEmpty.L.Unlock()
return !q.closed.Load() && !checkClosed()
}
// Signal signals waiting consumers.
func (q *queue) Signal() {
q.notEmpty.Broadcast()
}
// Close closes the queue.
func (q *queue) Close() {
if q.closed.CAS(false, true) {
q.rwMutex.RLock()
defer q.rwMutex.RUnlock()
q.notEmpty.Broadcast()
if q.dataPageFct != nil {
if err := q.dataPageFct.Close(); err != nil {
queueLogger.Error("close data page factory error",
logger.String("queue", q.dirPath), logger.Error(err))
}
}
if q.indexPageFct != nil {
if err := q.indexPageFct.Close(); err != nil {
queueLogger.Error("close index page factory error",
logger.String("queue", q.dirPath), logger.Error(err))
}
}
if q.metaPageFct != nil {
if err := q.metaPageFct.Close(); err != nil {
queueLogger.Error("close meta page factory error",
logger.String("queue", q.dirPath), logger.Error(err))
}
}
}
}
// GC removes all message which sequence < acknowledged sequence.
func (q *queue) GC() {
// get current acknowledged sequence.
ackSeq := q.AcknowledgedSeq()
if ackSeq < 0 {
return
}
indexPageID := ackSeq / indexItemsPerPage
indexPage, ok := q.indexPageFct.GetPage(indexPageID)
if !ok {
return
}
// calculate index offset of ack sequence
indexOffset := int((ackSeq % indexItemsPerPage) * indexItemLength)
dataPageID := int64(indexPage.ReadUint64(indexOffset + queueDataPageIndexOffset))
q.dataPageFct.TruncatePages(dataPageID)
q.indexPageFct.TruncatePages(indexPageID)
}
// alloc allocates the data page and offset for message writing
func (q *queue) alloc(dataLen int) (dataPageIndex int64, dataPage page.MappedPage, offset int, err error) {
q.rwMutex.Lock()
defer q.rwMutex.Unlock()
// prepare the data pointer
if q.messageOffset+dataLen > dataPageSize {
// check size limit before data page acquire
if err := q.checkDataSize(); err != nil {
return 0, nil, 0, err
}
// sync previous data page
if err := q.dataPage.Sync(); err != nil {
queueLogger.Error("sync data page err when alloc",
logger.String("queue", q.dirPath), logger.Error(err))
}
nextDataPageIndex := q.dataPageIndex + 1
// not enough space in current data page, need create new page
dataPage, err := q.dataPageFct.AcquirePage(nextDataPageIndex)
if err != nil {
return 0, nil, 0, err
}
q.dataPage = dataPage
q.dataPageIndex = nextDataPageIndex
q.messageOffset = 0 // need reset message offset for new data page
}
// advance dataOffset
messageOffset := q.messageOffset
q.messageOffset += dataLen // set next message offset
return q.dataPageIndex, q.dataPage, messageOffset, nil
}
// persistMetaOfMessage persists metadata of message after write data
func (q *queue) persistMetaOfMessage(dataPageIndex int64, dataLen, messageOffset int) error {
q.rwMutex.Lock()
defer q.rwMutex.Unlock()
seq := q.appendedSeq.Load() + 1 // append sequence
indexPageIndex := seq / indexItemsPerPage
if indexPageIndex != q.indexPageIndex {
// check size limit before index page acquire
if err := q.checkDataSize(); err != nil {
return err
}
// sync previous data page
if err := q.indexPage.Sync(); err != nil {
queueLogger.Error("sync index page err when alloc",
logger.String("queue", q.dirPath), logger.Error(err))
}
indexPage, err := q.indexPageFct.AcquirePage(indexPageIndex)
if err != nil {
return err
}
q.indexPage = indexPage
q.indexPageIndex = indexPageIndex
}
// save index data
indexOffset := int((seq % indexItemsPerPage) * indexItemLength)
q.indexPage.PutUint64(uint64(dataPageIndex), indexOffset+queueDataPageIndexOffset)
q.indexPage.PutUint32(uint32(messageOffset), indexOffset+messageOffsetOffset)
q.indexPage.PutUint32(uint32(dataLen), indexOffset+messageLengthOffset)
// save metadata
q.metaPage.PutUint64(uint64(seq), queueAppendedSeqOffset)
q.appendedSeq.Store(seq)
// new data written, notify all waiting consumer groups can consume data
q.notEmpty.Broadcast()
return nil
}
// initSequence initializes sequences from the metadata.
func (q *queue) initSequence() {
q.appendedSeq.Store(int64(q.metaPage.ReadUint64(queueAppendedSeqOffset)))
q.acknowledgedSeq.Store(int64(q.metaPage.ReadUint64(queueAcknowledgedSeqOffset)))
}
// initDataPageIndex finds out data page head index and message offset
func (q *queue) initDataPageIndex() (err error) {
if q.appendedSeq.Load() == SeqNoNewMessageAvailable {
// if queue is empty, start with new empty queue
q.dataPageIndex = 0
q.messageOffset = 0
if q.dataPage, err = q.dataPageFct.AcquirePage(0); err != nil {
return err
}
if q.indexPage, err = q.indexPageFct.AcquirePage(0); err != nil {
return err
}
return nil
}
previousSeq := q.appendedSeq.Load() // get previous sequence
q.indexPageIndex = previousSeq / indexItemsPerPage
if q.indexPage, err = q.indexPageFct.AcquirePage(q.indexPageIndex); err != nil {
return err
}
// calculate index offset of previous sequence
indexOffset := int((previousSeq % indexItemsPerPage) * indexItemLength)
q.dataPageIndex = int64(q.indexPage.ReadUint64(indexOffset + queueDataPageIndexOffset))
previousMessageOffset := q.indexPage.ReadUint32(indexOffset + messageOffsetOffset)
previousMessageLength := q.indexPage.ReadUint32(indexOffset + messageLengthOffset)
// calculate next message offset
q.messageOffset = int(previousMessageOffset + previousMessageLength)
if q.dataPage, err = q.dataPageFct.AcquirePage(q.dataPageIndex); err != nil {
return err
}
return nil
}
// validateSequence validates the sequence if in range
func (q *queue) validateSequence(sequence int64) error {
q.rwMutex.RLock()
defer q.rwMutex.RUnlock()
if sequence > q.appendedSeq.Load() || sequence <= q.acknowledgedSeq.Load() {
return fmt.Errorf("%w: get %d, range [%d~%d]", ErrOutOfSequenceRange,
sequence, q.appendedSeq.Load(), q.acknowledgedSeq.Load())
}
return nil
}
// checkDataSize checks the data size if exceeds the size limit
func (q *queue) checkDataSize() error {
if q.dataPageFct.Size()+q.indexPageFct.Size() > q.dataSizeLimit {
return ErrExceedingTotalSizeLimit
}
return nil
}