forked from m3db/m3
/
source.go
1581 lines (1395 loc) · 48.4 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 (
"errors"
"fmt"
"io"
"sync"
"time"
"github.com/m3db/m3/src/cluster/shard"
"github.com/m3db/m3/src/dbnode/encoding"
"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/index/convert"
"github.com/m3db/m3/src/dbnode/topology"
"github.com/m3db/m3/src/dbnode/ts"
"github.com/m3db/m3/src/dbnode/x/xio"
"github.com/m3db/m3/src/x/checked"
"github.com/m3db/m3/src/x/ident"
"github.com/m3db/m3/src/x/instrument"
"github.com/m3db/m3/src/x/pool"
xsync "github.com/m3db/m3/src/x/sync"
xtime "github.com/m3db/m3/src/x/time"
"github.com/uber-go/tally"
"go.uber.org/zap"
)
var (
errIndexingNotEnableForNamespace = errors.New("indexing not enabled for namespace")
)
const (
encoderChanBufSize = 1000
)
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 newReaderFn func(bytesPool pool.CheckedBytesPool, opts fs.Options) (fs.DataFileSetReader, error)
type commitLogSource struct {
opts Options
log *zap.Logger
// Filesystem inspection capture before node was started.
inspection fs.Inspection
newIteratorFn newIteratorFn
snapshotFilesFn snapshotFilesFn
newReaderFn newReaderFn
metrics commitLogSourceDataAndIndexMetrics
}
type encoder struct {
lastWriteAt time.Time
enc encoding.Encoder
}
func newCommitLogSource(opts Options, inspection fs.Inspection) bootstrap.Source {
scope := opts.
ResultOptions().
InstrumentOptions().
MetricsScope().
SubScope("bootstrapper-commitlog")
return &commitLogSource{
opts: opts,
log: opts.
ResultOptions().
InstrumentOptions().
Logger().
With(zap.String("bootstrapper", "commitlog")),
inspection: inspection,
newIteratorFn: commitlog.NewIterator,
snapshotFilesFn: fs.SnapshotFiles,
newReaderFn: fs.NewReader,
metrics: newCommitLogSourceDataAndIndexMetrics(scope),
}
}
func (s *commitLogSource) Can(strategy bootstrap.Strategy) bool {
switch strategy {
case bootstrap.BootstrapSequential:
return true
}
return false
}
func (s *commitLogSource) AvailableData(
ns namespace.Metadata,
shardsTimeRanges result.ShardTimeRanges,
runOpts bootstrap.RunOptions,
) (result.ShardTimeRanges, error) {
return s.availability(ns, shardsTimeRanges, runOpts)
}
// ReadData 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) ReadData(
ns namespace.Metadata,
shardsTimeRanges result.ShardTimeRanges,
runOpts bootstrap.RunOptions,
) (result.DataBootstrapResult, error) {
if shardsTimeRanges.IsEmpty() {
return result.NewDataBootstrapResult(), nil
}
var (
// Emit bootstrapping gauge for duration of ReadData
doneReadingData = s.metrics.data.emitBootstrapping()
encounteredCorruptData = false
fsOpts = s.opts.CommitLogOptions().FilesystemOptions()
filePathPrefix = fsOpts.FilePathPrefix()
)
defer doneReadingData()
// Determine which snapshot files are available.
snapshotFilesByShard, err := s.snapshotFilesByShard(
ns.ID(), filePathPrefix, shardsTimeRanges)
if err != nil {
return nil, err
}
var (
bOpts = s.opts.ResultOptions()
blOpts = bOpts.DatabaseBlockOptions()
blockSize = ns.Options().RetentionOptions().BlockSize()
)
readCommitLogPred, mostRecentCompleteSnapshotByBlockShard, err := s.newReadCommitlogPredAndMostRecentSnapshotByBlockShard(
ns, shardsTimeRanges, snapshotFilesByShard)
if err != nil {
return nil, err
}
// Setup the commit log iterator.
var (
nsID = ns.ID()
seriesSkipped int
datapointsSkipped int
datapointsRead int
// TODO(rartoul): When we implement caching data across namespaces, this will need
// to be commitlog.ReadAllSeriesPredicate() if CacheSeriesMetadata() is enabled
// because we'll need to read data for all namespaces, not just the one we're currently
// bootstrapping.
readSeriesPredicate = func(id ident.ID, namespace ident.ID) bool {
shouldReadSeries := nsID.Equal(namespace)
if !shouldReadSeries {
seriesSkipped++
}
return shouldReadSeries
}
iterOpts = commitlog.IteratorOpts{
CommitLogOptions: s.opts.CommitLogOptions(),
FileFilterPredicate: readCommitLogPred,
SeriesFilterPredicate: readSeriesPredicate,
}
)
defer func() {
s.log.Info("ReadData finished",
zap.Int("seriesSkipped", seriesSkipped),
zap.Int("datapointsSkipped", datapointsSkipped),
zap.Int("datapointsRead", datapointsRead))
}()
iter, corruptFiles, err := s.newIteratorFn(iterOpts)
if err != nil {
return nil, fmt.Errorf("unable to create commit log iterator: %v", err)
}
if len(corruptFiles) > 0 {
s.logAndEmitCorruptFiles(corruptFiles, true)
encounteredCorruptData = true
}
defer iter.Close()
// Setup the M3TSZ encoding pipeline
var (
// +1 so we can use the shard number as an index throughout without constantly
// remembering to subtract 1 to convert to zero-based indexing
numShards = s.findHighestShard(shardsTimeRanges) + 1
numConc = s.opts.EncodingConcurrency()
encoderPool = blOpts.EncoderPool()
workerErrs = make([]int, numConc)
shardDataByShard = s.newShardDataByShard(shardsTimeRanges, numShards)
)
encoderChans := make([]chan encoderArg, numConc)
for i := 0; i < numConc; i++ {
encoderChans[i] = make(chan encoderArg, encoderChanBufSize)
}
// Spin up numConc background go-routines to handle M3TSZ encoding. This must
// happen before we start reading to prevent infinitely blocking writes to
// the encoderChans.
wg := &sync.WaitGroup{}
for workerNum, encoderChan := range encoderChans {
wg.Add(1)
go s.startM3TSZEncodingWorker(
ns, runOpts, workerNum, encoderChan, shardDataByShard, encoderPool, workerErrs, blOpts, wg)
}
// Read / M3TSZ encode all the datapoints in the commit log that we need to read.
for iter.Next() {
series, dp, unit, annotation := iter.Current()
if !s.shouldEncodeForData(shardDataByShard, blockSize, series, dp.Timestamp) {
datapointsSkipped++
continue
}
datapointsRead++
// Distribute work such that each encoder goroutine is responsible for
// approximately numShards / numConc shards. This also means that all
// datapoints for a given shard/series will be processed in a serialized
// manner.
// We choose to distribute work by shard instead of series.UniqueIndex
// because it means that all accesses to the shardDataByShard slice don't need
// to be synchronized because each index belongs to a single shard so it
// will only be accessed serially from a single worker routine.
encoderChans[series.Shard%uint32(numConc)] <- encoderArg{
series: series,
dp: dp,
unit: unit,
annotation: annotation,
blockStart: dp.Timestamp.Truncate(blockSize),
}
}
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.data.corruptCommitlogFile.Inc(1)
encounteredCorruptData = true
}
for _, encoderChan := range encoderChans {
close(encoderChan)
}
// Block until all required data from the commit log has been read and
// encoded by the worker goroutines
wg.Wait()
s.logEncodingOutcome(workerErrs, iter)
// Merge all the different encoders from the commit log that we created with
// the data that is available in the snapshot files.
s.log.Info("starting merge...")
mergeStart := time.Now()
bootstrapResult, err := s.mergeAllShardsCommitLogEncodersAndSnapshots(
ns,
shardsTimeRanges,
snapshotFilesByShard,
mostRecentCompleteSnapshotByBlockShard,
int(numShards),
blockSize,
shardDataByShard,
)
if err != nil {
return nil, err
}
s.log.Info("done merging...", zap.Duration("took", time.Since(mergeStart)))
shouldReturnUnfulfilled, err := s.shouldReturnUnfulfilled(
encounteredCorruptData, ns, shardsTimeRanges, runOpts)
if err != nil {
return nil, err
}
if shouldReturnUnfulfilled {
bootstrapResult.SetUnfulfilled(shardsTimeRanges)
}
return bootstrapResult, 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 {
snapshotFiles, err := s.snapshotFilesFn(filePathPrefix, nsID, shard)
if err != nil {
return nil, err
}
snapshotFilesByShard[shard] = snapshotFiles
}
return snapshotFilesByShard, nil
}
func (s *commitLogSource) newShardDataByShard(
shardsTimeRanges result.ShardTimeRanges,
numShards uint32,
) []shardData {
shardDataByShard := make([]shardData, numShards)
for shard := range shardsTimeRanges {
shardDataByShard[shard] = shardData{
series: NewMap(MapOptions{}),
ranges: shardsTimeRanges[shard],
}
}
return shardDataByShard
}
// 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 {
// Anonymous func for easier clean up using defer.
func() {
var (
currBlockUnixNanos = xtime.ToUnixNano(currBlockStart)
mostRecentSnapshot fs.FileSetFile
)
defer func() {
existing := mostRecentSnapshotsByBlockShard[currBlockUnixNanos]
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[currBlockUnixNanos] = 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),
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,
shard uint32,
metadataOnly bool,
shardTimeRanges xtime.Ranges,
blockSize time.Duration,
snapshotFiles fs.FileSetFilesSlice,
mostRecentCompleteSnapshotByBlockShard map[xtime.UnixNano]map[uint32]fs.FileSetFile,
) (result.ShardResult, error) {
var (
shardResult result.ShardResult
allSeriesSoFar *result.Map
rangeIter = shardTimeRanges.Iter()
err error
)
for rangeIter.Next() {
var (
currRange = rangeIter.Value()
currRangeDuration = currRange.End.Sub(currRange.Start)
isMultipleOfBlockSize = currRangeDuration%blockSize == 0
)
if !isMultipleOfBlockSize {
return nil, fmt.Errorf(
"received bootstrap range that is not multiple of blockSize, blockSize: %d, start: %s, end: %s",
blockSize, currRange.End.String(), currRange.Start.String(),
)
}
// Reset this after we bootstrap each block to make sure its up to date.
if shardResult != nil {
allSeriesSoFar = shardResult.AllSeries()
}
for blockStart := currRange.Start.Truncate(blockSize); blockStart.Before(currRange.End); blockStart = blockStart.Add(blockSize) {
snapshotsForBlock := mostRecentCompleteSnapshotByBlockShard[xtime.ToUnixNano(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))
continue
}
shardResult, err = s.bootstrapShardBlockSnapshot(
ns, shard, blockStart, metadataOnly, shardResult, allSeriesSoFar, blockSize,
snapshotFiles, mostRecentCompleteSnapshotForShardBlock)
if err != nil {
return shardResult, err
}
}
}
if shardResult == nil {
shardResult = result.NewShardResult(0, s.opts.ResultOptions())
}
return shardResult, nil
}
func (s *commitLogSource) bootstrapShardBlockSnapshot(
ns namespace.Metadata,
shard uint32,
blockStart time.Time,
metadataOnly bool,
shardResult result.ShardResult,
allSeriesSoFar *result.Map,
blockSize time.Duration,
snapshotFiles fs.FileSetFilesSlice,
mostRecentCompleteSnapshot fs.FileSetFile,
) (result.ShardResult, error) {
var (
bOpts = s.opts.ResultOptions()
blOpts = bOpts.DatabaseBlockOptions()
blocksPool = blOpts.DatabaseBlockPool()
bytesPool = blOpts.BytesPool()
fsOpts = s.opts.CommitLogOptions().FilesystemOptions()
idPool = s.opts.CommitLogOptions().IdentifierPool()
nsCtx = namespace.NewContextFrom(ns)
)
// Bootstrap the snapshot file
reader, err := s.newReaderFn(bytesPool, fsOpts)
if err != nil {
return shardResult, 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 shardResult, 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),
zap.Int("volume", mostRecentCompleteSnapshot.ID.VolumeIndex),
zap.Error(err))
}
}()
s.log.Debug("reading snapshot for shard",
zap.Uint32("shard", shard),
zap.Time("blockStart", blockStart),
zap.Int("volume", mostRecentCompleteSnapshot.ID.VolumeIndex))
for {
var (
id ident.ID
tagsIter ident.TagIterator
data checked.Bytes
expectedChecksum uint32
)
if metadataOnly {
id, tagsIter, _, _, err = reader.ReadMetadata()
} else {
id, tagsIter, data, expectedChecksum, err = reader.Read()
}
if err != nil && err != io.EOF {
return shardResult, err
}
if err == io.EOF {
break
}
dbBlock := blocksPool.Get()
dbBlock.Reset(blockStart, blockSize, ts.NewSegment(data, nil, ts.FinalizeHead), nsCtx)
if !metadataOnly {
// Resetting the block will trigger a checksum calculation, so use that instead
// of calculating it twice.
checksum, err := dbBlock.Checksum()
if err != nil {
return shardResult, err
}
if checksum != expectedChecksum {
return shardResult, fmt.Errorf("checksum for series: %s was %d but expected %d", id, checksum, expectedChecksum)
}
}
var (
tags ident.Tags
shouldDecodeTags = true
)
if allSeriesSoFar != nil {
if existing, ok := allSeriesSoFar.Get(id); ok {
// If we've already bootstrapped this series for a different block, we don't need
// another copy of the IDs and tags.
id.Finalize()
id = existing.ID
tags = existing.Tags
shouldDecodeTags = false
}
}
if shouldDecodeTags {
// Only spend cycles decoding the tags if we've never seen them before.
if tagsIter.Remaining() > 0 {
tags, err = convert.TagsFromTagsIter(id, tagsIter, idPool)
if err != nil {
return shardResult, fmt.Errorf("unable to decode tags: %v", err)
}
}
}
// Always close even if we didn't use it.
tagsIter.Close()
// Mark the ID and Tags as no finalize to enable no-copy optimization later
// in the bootstrap process (when they're being loaded into the shard). Also,
// technically we'll be calling NoFinalize() repeatedly on the same IDs for
// different blocks since we're reusing them, but thats ok as it an idempotent
// operation and there is no concurrency here.
id.NoFinalize()
tags.NoFinalize()
if shardResult == nil {
// Delay initialization so we can estimate size.
shardResult = result.NewShardResult(reader.Entries(), s.opts.ResultOptions())
}
shardResult.AddBlock(id, tags, dbBlock)
}
return shardResult, nil
}
func (s *commitLogSource) newReadCommitlogPredAndMostRecentSnapshotByBlockShard(
ns namespace.Metadata,
shardsTimeRanges result.ShardTimeRanges,
snapshotFilesByShard map[uint32]fs.FileSetFilesSlice,
) (
commitlog.FileFilterPredicate,
map[xtime.UnixNano]map[uint32]fs.FileSetFile,
error,
) {
blockSize := ns.Options().RetentionOptions().BlockSize()
mostRecentCompleteSnapshotByBlockShard := s.mostRecentCompleteSnapshotByBlockShard(
shardsTimeRanges, blockSize, snapshotFilesByShard, s.opts.CommitLogOptions().FilesystemOptions())
for block, mostRecentByShard := range mostRecentCompleteSnapshotByBlockShard {
for shard, mostRecent := range mostRecentByShard {
if mostRecent.CachedSnapshotTime.IsZero() {
// Should never happen.
return nil, nil, instrument.InvariantErrorf(
"shard: %d and block: %s had zero value for most recent snapshot time",
shard, block.ToTime().String())
}
s.log.Debug("most recent snapshot for block",
zap.Time("blockStart", block.ToTime()),
zap.Uint32("shard", shard),
zap.Time("mostRecent", mostRecent.CachedSnapshotTime))
}
}
// TODO(rartoul): Refactor this to take the SnapshotMetadata files into account to reduce
// the number of commitlog files that need to be read.
return func(f commitlog.FileFilterInfo) bool {
// Read all the commitlog files that were available on disk before the node started
// accepting writes.
commitlogFilesPresentBeforeStart := s.inspection.CommitLogFilesSet()
if f.IsCorrupt {
// Corrupt files that existed on disk before the node started should be included so
// that the commitlog bootstrapper can detect them and determine if it will return
// unfulfilled or ignore them.
//
// Corrupt files that did not exist on disk before the node started should always be
// ignored since they have no impact on the bootstrapping process and likely only
// appear corrupt because they were just created recently by the current node as
// its alreadying accepting writes at this point.
_, ok := commitlogFilesPresentBeforeStart[f.Err.Path()]
return ok
}
// Only attempt to read commitlog files that were present on disk before the node started.
// If a commitlog file was not present when the node started then it was created once the
// node began accepting writes and the data is already in memory.
_, ok := commitlogFilesPresentBeforeStart[f.File.FilePath]
return ok
}, mostRecentCompleteSnapshotByBlockShard, nil
}
func (s *commitLogSource) startM3TSZEncodingWorker(
ns namespace.Metadata,
runOpts bootstrap.RunOptions,
workerNum int,
ec <-chan encoderArg,
unmerged []shardData,
encoderPool encoding.EncoderPool,
workerErrs []int,
blopts block.Options,
wg *sync.WaitGroup,
) {
nsCtx := namespace.NewContextFrom(ns)
for arg := range ec {
var (
series = arg.series
dp = arg.dp
unit = arg.unit
annotation = arg.annotation
blockStart = arg.blockStart
)
var (
unmergedShard = unmerged[series.Shard].series
unmergedSeries, ok = unmergedShard.Get(series.ID)
)
if !ok {
unmergedSeries = metadataAndEncodersByTime{
id: series.ID,
tags: series.Tags,
encoders: make(map[xtime.UnixNano][]encoder)}
// Have to use unsafe because we don't want to copy the IDs we put
// into this map because its lifecycle is much shorter than that of
// the IDs we're putting into it so copying would waste too much
// memory unnecessarily, and we don't want to finalize the IDs for the
// same reason.
unmergedShard.SetUnsafe(
series.ID, unmergedSeries,
SetUnsafeOptions{NoCopyKey: true, NoFinalizeKey: true})
}
var (
err error
blockStartNano = xtime.ToUnixNano(blockStart)
unmergedBlock = unmergedSeries.encoders[blockStartNano]
wroteExisting = false
)
for i := range unmergedBlock {
// TODO(r): Write unit test to ensure that different values that arrive
// later in the commit log will upsert the previous value when bootstrapping
// Tracking with issue: https://github.com/m3db/m3/issues/898
if unmergedBlock[i].lastWriteAt.Before(dp.Timestamp) {
unmergedBlock[i].lastWriteAt = dp.Timestamp
err = unmergedBlock[i].enc.Encode(dp, unit, annotation)
wroteExisting = true
break
}
}
if !wroteExisting {
enc := encoderPool.Get()
enc.Reset(blockStart, blopts.DatabaseBlockAllocSize(), nsCtx.Schema)
err = enc.Encode(dp, unit, annotation)
if err == nil {
unmergedBlock = append(unmergedBlock, encoder{
lastWriteAt: dp.Timestamp,
enc: enc,
})
unmergedSeries.encoders[blockStartNano] = unmergedBlock
}
}
if err != nil {
workerErrs[workerNum]++
}
}
wg.Done()
}
func (s *commitLogSource) shouldEncodeForData(
unmerged []shardData,
dataBlockSize time.Duration,
series ts.Series,
timestamp time.Time,
) bool {
// Check if the shard number is higher the amount of space we pre-allocated.
// If it is, then it's not one of the shards we're trying to bootstrap
if series.Shard > uint32(len(unmerged)-1) {
return false
}
// Check if the shard is one of the shards we're trying to bootstrap
ranges := unmerged[series.Shard].ranges
if ranges.IsEmpty() {
// Did not allocate map for this shard so not expecting data for it
return false
}
// Check if the block corresponds to the time-range that we're trying to bootstrap
blockStart := timestamp.Truncate(dataBlockSize)
blockEnd := blockStart.Add(dataBlockSize)
blockRange := xtime.Range{
Start: blockStart,
End: blockEnd,
}
return ranges.Overlaps(blockRange)
}
func (s *commitLogSource) shouldIncludeInIndex(
shard uint32,
ts time.Time,
highestShard uint32,
indexBlockSize time.Duration,
bootstrapRangesByShard []xtime.Ranges,
) bool {
if shard > highestShard {
// Not trying to bootstrap this shard
return false
}
rangesToBootstrap := bootstrapRangesByShard[shard]
if rangesToBootstrap.IsEmpty() {
// No ShardTimeRanges were provided for this shard, so we're not
// bootstrapping it.
return false
}
// Check if the timestamp corresponds to one of the index blocks we're
// trying to bootstrap.
indexBlockStart := ts.Truncate(indexBlockSize)
indexBlockEnd := indexBlockStart.Add(indexBlockSize)
indexBlockRange := xtime.Range{
Start: indexBlockStart,
End: indexBlockEnd,
}
return rangesToBootstrap.Overlaps(indexBlockRange)
}
func (s *commitLogSource) mergeAllShardsCommitLogEncodersAndSnapshots(
ns namespace.Metadata,
shardsTimeRanges result.ShardTimeRanges,
snapshotFiles map[uint32]fs.FileSetFilesSlice,
mostRecentCompleteSnapshotByBlockShard map[xtime.UnixNano]map[uint32]fs.FileSetFile,
numShards int,
blockSize time.Duration,
unmerged []shardData,
) (result.DataBootstrapResult, error) {
var (
shardErrs = make([]int, numShards)
shardEmptyErrs = make([]int, numShards)
bootstrapResult = result.NewDataBootstrapResult()
// Controls how many shards can be merged in parallel
workerPool = xsync.NewWorkerPool(s.opts.MergeShardsConcurrency())
bootstrapResultLock sync.Mutex
wg sync.WaitGroup
)
workerPool.Init()
for shard, unmergedShard := range unmerged {
if unmergedShard.series == nil {
// Not bootstrapping this shard
continue
}
nsCtx := namespace.NewContextFrom(ns)
snapshotData, err := s.bootstrapShardSnapshots(
ns,
uint32(shard),
false,
shardsTimeRanges[uint32(shard)],
blockSize,
snapshotFiles[uint32(shard)],
mostRecentCompleteSnapshotByBlockShard,
)
if err != nil {
bootstrapResultLock.Lock()
// Mark the shard time ranges as unfulfilled so a subsequent bootstrapper
// has the chance to fulfill it.
bootstrapResult.Add(
uint32(shard),
result.NewShardResult(0, s.opts.ResultOptions()),
shardsTimeRanges[uint32(shard)],
)
bootstrapResultLock.Unlock()
return nil, err
}
// Merge snapshot and commit log data
wg.Add(1)
shard, unmergedShard := shard, unmergedShard
mergeShardFunc := func() {
var shardResult result.ShardResult
shardResult, shardEmptyErrs[shard], shardErrs[shard] = s.mergeShardCommitLogEncodersAndSnapshots(nsCtx,
shard, snapshotData, unmergedShard, blockSize)
if shardResult != nil && shardResult.NumSeries() > 0 {
// Prevent race conditions while updating bootstrapResult from multiple go-routines
bootstrapResultLock.Lock()
if shardEmptyErrs[shard] != 0 || shardErrs[shard] != 0 {
// If there were any errors, keep the data but mark the shard time ranges as
// unfulfilled so a subsequent bootstrapper has the chance to fulfill it.
bootstrapResult.Add(uint32(shard), shardResult, shardsTimeRanges[uint32(shard)])
} else {
bootstrapResult.Add(uint32(shard), shardResult, xtime.Ranges{})
}
bootstrapResultLock.Unlock()
}
wg.Done()
}
workerPool.Go(mergeShardFunc)
}
// Wait for all merge goroutines to complete
wg.Wait()
s.logMergeShardsOutcome(shardErrs, shardEmptyErrs)
return bootstrapResult, nil
}
func (s *commitLogSource) mergeShardCommitLogEncodersAndSnapshots(
nsCtx namespace.Context,
shard int,
snapshotData result.ShardResult,
unmergedShard shardData,
blockSize time.Duration,
) (result.ShardResult, int, int) {
var (
bOpts = s.opts.ResultOptions()
blOpts = bOpts.DatabaseBlockOptions()
blocksPool = blOpts.DatabaseBlockPool()
multiReaderIteratorPool = blOpts.MultiReaderIteratorPool()
segmentReaderPool = blOpts.SegmentReaderPool()
encoderPool = blOpts.EncoderPool()
)
numSeries := 0
if unmergedShard.series != nil {
numSeries = unmergedShard.series.Len()
}
var (
shardResult = result.NewShardResult(numSeries, s.opts.ResultOptions())
numShardEmptyErrs int
numErrs int
)
allSnapshotSeries := snapshotData.AllSeries()
if unmergedShard.series != nil {
for _, unmergedBlocks := range unmergedShard.series.Iter() {
val := unmergedBlocks.Value()
snapshotSeriesData, _ := allSnapshotSeries.Get(val.id)
seriesBlocks, numSeriesEmptyErrs, numSeriesErrs := s.mergeSeries(
nsCtx,
snapshotSeriesData,
val,
blocksPool,
multiReaderIteratorPool,
segmentReaderPool,
encoderPool,
blockSize,
blOpts,
)
if seriesBlocks != nil && seriesBlocks.Len() > 0 {
shardResult.AddSeries(val.id, val.tags, seriesBlocks)
}
numShardEmptyErrs += numSeriesEmptyErrs
numErrs += numSeriesErrs
}
}
allShardResultSeries := shardResult.AllSeries()
for _, val := range allSnapshotSeries.Iter() {
id := val.Key()
blocks := val.Value()
if allShardResultSeries.Contains(id) {
// Already merged so we know the ID and tags from the snapshot
// won't be used and can be closed. We can't close the blocks
// though because we may have loaded some of the blocks into
// the shard result and we don't want to close them.
id.Finalize()
blocks.Tags.Finalize()
continue
}
shardResult.AddSeries(id, blocks.Tags, blocks.Blocks)
}
return shardResult, numShardEmptyErrs, numErrs
}
func (s *commitLogSource) mergeSeries(
nsCtx namespace.Context,
snapshotData result.DatabaseSeriesBlocks,
unmergedCommitlogBlocks metadataAndEncodersByTime,
blocksPool block.DatabaseBlockPool,
multiReaderIteratorPool encoding.MultiReaderIteratorPool,
segmentReaderPool xio.SegmentReaderPool,
encoderPool encoding.EncoderPool,
blockSize time.Duration,
blopts block.Options,
) (block.DatabaseSeriesBlocks, int, int) {
var seriesBlocks block.DatabaseSeriesBlocks
var numEmptyErrs int