/
source.go
1255 lines (1122 loc) · 41.2 KB
/
source.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) 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 (
"bytes"
"context"
"errors"
"fmt"
"io"
"sync"
"time"
"github.com/m3db/m3/src/cluster/shard"
"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/fs/commitlog"
"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/series"
"github.com/m3db/m3/src/dbnode/topology"
"github.com/m3db/m3/src/dbnode/ts"
"github.com/m3db/m3/src/x/checked"
xcontext "github.com/m3db/m3/src/x/context"
"github.com/m3db/m3/src/x/ident"
"github.com/m3db/m3/src/x/instrument"
xtime "github.com/m3db/m3/src/x/time"
"github.com/opentracing/opentracing-go"
"github.com/uber-go/tally"
"go.uber.org/zap"
"golang.org/x/sync/errgroup"
)
const (
workerChannelSize = 256
readSeriesBlocksWorkerChannelSize = 512
)
type newIteratorFn func(opts commitlog.IteratorOpts) (
iter commitlog.Iterator, corruptFiles []commitlog.ErrorWithPath, err error)
type snapshotFilesFn func(filePathPrefix string, namespace ident.ID, shard uint32) (fs.FileSetFilesSlice, error)
type bootstrapNamespace struct {
namespaceID []byte
bootstrapping bool
dataAndIndexShardRanges result.ShardTimeRanges
namespace namespace.Metadata
namespaceContext namespace.Context
dataBlockSize time.Duration
accumulator bootstrap.NamespaceDataAccumulator
}
type seriesMapKey struct {
fileReadID uint64
uniqueIndex uint64
}
type seriesMapEntry struct {
namespace *bootstrapNamespace
series bootstrap.CheckoutSeriesResult
shardNoLongerOwned bool
}
// accumulateArg contains all the information a worker go-routine needs to
// accumulate a write for encoding into the database.
type accumulateArg struct {
namespace *bootstrapNamespace
series bootstrap.CheckoutSeriesResult
shard uint32
dp ts.Datapoint
unit xtime.Unit
// longAnnotation stores the annotation value in case it does not fit in shortAnnotation.
longAnnotation ts.Annotation
// shortAnnotation is a predefined buffer for passing small allocations around instead of allocating.
shortAnnotation [ts.OptimizedAnnotationLen]byte
shortAnnotationLen uint8
}
type accumulateWorker struct {
inputCh chan accumulateArg
datapointsRead int
numErrors int
}
type seriesBlock struct {
resolver bootstrap.SeriesRefResolver
block block.DatabaseBlock
}
type readSeriesBlocksWorker struct {
dataCh chan seriesBlock
reader fs.DataFileSetReader
shard uint32
accumulator bootstrap.NamespaceDataAccumulator
blocksPool block.DatabaseBlockPool
blockStart xtime.UnixNano
blockSize time.Duration
nsCtx namespace.Context
}
func (w *readSeriesBlocksWorker) readSeriesBlocks(ctx context.Context) error {
defer close(w.dataCh)
numSeriesRead := 0
for {
id, tags, data, expectedChecksum, err := w.reader.Read()
if err != nil && !errors.Is(err, io.EOF) {
return err
}
if errors.Is(err, io.EOF) {
break
}
numSeriesRead++
dbBlock := w.blocksPool.Get()
dbBlock.Reset(w.blockStart, w.blockSize,
ts.NewSegment(data, nil, 0, ts.FinalizeHead), w.nsCtx)
// Resetting the block will trigger a checksum calculation, so use
// that instead of calculating it twice.
checksum, err := dbBlock.Checksum()
if err != nil {
return err
}
if checksum != expectedChecksum {
return fmt.Errorf("checksum for series: %s was %d but expected %d",
id, checksum, expectedChecksum)
}
res, owned, err := w.accumulator.CheckoutSeriesWithoutLock(w.shard, id, tags)
if err != nil {
if !owned {
// Skip bootstrapping this series if we don't own it.
continue
}
return err
}
w.dataCh <- seriesBlock{
resolver: res.Resolver,
block: dbBlock,
}
id.Finalize()
tags.Close()
// check if context was not canceled on a regular basis.
if numSeriesRead%1024 == 0 {
select {
case <-ctx.Done():
return nil
default:
// do not block.
}
}
}
return nil
}
type readNamespaceResult struct {
namespace bootstrap.Namespace
dataAndIndexShardRanges result.ShardTimeRanges
}
type commitLogResult struct {
shouldReturnUnfulfilled bool
// ensures we only read the commit log once
read bool
}
type commitLogSourceMetrics struct {
corruptCommitlogFile tally.Counter
bootstrapping tally.Gauge
commitLogEntriesRead tally.Counter
}
func newCommitLogSourceMetrics(scope tally.Scope) commitLogSourceMetrics {
return commitLogSourceMetrics{
corruptCommitlogFile: scope.SubScope("commitlog").Counter("corrupt"),
bootstrapping: scope.SubScope("status").Gauge("bootstrapping"),
commitLogEntriesRead: scope.SubScope("commitlog").Counter("entries-read"),
}
}
type gaugeLoopCloserFn func()
func (m commitLogSourceMetrics) emitBootstrapping() gaugeLoopCloserFn {
doneCh := make(chan struct{})
go func() {
for {
select {
case <-doneCh:
m.bootstrapping.Update(0)
return
default:
m.bootstrapping.Update(1)
time.Sleep(time.Second)
}
}
}()
return func() { close(doneCh) }
}
type commitLogSource struct {
opts Options
log *zap.Logger
nowFn func() time.Time
// Filesystem inspection capture before node was started.
inspection fs.Inspection
newIteratorFn newIteratorFn
snapshotFilesFn snapshotFilesFn
newReaderFn fs.NewReaderFn
metrics commitLogSourceMetrics
// Cache the results of reading the commit log between passes. The commit log is not sharded by time range, so the
// entire log needs to be read irrespective of the configured time ranges for the pass. The commit log only needs
// to be read once (during the first pass) and the results can be subsequently cached and returned on future passes.
// Since the bootstrapper is single threaded this does not need to be guarded with a mutex.
commitLogResult commitLogResult
instrumentation *instrumentation
}
func newCommitLogSource(
opts Options,
inspection fs.Inspection,
) bootstrap.Source {
scope := opts.
ResultOptions().
InstrumentOptions().
MetricsScope().
SubScope("bootstrapper-commitlog")
log := opts.
ResultOptions().
InstrumentOptions().
Logger().
With(zap.String("bootstrapper", "commitlog"))
return &commitLogSource{
opts: opts,
log: log,
nowFn: opts.ResultOptions().ClockOptions().NowFn(),
inspection: inspection,
newIteratorFn: commitlog.NewIterator,
snapshotFilesFn: fs.SnapshotFiles,
newReaderFn: fs.NewReader,
metrics: newCommitLogSourceMetrics(scope),
instrumentation: newInstrumentation(opts, scope, log),
}
}
func (s *commitLogSource) AvailableData(
ns namespace.Metadata,
shardsTimeRanges result.ShardTimeRanges,
_ bootstrap.Cache,
runOpts bootstrap.RunOptions,
) (result.ShardTimeRanges, error) {
return s.availability(ns, shardsTimeRanges, runOpts)
}
func (s *commitLogSource) AvailableIndex(
ns namespace.Metadata,
shardsTimeRanges result.ShardTimeRanges,
_ bootstrap.Cache,
runOpts bootstrap.RunOptions,
) (result.ShardTimeRanges, error) {
return s.availability(ns, shardsTimeRanges, runOpts)
}
// Read will read all commitlog files on disk, as well as as the latest snapshot for
// each shard/block combination (if it exists) and merge them.
// TODO(rartoul): Make this take the SnapshotMetadata files into account to reduce the
// number of commitlogs / snapshots that we need to read.
func (s *commitLogSource) Read(
ctx xcontext.Context,
namespaces bootstrap.Namespaces,
cache bootstrap.Cache,
) (bootstrap.NamespaceResults, error) {
instrCtx := s.instrumentation.commitLogBootstrapperSourceReadStarted(ctx)
defer instrCtx.finish()
var (
// Emit bootstrapping gauge for duration of ReadData.
doneReadingData = s.metrics.emitBootstrapping()
fsOpts = s.opts.CommitLogOptions().FilesystemOptions()
filePathPrefix = fsOpts.FilePathPrefix()
namespaceIter = namespaces.Namespaces.Iter()
)
defer doneReadingData()
instrCtx.bootstrapSnapshotsStarted()
for _, elem := range namespaceIter {
ns := elem.Value()
accumulator := ns.DataAccumulator
// NB(r): Combine all shard time ranges across data and index
// so we can do in one go.
shardTimeRanges := result.NewShardTimeRanges()
// NB(bodu): Use TargetShardTimeRanges which covers the entire original target shard range
// since the commitlog bootstrapper should run for the entire bootstrappable range per shard.
shardTimeRanges.AddRanges(ns.DataRunOptions.TargetShardTimeRanges)
if ns.Metadata.Options().IndexOptions().Enabled() {
shardTimeRanges.AddRanges(ns.IndexRunOptions.TargetShardTimeRanges)
}
// Determine which snapshot files are available.
snapshotFilesByShard, err := s.snapshotFilesByShard(
ns.Metadata.ID(), filePathPrefix, shardTimeRanges)
if err != nil {
return bootstrap.NamespaceResults{}, err
}
mostRecentCompleteSnapshotByBlockShard, err := s.mostRecentSnapshotByBlockShard(
ns.Metadata, shardTimeRanges, snapshotFilesByShard)
if err != nil {
return bootstrap.NamespaceResults{}, err
}
// Start by reading any available snapshot files.
blockSize := ns.Metadata.Options().RetentionOptions().BlockSize()
for shard, tr := range shardTimeRanges.Iter() {
err := s.bootstrapShardSnapshots(
ns.Metadata, accumulator, shard, tr, blockSize,
mostRecentCompleteSnapshotByBlockShard, cache)
if err != nil {
return bootstrap.NamespaceResults{}, err
}
}
}
instrCtx.bootstrapSnapshotsCompleted()
instrCtx.readCommitLogStarted()
if !s.commitLogResult.read {
var err error
s.commitLogResult, err = s.readCommitLog(namespaces, instrCtx.span)
if err != nil {
return bootstrap.NamespaceResults{}, err
}
} else {
s.log.Debug("commit log already read in a previous pass, using previous result.")
}
bootstrapResult := bootstrap.NamespaceResults{
Results: bootstrap.NewNamespaceResultsMap(bootstrap.NamespaceResultsMapOptions{}),
}
for _, elem := range namespaceIter {
ns := elem.Value()
id := ns.Metadata.ID()
dataResult := result.NewDataBootstrapResult()
if s.commitLogResult.shouldReturnUnfulfilled {
shardTimeRanges := ns.DataRunOptions.ShardTimeRanges
dataResult = shardTimeRanges.ToUnfulfilledDataResult()
}
var indexResult result.IndexBootstrapResult
if ns.Metadata.Options().IndexOptions().Enabled() {
indexResult = result.NewIndexBootstrapResult()
if s.commitLogResult.shouldReturnUnfulfilled {
shardTimeRanges := ns.IndexRunOptions.ShardTimeRanges
indexResult = shardTimeRanges.ToUnfulfilledIndexResult()
}
}
bootstrapResult.Results.Set(id, bootstrap.NamespaceResult{
Metadata: ns.Metadata,
Shards: ns.Shards,
DataResult: dataResult,
IndexResult: indexResult,
})
}
instrCtx.readCommitLogCompleted()
return bootstrapResult, nil
}
func (s *commitLogSource) readCommitLog(namespaces bootstrap.Namespaces, span opentracing.Span) (commitLogResult, error) {
// Setup the series accumulator pipeline.
var (
numWorkers = s.opts.AccumulateConcurrency()
workers = make([]*accumulateWorker, 0, numWorkers)
)
for i := 0; i < numWorkers; i++ {
worker := &accumulateWorker{
inputCh: make(chan accumulateArg, workerChannelSize),
}
workers = append(workers, worker)
}
closedWorkerChannels := false
closeWorkerChannels := func() {
if closedWorkerChannels {
return
}
closedWorkerChannels = true
for _, worker := range workers {
close(worker.inputCh)
}
}
// NB(r): Ensure that channels always get closed.
defer closeWorkerChannels()
var (
namespaceIter = namespaces.Namespaces.Iter()
namespaceResults = make(map[string]*readNamespaceResult, len(namespaceIter))
setInitialTopologyState bool
initialTopologyState *topology.StateSnapshot
)
for _, elem := range namespaceIter {
ns := elem.Value()
// NB(r): Combine all shard time ranges across data and index
// so we can do in one go.
shardTimeRanges := result.NewShardTimeRanges()
// NB(bodu): Use TargetShardTimeRanges which covers the entire original target shard range
// since the commitlog bootstrapper should run for the entire bootstrappable range per shard.
shardTimeRanges.AddRanges(ns.DataRunOptions.TargetShardTimeRanges)
if ns.Metadata.Options().IndexOptions().Enabled() {
shardTimeRanges.AddRanges(ns.IndexRunOptions.TargetShardTimeRanges)
}
namespaceResults[ns.Metadata.ID().String()] = &readNamespaceResult{
namespace: ns,
dataAndIndexShardRanges: shardTimeRanges,
}
// Make the initial topology state available.
if !setInitialTopologyState {
setInitialTopologyState = true
initialTopologyState = ns.DataRunOptions.RunOptions.InitialTopologyState()
}
}
// Setup the commit log iterator.
var (
iterOpts = commitlog.IteratorOpts{
CommitLogOptions: s.opts.CommitLogOptions(),
FileFilterPredicate: s.readCommitLogFilePredicate,
// NB(r): ReturnMetadataAsRef used to all series metadata as
// references instead of pulling from pool and allocating,
// which means need to not hold onto any references returned
// from a call to the commit log read log entry call.
ReturnMetadataAsRef: true,
}
datapointsSkippedNotBootstrappingNamespace = 0
datapointsSkippedNotBootstrappingShard = 0
datapointsSkippedShardNoLongerOwned = 0
startCommitLogsRead = s.nowFn()
encounteredCorruptData = false
)
s.log.Info("read commit logs start")
span.LogEvent("read_commitlogs_start")
defer func() {
datapointsRead := 0
for _, worker := range workers {
datapointsRead += worker.datapointsRead
}
s.log.Info("read commit logs done",
zap.Duration("took", s.nowFn().Sub(startCommitLogsRead)),
zap.Int("datapointsRead", datapointsRead),
zap.Int("datapointsSkippedNotBootstrappingNamespace", datapointsSkippedNotBootstrappingNamespace),
zap.Int("datapointsSkippedNotBootstrappingShard", datapointsSkippedNotBootstrappingShard),
zap.Int("datapointsSkippedShardNoLongerOwned", datapointsSkippedShardNoLongerOwned))
span.LogEvent("read_commitlogs_done")
}()
iter, corruptFiles, err := s.newIteratorFn(iterOpts)
if err != nil {
err = fmt.Errorf("unable to create commit log iterator: %v", err)
return commitLogResult{}, err
}
if len(corruptFiles) > 0 {
s.logAndEmitCorruptFiles(corruptFiles)
encounteredCorruptData = true
}
defer iter.Close()
// Spin up numWorkers background go-routines to handle accumulation. This must
// happen before we start reading to prevent infinitely blocking writes to
// the worker channels.
var wg sync.WaitGroup
for _, worker := range workers {
worker := worker
wg.Add(1)
go func() {
s.startAccumulateWorker(worker)
wg.Done()
}()
}
var (
// NB(r): Use pointer type for the namespaces so we don't have to
// memcopy the large namespace context struct to the work channel and
// can pass by pointer.
// For the commit log series map we use by value since it grows
// large in size and we want to avoid allocating a struct per series
// read and just have a by value struct stored in the map (also makes
// reusing memory set aside on a per series level between commit
// log files much easier to do).
commitLogNamespaces []*bootstrapNamespace
commitLogSeries = make(map[seriesMapKey]seriesMapEntry)
workerEnqueue = 0
tagDecoder = s.opts.CommitLogOptions().FilesystemOptions().TagDecoderPool().Get()
tagDecoderCheckedBytes = checked.NewBytes(nil, nil)
)
tagDecoderCheckedBytes.IncRef()
// Read and accumulate all the log entries in the commit log that we need
// to read.
var lastFileReadID uint64
for iter.Next() {
s.metrics.commitLogEntriesRead.Inc(1)
entry := iter.Current()
currFileReadID := entry.Metadata.FileReadID
if currFileReadID != lastFileReadID {
// NB(r): If switched between files, we can reuse slice and
// map which is useful so map doesn't grow infinitely.
for k := range commitLogSeries {
delete(commitLogSeries, k)
}
lastFileReadID = currFileReadID
}
// First lookup series, if not found we are guaranteed to have
// the series metadata returned by the commit log reader.
seriesKey := seriesMapKey{
fileReadID: entry.Metadata.FileReadID,
uniqueIndex: entry.Metadata.SeriesUniqueIndex,
}
seriesEntry, ok := commitLogSeries[seriesKey]
if !ok {
// Resolve the namespace.
var (
nsID = entry.Series.Namespace
nsIDBytes = nsID.Bytes()
ns *bootstrapNamespace
)
for _, elem := range commitLogNamespaces {
if bytes.Equal(elem.namespaceID, nsIDBytes) {
ns = elem
break
}
}
if ns == nil {
// NB(r): Need to create an entry into our namespaces, this will happen
// at most once per commit log file read and unique namespace.
nsResult, ok := namespaceResults[nsID.String()]
// Take a copy so that not taking ref to reused bytes from the commit log.
nsIDCopy := append([]byte(nil), nsIDBytes...)
if !ok {
// Not bootstrapping this namespace.
ns = &bootstrapNamespace{
namespaceID: nsIDCopy,
bootstrapping: false,
}
} else {
// Bootstrapping this namespace.
nsMetadata := nsResult.namespace.Metadata
ns = &bootstrapNamespace{
namespaceID: nsIDCopy,
bootstrapping: true,
dataAndIndexShardRanges: nsResult.dataAndIndexShardRanges,
namespace: nsMetadata,
namespaceContext: namespace.NewContextFrom(nsMetadata),
dataBlockSize: nsMetadata.Options().RetentionOptions().BlockSize(),
accumulator: nsResult.namespace.DataAccumulator,
}
}
// Append for quick re-lookup with other series.
commitLogNamespaces = append(commitLogNamespaces, ns)
}
if !ns.bootstrapping {
// NB(r): Just set the series map entry to the memoized
// fact that we are not bootstrapping this namespace.
seriesEntry = seriesMapEntry{
namespace: ns,
}
} else {
// Resolve the series in the accumulator.
accumulator := ns.accumulator
var tagIter ident.TagIterator
if len(entry.Series.EncodedTags) > 0 {
tagDecoderCheckedBytes.Reset(entry.Series.EncodedTags)
tagDecoder.Reset(tagDecoderCheckedBytes)
tagIter = tagDecoder
} else {
// NB(r): Always expect a tag iterator in checkout series.
tagIter = ident.EmptyTagIterator
}
// Check out the series for writing, no need for concurrency
// as commit log bootstrapper does not perform parallel
// checking out of series.
series, owned, err := accumulator.CheckoutSeriesWithoutLock(
entry.Series.Shard,
entry.Series.ID,
tagIter)
if err != nil {
if !owned {
// If we encounter a log entry for a shard that we're
// not responsible for, skip this entry. This can occur
// when a topology change happens and we bootstrap from
// a commit log which contains this data.
commitLogSeries[seriesKey] = seriesMapEntry{shardNoLongerOwned: true}
continue
}
return commitLogResult{}, err
}
seriesEntry = seriesMapEntry{
namespace: ns,
series: series,
}
}
commitLogSeries[seriesKey] = seriesEntry
}
// If series is no longer owned, then we can safely skip trying to
// bootstrap the result.
if seriesEntry.shardNoLongerOwned {
datapointsSkippedShardNoLongerOwned++
continue
}
// If not bootstrapping this namespace then skip this result.
if !seriesEntry.namespace.bootstrapping {
datapointsSkippedNotBootstrappingNamespace++
continue
}
// If not bootstrapping shard for this series then also skip.
// NB(r): This can occur when a topology change happens then we
// bootstrap from the commit log data that the node no longer owns.
shard := seriesEntry.series.Shard
_, ok = seriesEntry.namespace.dataAndIndexShardRanges.Get(shard)
if !ok {
datapointsSkippedNotBootstrappingShard++
continue
}
arg := accumulateArg{
namespace: seriesEntry.namespace,
series: seriesEntry.series,
shard: seriesEntry.series.Shard,
dp: entry.Datapoint,
unit: entry.Unit,
}
annotationLen := len(entry.Annotation)
if annotationLen > 0 {
// Use the predefined buffer if the annotation fits in it.
if annotationLen <= len(arg.shortAnnotation) {
copy(arg.shortAnnotation[:], entry.Annotation)
arg.shortAnnotationLen = uint8(annotationLen)
} else {
// Otherwise allocate.
arg.longAnnotation = append(make([]byte, 0, annotationLen), entry.Annotation...)
}
}
// Distribute work.
// NB(r): In future we could batch a few points together before sending
// to a channel to alleviate lock contention/stress on the channels.
workerEnqueue++
worker := workers[workerEnqueue%numWorkers]
worker.inputCh <- arg
}
if iterErr := iter.Err(); iterErr != nil {
// Log the error and mark that we encountered corrupt data, but don't
// return the error because we want to give the peers bootstrapper the
// opportunity to repair the data instead of failing the bootstrap
// altogether.
s.log.Error("error in commitlog iterator", zap.Error(iterErr))
s.metrics.corruptCommitlogFile.Inc(1)
encounteredCorruptData = true
}
// Close the worker channels since we've enqueued all required data.
closeWorkerChannels()
// Block until all required data from the commit log has been read and
// accumulated by the worker goroutines.
wg.Wait()
// Log the outcome and calculate if required to return unfulfilled.
s.logAccumulateOutcome(workers, iter)
shouldReturnUnfulfilled, err := s.shouldReturnUnfulfilled(
workers, encounteredCorruptData, initialTopologyState)
if err != nil {
return commitLogResult{}, err
}
return commitLogResult{shouldReturnUnfulfilled: shouldReturnUnfulfilled, read: true}, nil
}
func (s *commitLogSource) snapshotFilesByShard(
nsID ident.ID,
filePathPrefix string,
shardsTimeRanges result.ShardTimeRanges,
) (map[uint32]fs.FileSetFilesSlice, error) {
snapshotFilesByShard := map[uint32]fs.FileSetFilesSlice{}
for shard := range shardsTimeRanges.Iter() {
snapshotFiles, err := s.snapshotFilesFn(filePathPrefix, nsID, shard)
if err != nil {
return nil, err
}
snapshotFilesByShard[shard] = snapshotFiles
}
return snapshotFilesByShard, nil
}
// mostRecentCompleteSnapshotByBlockShard returns a
// map[xtime.UnixNano]map[uint32]fs.FileSetFile with the contract that
// for each shard/block combination in shardsTimeRanges, an entry will
// exist in the map such that FileSetFile.CachedSnapshotTime is the
// actual cached snapshot time, or the blockStart.
func (s *commitLogSource) mostRecentCompleteSnapshotByBlockShard(
shardsTimeRanges result.ShardTimeRanges,
blockSize time.Duration,
snapshotFilesByShard map[uint32]fs.FileSetFilesSlice,
fsOpts fs.Options,
) map[xtime.UnixNano]map[uint32]fs.FileSetFile {
var (
minBlock, maxBlock = shardsTimeRanges.MinMax()
mostRecentSnapshotsByBlockShard = map[xtime.UnixNano]map[uint32]fs.FileSetFile{}
)
for currBlockStart := minBlock.Truncate(blockSize); currBlockStart.Before(maxBlock); currBlockStart = currBlockStart.Add(blockSize) {
for shard := range shardsTimeRanges.Iter() {
// Anonymous func for easier clean up using defer.
func() {
var (
mostRecentSnapshot fs.FileSetFile
)
defer func() {
existing := mostRecentSnapshotsByBlockShard[currBlockStart]
if existing == nil {
existing = map[uint32]fs.FileSetFile{}
}
if mostRecentSnapshot.IsZero() {
// If we were unable to determine the most recent snapshot time for a given
// shard/blockStart combination, then just fall back to using the blockStart
// time as that will force us to read the entire commit log for that duration.
mostRecentSnapshot.CachedSnapshotTime = currBlockStart
}
existing[shard] = mostRecentSnapshot
mostRecentSnapshotsByBlockShard[currBlockStart] = existing
}()
snapshotFiles, ok := snapshotFilesByShard[shard]
if !ok {
// If there are no snapshot files for this shard, then rely on
// the defer to fallback to using the block start time.
return
}
mostRecentSnapshotVolume, ok := snapshotFiles.LatestVolumeForBlock(currBlockStart)
if !ok {
// If there are no complete snapshot files for this block, then rely on
// the defer to fallback to using the block start time.
return
}
// Make sure we're able to read the snapshot time. This will also set the
// CachedSnapshotTime field so that we can rely upon it from here on out.
_, _, err := mostRecentSnapshotVolume.SnapshotTimeAndID()
if err != nil {
namespace := mostRecentSnapshot.ID.Namespace
if namespace == nil {
namespace = ident.StringID("<nil>")
}
s.log.
With(
zap.Stringer("namespace", namespace),
zap.Time("blockStart", mostRecentSnapshot.ID.BlockStart.ToTime()),
zap.Uint32("shard", mostRecentSnapshot.ID.Shard),
zap.Int("index", mostRecentSnapshot.ID.VolumeIndex),
zap.Strings("filepaths", mostRecentSnapshot.AbsoluteFilePaths),
zap.Error(err),
).
Error("error resolving snapshot time for snapshot file")
// If we couldn't determine the snapshot time for the snapshot file, then rely
// on the defer to fallback to using the block start time.
return
}
mostRecentSnapshot = mostRecentSnapshotVolume
}()
}
}
return mostRecentSnapshotsByBlockShard
}
func (s *commitLogSource) bootstrapShardSnapshots(
ns namespace.Metadata,
accumulator bootstrap.NamespaceDataAccumulator,
shard uint32,
shardTimeRanges xtime.Ranges,
blockSize time.Duration,
mostRecentCompleteSnapshotByBlockShard map[xtime.UnixNano]map[uint32]fs.FileSetFile,
cache bootstrap.Cache,
) error {
// NB(bodu): We use info files on disk to check if a snapshot should be loaded in as cold or warm.
// We do this instead of cross refing blockstarts and current time to handle the case of bootstrapping a
// once warm block start after a node has been shut down for a long time. We consider all block starts we
// haven't flushed data for yet a warm block start.
readInfoFilesResults, err := cache.InfoFilesForShard(ns, shard)
if err != nil {
return err
}
shardBlockStartsOnDisk := make(map[xtime.UnixNano]struct{})
for _, result := range readInfoFilesResults {
if err := result.Err.Error(); err != nil {
// If we couldn't read the info files then keep going to be consistent
// with the way the db shard updates its flush states in UpdateFlushStates().
s.log.Error("unable to read info files in commit log bootstrap",
zap.Uint32("shard", shard),
zap.Stringer("namespace", ns.ID()),
zap.String("filepath", result.Err.Filepath()),
zap.Error(err))
continue
}
info := result.Info
at := xtime.FromNanoseconds(info.BlockStart)
shardBlockStartsOnDisk[xtime.ToUnixNano(at)] = struct{}{}
}
rangeIter := shardTimeRanges.Iter()
for rangeIter.Next() {
var (
currRange = rangeIter.Value()
currRangeDuration = currRange.End.Sub(currRange.Start)
isMultipleOfBlockSize = currRangeDuration%blockSize == 0
)
if !isMultipleOfBlockSize {
return fmt.Errorf(
"received bootstrap range that is not multiple of blockSize, blockSize: %d, start: %s, end: %s",
blockSize, currRange.End.String(), currRange.Start.String(),
)
}
for blockStart := currRange.Start.Truncate(blockSize); blockStart.Before(currRange.End); blockStart = blockStart.Add(blockSize) {
snapshotsForBlock := mostRecentCompleteSnapshotByBlockShard[blockStart]
mostRecentCompleteSnapshotForShardBlock := snapshotsForBlock[shard]
if mostRecentCompleteSnapshotForShardBlock.CachedSnapshotTime.Equal(blockStart) ||
// Should never happen
mostRecentCompleteSnapshotForShardBlock.IsZero() {
// There is no snapshot file for this time, and even if there was, there would
// be no point in reading it. In this specific case its not an error scenario
// because the fact that snapshotTime == blockStart means we already accounted
// for the fact that this snapshot did not exist when we were deciding which
// commit logs to read.
s.log.Debug("no snapshots for shard and blockStart",
zap.Uint32("shard", shard), zap.Time("blockStart", blockStart.ToTime()))
continue
}
writeType := series.WarmWrite
if _, ok := shardBlockStartsOnDisk[blockStart]; ok {
writeType = series.ColdWrite
}
if err := s.bootstrapShardBlockSnapshot(
ns, accumulator, shard, blockStart, blockSize,
mostRecentCompleteSnapshotForShardBlock, writeType); err != nil {
return err
}
}
}
return nil
}
func (s *commitLogSource) bootstrapShardBlockSnapshot(
ns namespace.Metadata,
accumulator bootstrap.NamespaceDataAccumulator,
shard uint32,
blockStart xtime.UnixNano,
blockSize time.Duration,
mostRecentCompleteSnapshot fs.FileSetFile,
writeType series.WriteType,
) error {
var (
bOpts = s.opts.ResultOptions()
blOpts = bOpts.DatabaseBlockOptions()
blocksPool = blOpts.DatabaseBlockPool()
bytesPool = blOpts.BytesPool()
fsOpts = s.opts.CommitLogOptions().FilesystemOptions()
nsCtx = namespace.NewContextFrom(ns)
numWorkers = s.opts.AccumulateConcurrency()
)
// Bootstrap the snapshot file.
reader, err := s.newReaderFn(bytesPool, fsOpts)
if err != nil {
return err
}
err = reader.Open(fs.DataReaderOpenOptions{
Identifier: fs.FileSetFileIdentifier{
Namespace: ns.ID(),
BlockStart: blockStart,
Shard: shard,
VolumeIndex: mostRecentCompleteSnapshot.ID.VolumeIndex,
},
FileSetType: persist.FileSetSnapshotType,
})
if err != nil {
return err
}
defer func() {
err := reader.Close()
if err != nil {
s.log.Error("error closing reader for shard",
zap.Uint32("shard", shard),
zap.Time("blockStart", blockStart.ToTime()),
zap.Int("volume", mostRecentCompleteSnapshot.ID.VolumeIndex),
zap.Error(err))
}
}()
s.log.Debug("reading snapshot for shard",
zap.Uint32("shard", shard),
zap.Time("blockStart", blockStart.ToTime()),
zap.Int("volume", mostRecentCompleteSnapshot.ID.VolumeIndex))
worker := &readSeriesBlocksWorker{
dataCh: make(chan seriesBlock, readSeriesBlocksWorkerChannelSize),
reader: reader,
shard: shard,
accumulator: accumulator,
blocksPool: blocksPool,
blockStart: blockStart,
blockSize: blockSize,
nsCtx: nsCtx,
}
errs, ctx := errgroup.WithContext(context.Background())
errs.Go(func() error {
return worker.readSeriesBlocks(ctx)
})
for i := 0; i < numWorkers; i++ {
errs.Go(func() error {
return s.loadBlocks(worker.dataCh, writeType)
})
}
return errs.Wait()
}
func (s *commitLogSource) loadBlocks(dataCh <-chan seriesBlock, writeType series.WriteType) error {
for seriesBlock := range dataCh {
// Load into series.
seriesRef, err := seriesBlock.resolver.SeriesRef()
if err != nil {
return fmt.Errorf("(commitlog) unable to resolve series ref: %w", err)
}
if err := seriesRef.LoadBlock(seriesBlock.block, writeType); err != nil {
return err
}
}
return nil