-
Notifications
You must be signed in to change notification settings - Fork 453
/
shard.go
2863 lines (2542 loc) · 87.8 KB
/
shard.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
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
// Copyright (c) 2020 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 storage
import (
"container/list"
"errors"
"fmt"
"io"
"math"
"sync"
"time"
"github.com/m3db/m3/src/dbnode/generated/proto/pagetoken"
"github.com/m3db/m3/src/dbnode/namespace"
"github.com/m3db/m3/src/dbnode/persist"
"github.com/m3db/m3/src/dbnode/persist/fs"
"github.com/m3db/m3/src/dbnode/persist/schema"
"github.com/m3db/m3/src/dbnode/retention"
"github.com/m3db/m3/src/dbnode/runtime"
"github.com/m3db/m3/src/dbnode/storage/block"
"github.com/m3db/m3/src/dbnode/storage/bootstrap"
"github.com/m3db/m3/src/dbnode/storage/bootstrap/result"
"github.com/m3db/m3/src/dbnode/storage/index"
"github.com/m3db/m3/src/dbnode/storage/index/convert"
"github.com/m3db/m3/src/dbnode/storage/repair"
"github.com/m3db/m3/src/dbnode/storage/series"
"github.com/m3db/m3/src/dbnode/tracepoint"
"github.com/m3db/m3/src/dbnode/ts"
"github.com/m3db/m3/src/dbnode/ts/writes"
"github.com/m3db/m3/src/dbnode/x/xio"
"github.com/m3db/m3/src/m3ninx/doc"
"github.com/m3db/m3/src/x/checked"
"github.com/m3db/m3/src/x/clock"
"github.com/m3db/m3/src/x/context"
xerrors "github.com/m3db/m3/src/x/errors"
"github.com/m3db/m3/src/x/ident"
"github.com/m3db/m3/src/x/instrument"
xresource "github.com/m3db/m3/src/x/resource"
xtime "github.com/m3db/m3/src/x/time"
"github.com/gogo/protobuf/proto"
"github.com/opentracing/opentracing-go/log"
"github.com/uber-go/tally"
"go.uber.org/zap"
)
const (
shardIterateBatchPercent = 0.01
shardIterateBatchMinSize = 16
)
var (
errShardEntryNotFound = errors.New("shard entry not found")
errShardNotOpen = errors.New("shard is not open")
errShardAlreadyTicking = errors.New("shard is already ticking")
errShardClosingTickTerminated = errors.New("shard is closing, terminating tick")
errShardInvalidPageToken = errors.New("shard could not unmarshal page token")
errShardIsNotBootstrapped = errors.New("shard is not bootstrapped")
errShardAlreadyBootstrapped = errors.New("shard is already bootstrapped")
errFlushStateIsNotInitialized = errors.New("shard flush state is not initialized")
errTriedToLoadNilSeries = errors.New("tried to load nil series into shard")
// ErrDatabaseLoadLimitHit is the error returned when the database load limit
// is hit or exceeded.
ErrDatabaseLoadLimitHit = errors.New("error loading series, database load limit hit")
emptyDoc = doc.Metadata{}
)
type filesetsFn func(
filePathPrefix string,
namespace ident.ID,
shardID uint32,
) (fs.FileSetFilesSlice, error)
type filesetPathsBeforeFn func(
filePathPrefix string,
namespace ident.ID,
shardID uint32,
t xtime.UnixNano,
) ([]string, error)
type tickPolicy int
const (
tickPolicyRegular tickPolicy = iota
tickPolicyCloseShard
)
type dbShardState int
const (
dbShardStateOpen dbShardState = iota
dbShardStateClosing
)
type dbShard struct {
sync.RWMutex
block.DatabaseBlockRetriever
opts Options
seriesOpts series.Options
nowFn clock.NowFn
state dbShardState
namespace namespace.Metadata
seriesBlockRetriever series.QueryableBlockRetriever
seriesOnRetrieveBlock block.OnRetrieveBlock
namespaceReaderMgr databaseNamespaceReaderManager
increasingIndex increasingIndex
seriesPool series.DatabaseSeriesPool
reverseIndex NamespaceIndex
insertQueue *dbShardInsertQueue
lookup *shardMap
list *list.List
bootstrapState BootstrapState
newMergerFn fs.NewMergerFn
newFSMergeWithMemFn newFSMergeWithMemFn
filesetsFn filesetsFn
filesetPathsBeforeFn filesetPathsBeforeFn
deleteFilesFn deleteFilesFn
snapshotFilesFn snapshotFilesFn
newReaderFn fs.NewReaderFn
sleepFn func(time.Duration)
identifierPool ident.Pool
contextPool context.Pool
flushState shardFlushState
tickWg *sync.WaitGroup
runtimeOptsListenClosers []xresource.SimpleCloser
currRuntimeOptions dbShardRuntimeOptions
logger *zap.Logger
metrics dbShardMetrics
tileAggregator TileAggregator
ticking bool
shard uint32
coldWritesEnabled bool
indexEnabled bool
}
// NB(r): dbShardRuntimeOptions does not contain its own
// mutex as some of the variables are needed each write
// which already at least acquires read lock from the shard
// mutex, so to keep the lock acquisitions to a minimum
// these are protected under the same shard mutex.
type dbShardRuntimeOptions struct {
writeNewSeriesAsync bool
tickSleepSeriesBatchSize int
tickSleepPerSeries time.Duration
}
type dbShardMetrics struct {
create tally.Counter
close tally.Counter
closeStart tally.Counter
closeLatency tally.Timer
seriesTicked tally.Gauge
insertAsyncInsertErrors tally.Counter
insertAsyncWriteInternalErrors tally.Counter
insertAsyncWriteInvalidParamsErrors tally.Counter
insertAsyncIndexErrors tally.Counter
snapshotTotalLatency tally.Timer
snapshotCheckNeedsSnapshotLatency tally.Timer
snapshotPrepareLatency tally.Timer
snapshotMergeByBucketLatency tally.Timer
snapshotMergeAcrossBucketsLatency tally.Timer
snapshotChecksumLatency tally.Timer
snapshotPersistLatency tally.Timer
snapshotCloseLatency tally.Timer
}
func newDatabaseShardMetrics(shardID uint32, scope tally.Scope) dbShardMetrics {
const insertErrorName = "insert-async.errors"
snapshotScope := scope.SubScope("snapshot")
return dbShardMetrics{
create: scope.Counter("create"),
close: scope.Counter("close"),
closeStart: scope.Counter("close-start"),
closeLatency: scope.Timer("close-latency"),
seriesTicked: scope.Tagged(map[string]string{
"shard": fmt.Sprintf("%d", shardID),
}).Gauge("series-ticked"),
insertAsyncInsertErrors: scope.Tagged(map[string]string{
"error_type": "insert-series",
"suberror_type": "shard-entry-insert-error",
}).Counter(insertErrorName),
insertAsyncWriteInternalErrors: scope.Tagged(map[string]string{
"error_type": "write-value",
"suberror_type": "internal-error",
}).Counter(insertErrorName),
insertAsyncWriteInvalidParamsErrors: scope.Tagged(map[string]string{
"error_type": "write-value",
"suberror_type": "invalid-params-error",
}).Counter(insertErrorName),
insertAsyncIndexErrors: scope.Tagged(map[string]string{
"error_type": "reverse-index",
"suberror_type": "write-batch-error",
}).Counter(insertErrorName),
snapshotTotalLatency: snapshotScope.Timer("total-latency"),
snapshotCheckNeedsSnapshotLatency: snapshotScope.Timer("check-needs-snapshot-latency"),
snapshotPrepareLatency: snapshotScope.Timer("prepare-latency"),
snapshotMergeByBucketLatency: snapshotScope.Timer("merge-by-bucket-latency"),
snapshotMergeAcrossBucketsLatency: snapshotScope.Timer("merge-across-buckets-latency"),
snapshotChecksumLatency: snapshotScope.Timer("checksum-latency"),
snapshotPersistLatency: snapshotScope.Timer("persist-latency"),
snapshotCloseLatency: snapshotScope.Timer("close-latency"),
}
}
type dbShardEntryWorkFn func(entry *Entry) bool
type dbShardEntryBatchWorkFn func(entries []*Entry) bool
type shardListElement *list.Element
type shardFlushState struct {
sync.RWMutex
statesByTime map[xtime.UnixNano]fileOpState
initialized bool
}
func newShardFlushState() shardFlushState {
return shardFlushState{
statesByTime: make(map[xtime.UnixNano]fileOpState),
}
}
func newDatabaseShard(
namespaceMetadata namespace.Metadata,
shard uint32,
blockRetriever block.DatabaseBlockRetriever,
namespaceReaderMgr databaseNamespaceReaderManager,
increasingIndex increasingIndex,
reverseIndex NamespaceIndex,
needsBootstrap bool,
opts Options,
seriesOpts series.Options,
) databaseShard {
scope := opts.InstrumentOptions().MetricsScope().
SubScope("dbshard")
s := &dbShard{
opts: opts,
seriesOpts: seriesOpts,
nowFn: opts.ClockOptions().NowFn(),
state: dbShardStateOpen,
namespace: namespaceMetadata,
shard: shard,
namespaceReaderMgr: namespaceReaderMgr,
increasingIndex: increasingIndex,
seriesPool: opts.DatabaseSeriesPool(),
reverseIndex: reverseIndex,
lookup: newShardMap(shardMapOptions{}),
list: list.New(),
newMergerFn: fs.NewMerger,
newFSMergeWithMemFn: newFSMergeWithMem,
filesetsFn: fs.DataFiles,
filesetPathsBeforeFn: fs.DataFileSetsBefore,
deleteFilesFn: fs.DeleteFiles,
snapshotFilesFn: fs.SnapshotFiles,
sleepFn: time.Sleep,
newReaderFn: fs.NewReader,
identifierPool: opts.IdentifierPool(),
contextPool: opts.ContextPool(),
flushState: newShardFlushState(),
tickWg: &sync.WaitGroup{},
coldWritesEnabled: namespaceMetadata.Options().ColdWritesEnabled(),
indexEnabled: namespaceMetadata.Options().IndexOptions().Enabled(),
logger: opts.InstrumentOptions().Logger(),
metrics: newDatabaseShardMetrics(shard, scope),
tileAggregator: opts.TileAggregator(),
}
s.insertQueue = newDatabaseShardInsertQueue(s.insertSeriesBatch,
s.nowFn, scope, opts.InstrumentOptions().Logger())
registerRuntimeOptionsListener := func(listener runtime.OptionsListener) {
elem := opts.RuntimeOptionsManager().RegisterListener(listener)
s.runtimeOptsListenClosers = append(s.runtimeOptsListenClosers, elem)
}
registerRuntimeOptionsListener(s)
registerRuntimeOptionsListener(s.insertQueue)
// Start the insert queue after registering runtime options listeners
// that may immediately fire with values
s.insertQueue.Start()
if !needsBootstrap {
s.bootstrapState = Bootstrapped
}
if blockRetriever != nil {
s.setBlockRetriever(blockRetriever)
}
s.metrics.create.Inc(1)
return s
}
func (s *dbShard) setBlockRetriever(retriever block.DatabaseBlockRetriever) {
// If using the block retriever then set the block retriever field
// and set the series block retriever as the shard itself and
// the on retrieve block callback as the shard itself as well
s.DatabaseBlockRetriever = retriever
s.seriesBlockRetriever = s
s.seriesOnRetrieveBlock = s
}
func (s *dbShard) SetRuntimeOptions(value runtime.Options) {
s.Lock()
s.currRuntimeOptions = dbShardRuntimeOptions{
writeNewSeriesAsync: value.WriteNewSeriesAsync(),
tickSleepSeriesBatchSize: value.TickSeriesBatchSize(),
tickSleepPerSeries: value.TickPerSeriesSleepDuration(),
}
s.Unlock()
}
func (s *dbShard) ID() uint32 {
return s.shard
}
func (s *dbShard) NumSeries() int64 {
s.RLock()
n := s.list.Len()
s.RUnlock()
return int64(n)
}
// Stream implements series.QueryableBlockRetriever
func (s *dbShard) Stream(
ctx context.Context,
id ident.ID,
blockStart xtime.UnixNano,
onRetrieve block.OnRetrieveBlock,
nsCtx namespace.Context,
) (xio.BlockReader, error) {
return s.DatabaseBlockRetriever.Stream(ctx, s.shard, id,
blockStart, onRetrieve, nsCtx)
}
// StreamWideEntry implements series.QueryableBlockRetriever
func (s *dbShard) StreamWideEntry(
ctx context.Context,
id ident.ID,
blockStart xtime.UnixNano,
filter schema.WideEntryFilter,
nsCtx namespace.Context,
) (block.StreamedWideEntry, error) {
return s.DatabaseBlockRetriever.StreamWideEntry(ctx, s.shard, id,
blockStart, filter, nsCtx)
}
// IsBlockRetrievable implements series.QueryableBlockRetriever
func (s *dbShard) IsBlockRetrievable(blockStart xtime.UnixNano) (bool, error) {
return s.hasWarmFlushed(blockStart)
}
func (s *dbShard) hasWarmFlushed(blockStart xtime.UnixNano) (bool, error) {
flushState, err := s.FlushState(blockStart)
if err != nil {
return false, err
}
return s.warmStatusIsRetrievable(flushState.WarmStatus), nil
}
func (s *dbShard) warmStatusIsRetrievable(status warmStatus) bool {
if !statusIsRetrievable(status.DataFlushed) {
return false
}
// If the index is disabled, then we only are tracking data flushing.
// Otherwise, warm status requires both data and index flushed.
if !s.indexEnabled {
return true
}
return statusIsRetrievable(status.IndexFlushed)
}
func statusIsRetrievable(status fileOpStatus) bool {
switch status {
case fileOpNotStarted, fileOpInProgress, fileOpFailed:
return false
case fileOpSuccess:
return true
}
panic(fmt.Errorf("shard queried is retrievable with bad flush state %d",
status))
}
// RetrievableBlockColdVersion implements series.QueryableBlockRetriever
func (s *dbShard) RetrievableBlockColdVersion(blockStart xtime.UnixNano) (int, error) {
flushState, err := s.FlushState(blockStart)
if err != nil {
return -1, err
}
return flushState.ColdVersionFlushed, nil
}
// BlockStatesSnapshot implements series.QueryableBlockRetriever
func (s *dbShard) BlockStatesSnapshot() series.ShardBlockStateSnapshot {
s.RLock()
snapshots := s.blockStatesSnapshotWithRLock()
s.RUnlock()
return snapshots
}
func (s *dbShard) blockStatesSnapshotWithRLock() series.ShardBlockStateSnapshot {
bootstrapped := s.bootstrapState == Bootstrapped
if !bootstrapped {
// Needs to be bootstrapped.
return series.NewShardBlockStateSnapshot(false, series.BootstrappedBlockStateSnapshot{})
}
s.flushState.RLock()
defer s.flushState.RUnlock()
if !s.flushState.initialized {
// Also needs to have the shard flush states initialized.
return series.NewShardBlockStateSnapshot(false, series.BootstrappedBlockStateSnapshot{})
}
snapshot := make(map[xtime.UnixNano]series.BlockState, len(s.flushState.statesByTime))
for time, state := range s.flushState.statesByTime {
snapshot[time] = series.BlockState{
WarmRetrievable: s.warmStatusIsRetrievable(state.WarmStatus),
// Use ColdVersionRetrievable instead of ColdVersionFlushed since the snapshot
// will be used to make eviction decisions and we don't want to evict data before
// it is retrievable.
ColdVersion: state.ColdVersionRetrievable,
}
}
return series.NewShardBlockStateSnapshot(true, series.BootstrappedBlockStateSnapshot{
Snapshot: snapshot,
})
}
func (s *dbShard) OnRetrieveBlock(
id ident.ID,
tags ident.TagIterator,
startTime xtime.UnixNano,
segment ts.Segment,
nsCtx namespace.Context,
) {
s.RLock()
entry, err := s.lookupEntryWithLock(id)
if entry != nil {
entry.IncrementReaderWriterCount()
defer entry.DecrementReaderWriterCount()
}
s.RUnlock()
if err != nil && err != errShardEntryNotFound {
return // Likely closing
}
if entry != nil {
entry.Series.OnRetrieveBlock(id, tags, startTime, segment, nsCtx)
return
}
entry, err = s.newShardEntry(id, convert.NewTagsIterMetadataResolver(tags))
if err != nil {
// should never happen
instrument.EmitAndLogInvariantViolation(s.opts.InstrumentOptions(),
func(logger *zap.Logger) {
logger.Error("unable to create shardEntry from retrieved block data",
zap.Stringer("id", id),
zap.Time("startTime", startTime.ToTime()),
zap.Error(err))
})
return
}
// NB(r): Do not need to specify that needs to be indexed as series would
// have been already been indexed when it was written
copiedID := entry.Series.ID()
copiedTagsIter := s.identifierPool.TagsIterator()
copiedTagsIter.ResetFields(entry.Series.Metadata().Fields)
s.insertQueue.Insert(dbShardInsert{
entry: entry,
opts: dbShardInsertAsyncOptions{
// NB(r): Caching blocks should not be considered for
// new series insert rate limit.
skipRateLimit: true,
hasPendingRetrievedBlock: true,
pendingRetrievedBlock: dbShardPendingRetrievedBlock{
id: copiedID,
tags: copiedTagsIter,
start: startTime,
segment: segment,
nsCtx: nsCtx,
},
},
})
}
func (s *dbShard) OnEvictedFromWiredList(id ident.ID, blockStart xtime.UnixNano) {
s.RLock()
entry, err := s.lookupEntryWithLock(id)
s.RUnlock()
if err != nil && err != errShardEntryNotFound {
return // Shard is probably closing
}
if entry == nil {
// Its counter-intuitive that this can ever occur because the series should
// always exist if it has any active blocks, and if we've reached this point
// then the WiredList had a reference to a block that should still be in the
// series, and thus the series should exist. The reason this can occur is that
// even though the WiredList controls the lifecycle of blocks retrieved from
// disk, those blocks can still be removed from the series if they've completely
// fallen out of the retention period. In that case, the series tick will still
// remove the block, and then the shard tick can remove the series. At that point,
// it's possible for the WiredList to have a reference to an expired block for a
// series that is no longer in the shard.
return
}
entry.Series.OnEvictedFromWiredList(id, blockStart)
}
func (s *dbShard) forEachShardEntry(entryFn dbShardEntryWorkFn) {
s.forEachShardEntryBatch(func(currEntries []*Entry) bool {
for _, entry := range currEntries {
if continueForEach := entryFn(entry); !continueForEach {
return false
}
}
return true
})
}
func iterateBatchSize(elemsLen int) int {
if elemsLen < shardIterateBatchMinSize {
return shardIterateBatchMinSize
}
t := math.Ceil(shardIterateBatchPercent * float64(elemsLen))
return int(math.Max(shardIterateBatchMinSize, t))
}
func (s *dbShard) forEachShardEntryBatch(entriesBatchFn dbShardEntryBatchWorkFn) {
// NB(r): consider using a lockless list for ticking.
s.RLock()
elemsLen := s.list.Len()
s.RUnlock()
batchSize := iterateBatchSize(elemsLen)
decRefElem := func(e *list.Element) {
if e == nil {
return
}
e.Value.(*Entry).DecrementReaderWriterCount()
}
var (
currEntries = make([]*Entry, 0, batchSize)
first = true
nextElem *list.Element
)
for nextElem != nil || first {
s.RLock()
// NB(prateek): release held reference on the next element pointer now
// that we have the read lock and are guaranteed it cannot be changed
// from under us.
decRefElem(nextElem)
// lazily pull from the head of the list at first
if first {
nextElem = s.list.Front()
first = false
}
elem := nextElem
for ticked := 0; ticked < batchSize && elem != nil; ticked++ {
nextElem = elem.Next()
entry := elem.Value.(*Entry)
entry.IncrementReaderWriterCount()
currEntries = append(currEntries, entry)
elem = nextElem
}
// NB(prateek): inc a reference to the next element while we have a lock,
// to guarantee the element pointer cannot be changed from under us.
if nextElem != nil {
nextElem.Value.(*Entry).IncrementReaderWriterCount()
}
s.RUnlock()
continueExecution := entriesBatchFn(currEntries)
for i := range currEntries {
currEntries[i].DecrementReaderWriterCount()
currEntries[i] = nil
}
currEntries = currEntries[:0]
if !continueExecution {
decRefElem(nextElem)
return
}
}
}
func (s *dbShard) IsBootstrapped() bool {
return s.BootstrapState() == Bootstrapped
}
func (s *dbShard) Close() error {
s.Lock()
if s.state != dbShardStateOpen {
s.Unlock()
return errShardNotOpen
}
s.state = dbShardStateClosing
s.Unlock()
s.insertQueue.Stop()
for _, closer := range s.runtimeOptsListenClosers {
closer.Close()
}
s.metrics.closeStart.Inc(1)
stopwatch := s.metrics.closeLatency.Start()
defer func() {
s.metrics.close.Inc(1)
stopwatch.Stop()
}()
// NB(prateek): wait till any existing ticks are finished. In the usual
// case, no other ticks are running, and tickWg count is at 0, so the
// call to Wait() will return immediately.
// In the case when there is an existing Tick running, the count for
// tickWg will be > 0, and we'll wait until it's reset to zero, which
// will happen because earlier in this function we set the shard state
// to dbShardStateClosing, which triggers an early termination of
// any active ticks.
s.tickWg.Wait()
// NB(r): Asynchronously we purge expired series to ensure pressure on the
// GC is not placed all at one time. If the deadline is too low and still
// causes the GC to impact performance when closing shards the deadline
// should be increased.
cancellable := context.NewNoOpCanncellable()
_, err := s.tickAndExpire(cancellable, tickPolicyCloseShard, namespace.Context{})
return err
}
func (s *dbShard) isClosing() bool {
s.RLock()
closing := s.isClosingWithLock()
s.RUnlock()
return closing
}
func (s *dbShard) isClosingWithLock() bool {
return s.state == dbShardStateClosing
}
func (s *dbShard) Tick(c context.Cancellable, startTime xtime.UnixNano, nsCtx namespace.Context) (tickResult, error) {
s.removeAnyFlushStatesTooEarly(startTime)
return s.tickAndExpire(c, tickPolicyRegular, nsCtx)
}
func (s *dbShard) tickAndExpire(
c context.Cancellable,
policy tickPolicy,
nsCtx namespace.Context,
) (tickResult, error) {
s.Lock()
// ensure only one tick can execute at a time
if s.ticking {
s.Unlock()
// i.e. we were previously ticking
return tickResult{}, errShardAlreadyTicking
}
// NB(prateek): we bail out early if the shard is closing,
// unless it's the final tick issued during the Close(). This
// final tick is required to release resources back to our pools.
if policy != tickPolicyCloseShard && s.isClosingWithLock() {
s.Unlock()
return tickResult{}, errShardClosingTickTerminated
}
// enable Close() to track the lifecycle of the tick
s.ticking = true
s.tickWg.Add(1)
s.Unlock()
// reset ticking state
defer func() {
s.Lock()
s.ticking = false
s.tickWg.Done()
s.Unlock()
s.metrics.seriesTicked.Update(0.0) // reset external visibility
}()
var (
r tickResult
terminatedTickingDueToClosing bool
i int
slept time.Duration
expired []*Entry
)
s.RLock()
tickSleepBatch := s.currRuntimeOptions.tickSleepSeriesBatchSize
tickSleepPerSeries := s.currRuntimeOptions.tickSleepPerSeries
// Use blockStatesSnapshotWithRLock here to prevent nested read locks.
// Nested read locks will cause deadlocks if there is write lock attempt in
// between the nested read locks, since the write lock attempt will block
// future read lock attempts.
blockStates := s.blockStatesSnapshotWithRLock()
s.RUnlock()
s.forEachShardEntryBatch(func(currEntries []*Entry) bool {
// re-using `expired` to amortize allocs, still need to reset it
// to be safe for re-use.
for i := range expired {
expired[i] = nil
}
expired = expired[:0]
for _, entry := range currEntries {
if i > 0 && i%tickSleepBatch == 0 {
// NB(xichen): if the tick is cancelled, we bail out immediately.
// The cancellation check is performed on every batch of entries
// instead of every entry to reduce load.
if c.IsCancelled() {
return false
}
// NB(prateek): Also bail out early if the shard is closing,
// unless it's the final tick issued during the Close(). This
// final tick is required to release resources back to our pools.
if policy != tickPolicyCloseShard && s.isClosing() {
terminatedTickingDueToClosing = true
return false
}
// Expose shard level Tick() progress externally.
s.metrics.seriesTicked.Update(float64(i))
// Throttle the tick
sleepFor := time.Duration(tickSleepBatch) * tickSleepPerSeries
s.sleepFn(sleepFor)
slept += sleepFor
}
var (
result series.TickResult
err error
)
switch policy {
case tickPolicyRegular:
result, err = entry.Series.Tick(blockStates, nsCtx)
case tickPolicyCloseShard:
err = series.ErrSeriesAllDatapointsExpired
}
if err == series.ErrSeriesAllDatapointsExpired {
expired = append(expired, entry)
r.expiredSeries++
} else {
r.activeSeries++
if err != nil {
r.errors++
}
}
r.activeBlocks += result.ActiveBlocks
r.wiredBlocks += result.WiredBlocks
r.unwiredBlocks += result.UnwiredBlocks
r.pendingMergeBlocks += result.PendingMergeBlocks
r.madeExpiredBlocks += result.MadeExpiredBlocks
r.madeUnwiredBlocks += result.MadeUnwiredBlocks
r.mergedOutOfOrderBlocks += result.MergedOutOfOrderBlocks
r.evictedBuckets += result.EvictedBuckets
i++
}
// Purge any series requiring purging.
if len(expired) > 0 {
s.purgeExpiredSeries(expired)
for i := range expired {
expired[i] = nil
}
expired = expired[:0]
}
// Continue.
return true
})
if terminatedTickingDueToClosing {
return tickResult{}, errShardClosingTickTerminated
}
return r, nil
}
// NB(prateek): purgeExpiredSeries requires that all entries passed to it have at least one reader/writer,
// i.e. have a readWriteCount of at least 1.
// Currently, this function is only called by the lambda inside `tickAndExpire`'s `forEachShardEntryBatch`
// call. This satisfies the contract of all entries it operating upon being guaranteed to have a
// readerWriterEntryCount of at least 1, by virtue of the implementation of `forEachShardEntryBatch`.
func (s *dbShard) purgeExpiredSeries(expiredEntries []*Entry) {
// Remove all expired series from lookup and list.
s.Lock()
for _, entry := range expiredEntries {
// Only purge series after they've been GCed from the index, so that these happen and in order
// and there is no raciness around GCing something from the index when the series has already
// been removed from memory.
if s.indexEnabled && !entry.IndexGarbageCollected.Load() {
continue
}
series := entry.Series
id := series.ID()
elem, exists := s.lookup.Get(id)
if !exists {
continue
}
count := entry.ReaderWriterCount()
// The contract requires all entries to have count >= 1.
if count < 1 {
s.logger.Error("purgeExpiredSeries encountered invalid series read/write count",
zap.Stringer("namespace", s.namespace.ID()),
zap.Uint32("shard", s.ID()),
zap.Stringer("series", series.ID()),
zap.Int32("readerWriterCount", count))
continue
}
// If this series is currently being written to or read from, we don't
// remove to ensure a consistent view of the series to other users.
if count > 1 {
continue
}
// If there have been datapoints written to the series since its
// last empty check, we don't remove it.
if !series.IsEmpty() {
continue
}
// NB(xichen): if we get here, we are guaranteed that there can be
// no more reads/writes to this series while the lock is held, so it's
// safe to remove it.
series.Close()
s.list.Remove(elem)
s.lookup.Delete(id)
}
s.Unlock()
}
func (s *dbShard) WriteTagged(
ctx context.Context,
id ident.ID,
tagResolver convert.TagMetadataResolver,
timestamp xtime.UnixNano,
value float64,
unit xtime.Unit,
annotation []byte,
wOpts series.WriteOptions,
) (SeriesWrite, error) {
return s.writeAndIndex(ctx, id, tagResolver, timestamp,
value, unit, annotation, wOpts, true)
}
func (s *dbShard) Write(
ctx context.Context,
id ident.ID,
timestamp xtime.UnixNano,
value float64,
unit xtime.Unit,
annotation []byte,
wOpts series.WriteOptions,
) (SeriesWrite, error) {
return s.writeAndIndex(ctx, id, convert.EmptyTagMetadataResolver, timestamp,
value, unit, annotation, wOpts, false)
}
func (s *dbShard) writeAndIndex(
ctx context.Context,
id ident.ID,
tagResolver convert.TagMetadataResolver,
timestamp xtime.UnixNano,
value float64,
unit xtime.Unit,
annotation []byte,
wOpts series.WriteOptions,
shouldReverseIndex bool,
) (SeriesWrite, error) {
// Prepare write
entry, opts, err := s.TryRetrieveSeriesAndIncrementReaderWriterCount(id)
if err != nil {
return SeriesWrite{}, err
}
writable := entry != nil
// If no entry and we are not writing new series asynchronously.
if !writable && !opts.WriteNewSeriesAsync {
// Avoid double lookup by enqueueing insert immediately.
result, err := s.insertSeriesAsyncBatched(id, tagResolver, dbShardInsertAsyncOptions{
hasPendingIndexing: shouldReverseIndex,
pendingIndex: dbShardPendingIndex{
timestamp: timestamp,
enqueuedAt: s.nowFn(),
},
})
if err != nil {
return SeriesWrite{}, err
}
// Wait for the insert to be batched together and inserted
result.wg.Wait()
// Retrieve the inserted entry
entry, err = s.writableSeries(id, tagResolver)
if err != nil {
return SeriesWrite{}, err
}
writable = true
// NB(r): We just indexed this series if shouldReverseIndex was true
shouldReverseIndex = false
}
var (
commitLogSeriesID ident.ID
commitLogSeriesUniqueIndex uint64
needsIndex bool
pendingIndexInsert writes.PendingIndexInsert
// Err on the side of caution and always write to the commitlog if writing
// async, since there is no information about whether the write succeeded
// or not.
wasWritten = true
)
if writable {
// Perform write. No need to copy the annotation here because we're using it
// synchronously and all downstream code will copy anthing they need to maintain
// a reference to.
wasWritten, _, err = entry.Series.Write(ctx, timestamp, value, unit, annotation, wOpts)
// Load series metadata before decrementing the writer count
// to ensure this metadata is snapshotted at a consistent state
// NB(r): We explicitly do not place the series ID back into a
// pool as high frequency users of series IDs such
// as the commit log need to use the reference without the
// overhead of ownership tracking. This makes taking a ref here safe.
commitLogSeriesID = entry.Series.ID()
commitLogSeriesUniqueIndex = entry.Index
if err == nil && shouldReverseIndex {
if entry.NeedsIndexUpdate(s.reverseIndex.BlockStartForWriteTime(timestamp)) {
if !opts.WriteNewSeriesAsync {
return SeriesWrite{}, fmt.Errorf("to index async need write new series to be enabled")
}
needsIndex = true
pendingIndexInsert = s.pendingIndexInsert(entry, timestamp)
}
}
// release the reference we got on entry from `writableSeries`
entry.DecrementReaderWriterCount()
if err != nil {
return SeriesWrite{}, err
}
} else {
// This is an asynchronous insert and write which means we need to clone the annotation
// because its lifecycle in the commit log is independent of the calling function.
var annotationClone checked.Bytes
if len(annotation) != 0 {
annotationClone = s.opts.BytesPool().Get(len(annotation))
// IncRef here so we can write the bytes in, but don't DecRef because the queue is about
// to take ownership and will DecRef when its done.
annotationClone.IncRef()
annotationClone.AppendAll(annotation)
}
result, err := s.insertSeriesAsyncBatched(id, tagResolver, dbShardInsertAsyncOptions{
hasPendingWrite: true,
pendingWrite: dbShardPendingWrite{
timestamp: timestamp,
value: value,
unit: unit,
annotation: annotationClone,
opts: wOpts,
},
})
if err != nil {