-
Notifications
You must be signed in to change notification settings - Fork 458
/
Copy pathnamespace.go
1244 lines (1104 loc) · 37.2 KB
/
namespace.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 storage
import (
"errors"
"fmt"
"math"
"runtime"
"sync"
"time"
"github.com/m3db/m3/src/dbnode/clock"
"github.com/m3db/m3/src/dbnode/persist"
"github.com/m3db/m3/src/dbnode/persist/fs"
"github.com/m3db/m3/src/dbnode/sharding"
"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/namespace"
"github.com/m3db/m3/src/dbnode/storage/series"
"github.com/m3db/m3/src/dbnode/ts"
"github.com/m3db/m3/src/dbnode/x/xio"
"github.com/m3db/m3x/context"
xerrors "github.com/m3db/m3x/errors"
"github.com/m3db/m3x/ident"
"github.com/m3db/m3x/instrument"
xlog "github.com/m3db/m3x/log"
xsync "github.com/m3db/m3x/sync"
xtime "github.com/m3db/m3x/time"
"github.com/uber-go/tally"
)
var (
errNamespaceAlreadyClosed = errors.New("namespace already closed")
errNamespaceIndexingDisabled = errors.New("namespace indexing is disabled")
)
type commitLogWriter interface {
Write(
ctx context.Context,
series ts.Series,
datapoint ts.Datapoint,
unit xtime.Unit,
annotation ts.Annotation,
) error
}
type commitLogWriterFn func(
ctx context.Context,
series ts.Series,
datapoint ts.Datapoint,
unit xtime.Unit,
annotation ts.Annotation,
) error
func (fn commitLogWriterFn) Write(
ctx context.Context,
series ts.Series,
datapoint ts.Datapoint,
unit xtime.Unit,
annotation ts.Annotation,
) error {
return fn(ctx, series, datapoint, unit, annotation)
}
var commitLogWriteNoOp = commitLogWriter(commitLogWriterFn(func(
ctx context.Context,
series ts.Series,
datapoint ts.Datapoint,
unit xtime.Unit,
annotation ts.Annotation,
) error {
return nil
}))
type dbNamespace struct {
sync.RWMutex
closed bool
shutdownCh chan struct{}
id ident.ID
shardSet sharding.ShardSet
blockRetriever block.DatabaseBlockRetriever
namespaceReaderMgr databaseNamespaceReaderManager
opts Options
metadata namespace.Metadata
nopts namespace.Options
seriesOpts series.Options
nowFn clock.NowFn
snapshotFilesFn snapshotFilesFn
log xlog.Logger
bootstrapState BootstrapState
// Contains an entry to all shards for fast shard lookup, an
// entry will be nil when this shard does not belong to current database
shards []databaseShard
increasingIndex increasingIndex
commitLogWriter commitLogWriter
reverseIndex namespaceIndex
tickWorkers xsync.WorkerPool
tickWorkersConcurrency int
statsLastTick databaseNamespaceStatsLastTick
metrics databaseNamespaceMetrics
}
type databaseNamespaceStatsLastTick struct {
sync.RWMutex
activeSeries int64
activeBlocks int64
index databaseNamespaceIndexStatsLastTick
}
type databaseNamespaceIndexStatsLastTick struct {
numDocs int64
numBlocks int64
numSegments int64
}
type databaseNamespaceMetrics struct {
bootstrap instrument.MethodMetrics
flush instrument.MethodMetrics
flushIndex instrument.MethodMetrics
snapshot instrument.MethodMetrics
write instrument.MethodMetrics
writeTagged instrument.MethodMetrics
read instrument.MethodMetrics
fetchBlocks instrument.MethodMetrics
fetchBlocksMetadata instrument.MethodMetrics
queryIDs instrument.MethodMetrics
unfulfilled tally.Counter
bootstrapStart tally.Counter
bootstrapEnd tally.Counter
shards databaseNamespaceShardMetrics
tick databaseNamespaceTickMetrics
status databaseNamespaceStatusMetrics
}
type databaseNamespaceShardMetrics struct {
add tally.Counter
close tally.Counter
closeErrors tally.Counter
}
type databaseNamespaceTickMetrics struct {
activeSeries tally.Gauge
expiredSeries tally.Counter
activeBlocks tally.Gauge
openBlocks tally.Gauge
wiredBlocks tally.Gauge
unwiredBlocks tally.Gauge
pendingMergeBlocks tally.Gauge
madeUnwiredBlocks tally.Counter
madeExpiredBlocks tally.Counter
mergedOutOfOrderBlocks tally.Counter
errors tally.Counter
index databaseNamespaceIndexTickMetrics
}
type databaseNamespaceIndexTickMetrics struct {
numBlocks tally.Gauge
numDocs tally.Gauge
numSegments tally.Gauge
numBlocksSealed tally.Counter
numBlocksEvicted tally.Counter
}
// databaseNamespaceStatusMetrics are metrics emitted at a fixed interval
// so that summing the value of gauges across hosts when graphed summarizing
// values at the same fixed intervals can show meaningful results (vs variably
// emitted values that can be aggregated across hosts to see a snapshot).
type databaseNamespaceStatusMetrics struct {
activeSeries tally.Gauge
activeBlocks tally.Gauge
index databaseNamespaceIndexStatusMetrics
}
type databaseNamespaceIndexStatusMetrics struct {
numDocs tally.Gauge
numBlocks tally.Gauge
numSegments tally.Gauge
}
func newDatabaseNamespaceMetrics(scope tally.Scope, samplingRate float64) databaseNamespaceMetrics {
const (
// NB: tally.Timer when backed by a Prometheus Summary type is *very* expensive
// for high frequency measurements. Overriding sampling rate for writes to avoid this issue.
// TODO: make tally.Timers default to Prom Histograms instead of Summary. And update the dashboard
// to reflect this.
overrideWriteSamplingRate = 0.01
)
shardsScope := scope.SubScope("dbnamespace").SubScope("shards")
tickScope := scope.SubScope("tick")
indexTickScope := tickScope.SubScope("index")
statusScope := scope.SubScope("status")
indexStatusScope := statusScope.SubScope("index")
return databaseNamespaceMetrics{
bootstrap: instrument.NewMethodMetrics(scope, "bootstrap", samplingRate),
flush: instrument.NewMethodMetrics(scope, "flush", samplingRate),
flushIndex: instrument.NewMethodMetrics(scope, "flushIndex", samplingRate),
snapshot: instrument.NewMethodMetrics(scope, "snapshot", samplingRate),
write: instrument.NewMethodMetrics(scope, "write", overrideWriteSamplingRate),
writeTagged: instrument.NewMethodMetrics(scope, "write-tagged", overrideWriteSamplingRate),
read: instrument.NewMethodMetrics(scope, "read", samplingRate),
fetchBlocks: instrument.NewMethodMetrics(scope, "fetchBlocks", samplingRate),
fetchBlocksMetadata: instrument.NewMethodMetrics(scope, "fetchBlocksMetadata", samplingRate),
queryIDs: instrument.NewMethodMetrics(scope, "queryIDs", samplingRate),
unfulfilled: scope.Counter("bootstrap.unfulfilled"),
bootstrapStart: scope.Counter("bootstrap.start"),
bootstrapEnd: scope.Counter("bootstrap.end"),
shards: databaseNamespaceShardMetrics{
add: shardsScope.Counter("add"),
close: shardsScope.Counter("close"),
closeErrors: shardsScope.Counter("close-errors"),
},
tick: databaseNamespaceTickMetrics{
activeSeries: tickScope.Gauge("active-series"),
expiredSeries: tickScope.Counter("expired-series"),
activeBlocks: tickScope.Gauge("active-blocks"),
openBlocks: tickScope.Gauge("open-blocks"),
wiredBlocks: tickScope.Gauge("wired-blocks"),
unwiredBlocks: tickScope.Gauge("unwired-blocks"),
pendingMergeBlocks: tickScope.Gauge("pending-merge-blocks"),
madeUnwiredBlocks: tickScope.Counter("made-unwired-blocks"),
madeExpiredBlocks: tickScope.Counter("made-expired-blocks"),
mergedOutOfOrderBlocks: tickScope.Counter("merged-out-of-order-blocks"),
errors: tickScope.Counter("errors"),
index: databaseNamespaceIndexTickMetrics{
numDocs: indexTickScope.Gauge("num-docs"),
numBlocks: indexTickScope.Gauge("num-blocks"),
numSegments: indexTickScope.Gauge("num-segments"),
numBlocksSealed: indexTickScope.Counter("num-blocks-sealed"),
numBlocksEvicted: indexTickScope.Counter("num-blocks-evicted"),
},
},
status: databaseNamespaceStatusMetrics{
activeSeries: statusScope.Gauge("active-series"),
activeBlocks: statusScope.Gauge("active-blocks"),
index: databaseNamespaceIndexStatusMetrics{
numDocs: indexStatusScope.Gauge("num-docs"),
numBlocks: indexStatusScope.Gauge("num-blocks"),
numSegments: indexStatusScope.Gauge("num-segments"),
},
},
}
}
func newDatabaseNamespace(
metadata namespace.Metadata,
shardSet sharding.ShardSet,
blockRetriever block.DatabaseBlockRetriever,
increasingIndex increasingIndex,
commitLogWriter commitLogWriter,
opts Options,
) (databaseNamespace, error) {
var (
nopts = metadata.Options()
id = metadata.ID()
)
if !nopts.WritesToCommitLog() {
commitLogWriter = commitLogWriteNoOp
}
iops := opts.InstrumentOptions()
logger := iops.Logger().WithFields(xlog.NewField("namespace", id.String()))
iops = iops.SetLogger(logger)
opts = opts.SetInstrumentOptions(iops)
scope := iops.MetricsScope().SubScope("database").
Tagged(map[string]string{
"namespace": id.String(),
})
tickWorkersConcurrency := int(math.Max(1, float64(runtime.NumCPU())/8))
tickWorkers := xsync.NewWorkerPool(tickWorkersConcurrency)
tickWorkers.Init()
seriesOpts := NewSeriesOptionsFromOptions(opts, nopts.RetentionOptions()).
SetStats(series.NewStats(scope))
if err := seriesOpts.Validate(); err != nil {
return nil, fmt.Errorf(
"unable to create namespace %v, invalid series options: %v",
metadata.ID().String(), err)
}
var (
index namespaceIndex
err error
)
if metadata.Options().IndexOptions().Enabled() {
index, err = newNamespaceIndex(metadata, opts)
if err != nil {
return nil, err
}
}
n := &dbNamespace{
id: id,
shutdownCh: make(chan struct{}),
shardSet: shardSet,
blockRetriever: blockRetriever,
namespaceReaderMgr: newNamespaceReaderManager(metadata, scope, opts),
opts: opts,
metadata: metadata,
nopts: nopts,
seriesOpts: seriesOpts,
nowFn: opts.ClockOptions().NowFn(),
snapshotFilesFn: fs.SnapshotFiles,
log: logger,
increasingIndex: increasingIndex,
commitLogWriter: commitLogWriter,
reverseIndex: index,
tickWorkers: tickWorkers,
tickWorkersConcurrency: tickWorkersConcurrency,
metrics: newDatabaseNamespaceMetrics(scope, iops.MetricsSamplingRate()),
}
n.initShards(nopts.BootstrapEnabled())
go n.reportStatusLoop()
return n, nil
}
func (n *dbNamespace) reportStatusLoop() {
reportInterval := n.opts.InstrumentOptions().ReportInterval()
ticker := time.NewTicker(reportInterval)
defer ticker.Stop()
for {
select {
case <-n.shutdownCh:
return
case <-ticker.C:
n.statsLastTick.RLock()
n.metrics.status.activeSeries.Update(float64(n.statsLastTick.activeSeries))
n.metrics.status.activeBlocks.Update(float64(n.statsLastTick.activeBlocks))
n.metrics.status.index.numDocs.Update(float64(n.statsLastTick.index.numDocs))
n.metrics.status.index.numBlocks.Update(float64(n.statsLastTick.index.numBlocks))
n.metrics.status.index.numSegments.Update(float64(n.statsLastTick.index.numSegments))
n.statsLastTick.RUnlock()
}
}
}
func (n *dbNamespace) Options() namespace.Options {
return n.nopts
}
func (n *dbNamespace) ID() ident.ID {
return n.id
}
func (n *dbNamespace) NumSeries() int64 {
var count int64
for _, shard := range n.GetOwnedShards() {
count += shard.NumSeries()
}
return count
}
func (n *dbNamespace) Shards() []Shard {
n.RLock()
shards := n.shardSet.AllIDs()
databaseShards := make([]Shard, len(shards))
for i, shard := range shards {
databaseShards[i] = n.shards[shard]
}
n.RUnlock()
return databaseShards
}
func (n *dbNamespace) AssignShardSet(shardSet sharding.ShardSet) {
var (
incoming = make(map[uint32]struct{}, len(shardSet.All()))
existing []databaseShard
closing []databaseShard
)
for _, shard := range shardSet.AllIDs() {
incoming[shard] = struct{}{}
}
n.Lock()
existing = n.shards
for _, shard := range existing {
if shard == nil {
continue
}
if _, ok := incoming[shard.ID()]; !ok {
closing = append(closing, shard)
}
}
n.shardSet = shardSet
n.shards = make([]databaseShard, n.shardSet.Max()+1)
for _, shard := range n.shardSet.AllIDs() {
if int(shard) < len(existing) && existing[shard] != nil {
n.shards[shard] = existing[shard]
} else {
bootstrapEnabled := n.nopts.BootstrapEnabled()
n.shards[shard] = newDatabaseShard(n.metadata, shard, n.blockRetriever,
n.namespaceReaderMgr, n.increasingIndex, n.reverseIndex,
bootstrapEnabled, n.opts, n.seriesOpts)
n.metrics.shards.add.Inc(1)
}
}
n.Unlock()
n.closeShards(closing, false)
}
func (n *dbNamespace) closeShards(shards []databaseShard, blockUntilClosed bool) {
var wg sync.WaitGroup
// NB(r): There is a shard close deadline that controls how fast each
// shard closes set in the options. To make sure this is the single
// point of control for determining how impactful closing shards may
// be to performance, we let this be the single gate and simply spin
// up a goroutine per shard that we need to close and rely on the self
// throttling of each shard as determined by the close shard deadline to
// gate the impact.
closeFn := func(shard databaseShard) {
defer wg.Done()
if err := shard.Close(); err != nil {
n.log.
WithFields(xlog.NewField("shard", shard.ID())).
Errorf("error occurred closing shard: %v", err)
n.metrics.shards.closeErrors.Inc(1)
} else {
n.metrics.shards.close.Inc(1)
}
}
wg.Add(len(shards))
for _, shard := range shards {
dbShard := shard
if dbShard == nil {
continue
}
go closeFn(dbShard)
}
if blockUntilClosed {
wg.Wait()
}
}
func (n *dbNamespace) Tick(c context.Cancellable, tickStart time.Time) error {
// Allow the reader cache to tick
n.namespaceReaderMgr.tick()
// Fetch the owned shards
shards := n.GetOwnedShards()
if len(shards) == 0 {
return nil
}
// Tick through the shards at a capped level of concurrency
var (
r tickResult
multiErr xerrors.MultiError
l sync.Mutex
wg sync.WaitGroup
)
for _, shard := range shards {
shard := shard
wg.Add(1)
n.tickWorkers.Go(func() {
defer wg.Done()
if c.IsCancelled() {
return
}
shardResult, err := shard.Tick(c, tickStart)
l.Lock()
r = r.merge(shardResult)
multiErr = multiErr.Add(err)
l.Unlock()
})
}
wg.Wait()
// Tick namespaceIndex if it exists
var (
indexTickResults namespaceIndexTickResult
err error
)
if idx := n.reverseIndex; idx != nil {
indexTickResults, err = idx.Tick(c, tickStart)
if err != nil {
multiErr = multiErr.Add(err)
}
}
// NB: we early terminate here to ensure we are not reporting metrics
// based on in-accurate/partial tick results.
if err := multiErr.FinalError(); err != nil || c.IsCancelled() {
return err
}
n.statsLastTick.Lock()
n.statsLastTick.activeSeries = int64(r.activeSeries)
n.statsLastTick.activeBlocks = int64(r.activeBlocks)
n.statsLastTick.index = databaseNamespaceIndexStatsLastTick{
numDocs: indexTickResults.NumTotalDocs,
numBlocks: indexTickResults.NumBlocks,
numSegments: indexTickResults.NumSegments,
}
n.statsLastTick.Unlock()
n.metrics.tick.activeSeries.Update(float64(r.activeSeries))
n.metrics.tick.expiredSeries.Inc(int64(r.expiredSeries))
n.metrics.tick.activeBlocks.Update(float64(r.activeBlocks))
n.metrics.tick.openBlocks.Update(float64(r.openBlocks))
n.metrics.tick.wiredBlocks.Update(float64(r.wiredBlocks))
n.metrics.tick.unwiredBlocks.Update(float64(r.unwiredBlocks))
n.metrics.tick.pendingMergeBlocks.Update(float64(r.pendingMergeBlocks))
n.metrics.tick.madeExpiredBlocks.Inc(int64(r.madeExpiredBlocks))
n.metrics.tick.madeUnwiredBlocks.Inc(int64(r.madeUnwiredBlocks))
n.metrics.tick.mergedOutOfOrderBlocks.Inc(int64(r.mergedOutOfOrderBlocks))
n.metrics.tick.index.numDocs.Update(float64(indexTickResults.NumTotalDocs))
n.metrics.tick.index.numBlocks.Update(float64(indexTickResults.NumBlocks))
n.metrics.tick.index.numSegments.Update(float64(indexTickResults.NumSegments))
n.metrics.tick.index.numBlocksEvicted.Inc(indexTickResults.NumBlocksEvicted)
n.metrics.tick.index.numBlocksSealed.Inc(indexTickResults.NumBlocksSealed)
n.metrics.tick.errors.Inc(int64(r.errors))
return nil
}
func (n *dbNamespace) Write(
ctx context.Context,
id ident.ID,
timestamp time.Time,
value float64,
unit xtime.Unit,
annotation []byte,
) (ts.Series, error) {
callStart := n.nowFn()
shard, err := n.shardFor(id)
if err != nil {
n.metrics.write.ReportError(n.nowFn().Sub(callStart))
return ts.Series{}, err
}
series, err := shard.Write(ctx, id, timestamp, value, unit, annotation)
n.metrics.write.ReportSuccessOrError(err, n.nowFn().Sub(callStart))
return series, err
}
func (n *dbNamespace) WriteTagged(
ctx context.Context,
id ident.ID,
tags ident.TagIterator,
timestamp time.Time,
value float64,
unit xtime.Unit,
annotation []byte,
) (ts.Series, error) {
callStart := n.nowFn()
if n.reverseIndex == nil { // only happens if indexing is enabled.
n.metrics.writeTagged.ReportError(n.nowFn().Sub(callStart))
return ts.Series{}, errNamespaceIndexingDisabled
}
shard, err := n.shardFor(id)
if err != nil {
n.metrics.writeTagged.ReportError(n.nowFn().Sub(callStart))
return ts.Series{}, err
}
series, err := shard.WriteTagged(ctx, id, tags, timestamp, value, unit, annotation)
n.metrics.writeTagged.ReportSuccessOrError(err, n.nowFn().Sub(callStart))
return series, err
}
func (n *dbNamespace) QueryIDs(
ctx context.Context,
query index.Query,
opts index.QueryOptions,
) (index.QueryResults, error) {
callStart := n.nowFn()
if n.reverseIndex == nil { // only happens if indexing is enabled.
n.metrics.queryIDs.ReportError(n.nowFn().Sub(callStart))
return index.QueryResults{}, errNamespaceIndexingDisabled
}
if n.reverseIndex.BootstrapsDone() < 1 {
// Similar to reading shard data, return not bootstrapped
n.metrics.queryIDs.ReportError(n.nowFn().Sub(callStart))
return index.QueryResults{}, xerrors.NewRetryableError(errIndexNotBootstrappedToRead)
}
res, err := n.reverseIndex.Query(ctx, query, opts)
n.metrics.queryIDs.ReportSuccessOrError(err, n.nowFn().Sub(callStart))
return res, err
}
func (n *dbNamespace) ReadEncoded(
ctx context.Context,
id ident.ID,
start, end time.Time,
) ([][]xio.BlockReader, error) {
callStart := n.nowFn()
shard, err := n.readableShardFor(id)
if err != nil {
n.metrics.read.ReportError(n.nowFn().Sub(callStart))
return nil, err
}
res, err := shard.ReadEncoded(ctx, id, start, end)
n.metrics.read.ReportSuccessOrError(err, n.nowFn().Sub(callStart))
return res, err
}
func (n *dbNamespace) FetchBlocks(
ctx context.Context,
shardID uint32,
id ident.ID,
starts []time.Time,
) ([]block.FetchBlockResult, error) {
callStart := n.nowFn()
shard, err := n.readableShardAt(shardID)
if err != nil {
n.metrics.fetchBlocks.ReportError(n.nowFn().Sub(callStart))
return nil, err
}
res, err := shard.FetchBlocks(ctx, id, starts)
n.metrics.fetchBlocks.ReportSuccessOrError(err, n.nowFn().Sub(callStart))
return res, err
}
func (n *dbNamespace) FetchBlocksMetadataV2(
ctx context.Context,
shardID uint32,
start, end time.Time,
limit int64,
pageToken PageToken,
opts block.FetchBlocksMetadataOptions,
) (block.FetchBlocksMetadataResults, PageToken, error) {
callStart := n.nowFn()
shard, err := n.readableShardAt(shardID)
if err != nil {
n.metrics.fetchBlocksMetadata.ReportError(n.nowFn().Sub(callStart))
return nil, nil, err
}
res, nextPageToken, err := shard.FetchBlocksMetadataV2(ctx, start, end, limit,
pageToken, opts)
n.metrics.fetchBlocksMetadata.ReportSuccessOrError(err, n.nowFn().Sub(callStart))
return res, nextPageToken, err
}
func (n *dbNamespace) Bootstrap(start time.Time, process bootstrap.Process) error {
callStart := n.nowFn()
n.Lock()
if n.bootstrapState == Bootstrapping {
n.Unlock()
n.metrics.bootstrap.ReportError(n.nowFn().Sub(callStart))
return errNamespaceIsBootstrapping
}
n.bootstrapState = Bootstrapping
n.Unlock()
n.metrics.bootstrapStart.Inc(1)
success := false
defer func() {
n.Lock()
if success {
n.bootstrapState = Bootstrapped
} else {
n.bootstrapState = BootstrapNotStarted
}
n.Unlock()
n.metrics.bootstrapEnd.Inc(1)
}()
if !n.nopts.BootstrapEnabled() {
success = true
n.metrics.bootstrap.ReportSuccess(n.nowFn().Sub(callStart))
return nil
}
var (
owned = n.GetOwnedShards()
shards = make([]databaseShard, 0, len(owned))
)
for _, shard := range owned {
if !shard.IsBootstrapped() {
shards = append(shards, shard)
}
}
if len(shards) == 0 {
success = true
n.metrics.bootstrap.ReportSuccess(n.nowFn().Sub(callStart))
return nil
}
shardIDs := make([]uint32, len(shards))
for i, shard := range shards {
shardIDs[i] = shard.ID()
}
bootstrapResult, err := process.Run(start, n.metadata, shardIDs)
if err != nil {
n.log.Errorf("bootstrap for namespace %s aborted due to error: %v",
n.id.String(), err)
return err
}
n.metrics.bootstrap.Success.Inc(1)
// Bootstrap shards using at least half the CPUs available
workers := xsync.NewWorkerPool(int(math.Ceil(float64(runtime.NumCPU()) / 2)))
workers.Init()
numSeries := bootstrapResult.DataResult.ShardResults().NumSeries()
n.log.WithFields(
xlog.NewField("numShards", len(shards)),
xlog.NewField("numSeries", numSeries),
).Infof("bootstrap data fetched now initializing shards with series blocks")
var (
multiErr = xerrors.NewMultiError()
results = bootstrapResult.DataResult.ShardResults()
mutex sync.Mutex
wg sync.WaitGroup
)
for _, shard := range shards {
shard := shard
wg.Add(1)
workers.Go(func() {
var bootstrapped *result.Map
if shardResult, ok := results[shard.ID()]; ok {
bootstrapped = shardResult.AllSeries()
} else {
bootstrapped = result.NewMap(result.MapOptions{})
}
err := shard.Bootstrap(bootstrapped)
mutex.Lock()
multiErr = multiErr.Add(err)
mutex.Unlock()
wg.Done()
})
}
wg.Wait()
if n.reverseIndex != nil {
err := n.reverseIndex.Bootstrap(bootstrapResult.IndexResult.IndexResults())
multiErr = multiErr.Add(err)
}
markAnyUnfulfilled := func(label string, unfulfilled result.ShardTimeRanges) {
shardsUnfulfilled := int64(len(unfulfilled))
n.metrics.unfulfilled.Inc(shardsUnfulfilled)
if shardsUnfulfilled > 0 {
str := unfulfilled.SummaryString()
err := fmt.Errorf("bootstrap completed with unfulfilled ranges: %s", str)
multiErr = multiErr.Add(err)
n.log.WithFields(
xlog.NewField("namespace", n.id.String()),
xlog.NewField("bootstrap-type", label),
).Errorf(err.Error())
}
}
markAnyUnfulfilled("data", bootstrapResult.DataResult.Unfulfilled())
markAnyUnfulfilled("index", bootstrapResult.IndexResult.Unfulfilled())
err = multiErr.FinalError()
n.metrics.bootstrap.ReportSuccessOrError(err, n.nowFn().Sub(callStart))
success = err == nil
return err
}
func (n *dbNamespace) Flush(
blockStart time.Time,
shardBootstrapStatesAtTickStart ShardBootstrapStates,
flush persist.DataFlush,
) error {
// NB(rartoul): This value can be used for emitting metrics, but should not be used
// for business logic.
callStart := n.nowFn()
n.RLock()
if n.bootstrapState != Bootstrapped {
n.RUnlock()
n.metrics.flush.ReportError(n.nowFn().Sub(callStart))
return errNamespaceNotBootstrapped
}
n.RUnlock()
if !n.nopts.FlushEnabled() {
n.metrics.flush.ReportSuccess(n.nowFn().Sub(callStart))
return nil
}
// check if blockStart is aligned with the namespace's retention options
bs := n.nopts.RetentionOptions().BlockSize()
if t := blockStart.Truncate(bs); !blockStart.Equal(t) {
return fmt.Errorf("failed to flush at time %v, not aligned to blockSize", blockStart.String())
}
multiErr := xerrors.NewMultiError()
shards := n.GetOwnedShards()
for _, shard := range shards {
// This is different than calling shard.IsBootstrapped() because it was determined
// before the start of the tick that preceded this flush, meaning it can be reliably
// used to determine if all of the bootstrapped blocks have been merged / drained (ticked)
// and are ready to be flushed.
shardBootstrapStateBeforeTick, ok := shardBootstrapStatesAtTickStart[shard.ID()]
if !ok || shardBootstrapStateBeforeTick != Bootstrapped {
// We don't own this shard anymore (!ok) or the shard was not bootstrapped
// before the previous tick which means that we have no guarantee that all
// bootstrapped blocks have been rotated out of the series buffer buckets,
// so we wait until the next opportunity.
n.log.
WithFields(xlog.NewField("shard", shard.ID())).
WithFields(xlog.NewField("bootstrapStateBeforeTick", shardBootstrapStateBeforeTick)).
WithFields(xlog.NewField("bootstrapStateExists", ok)).
Debug("skipping snapshot due to shard bootstrap state before tick")
continue
}
// skip flushing if the shard has already flushed data for the `blockStart`
if s := shard.FlushState(blockStart); s.Status == fileOpSuccess {
continue
}
// NB(xichen): we still want to proceed if a shard fails to flush its data.
// Probably want to emit a counter here, but for now just log it.
if err := shard.Flush(blockStart, flush); err != nil {
detailedErr := fmt.Errorf("shard %d failed to flush data: %v",
shard.ID(), err)
multiErr = multiErr.Add(detailedErr)
}
}
res := multiErr.FinalError()
n.metrics.flush.ReportSuccessOrError(res, n.nowFn().Sub(callStart))
return res
}
func (n *dbNamespace) FlushIndex(
flush persist.IndexFlush,
) error {
callStart := n.nowFn()
n.RLock()
if n.bootstrapState != Bootstrapped {
n.RUnlock()
n.metrics.flushIndex.ReportError(n.nowFn().Sub(callStart))
return errNamespaceNotBootstrapped
}
n.RUnlock()
if !n.nopts.FlushEnabled() || !n.nopts.IndexOptions().Enabled() {
n.metrics.flush.ReportSuccess(n.nowFn().Sub(callStart))
return nil
}
err := n.reverseIndex.Flush(flush, n.GetOwnedShards())
n.metrics.flush.ReportSuccessOrError(err, n.nowFn().Sub(callStart))
return err
}
func (n *dbNamespace) Snapshot(
blockStart,
snapshotTime time.Time,
shardBootstrapStatesAtTickStart ShardBootstrapStates,
flush persist.DataFlush) error {
// NB(rartoul): This value can be used for emitting metrics, but should not be used
// for business logic.
callStart := n.nowFn()
n.RLock()
if n.bootstrapState != Bootstrapped {
n.RUnlock()
n.metrics.snapshot.ReportError(n.nowFn().Sub(callStart))
return errNamespaceNotBootstrapped
}
n.RUnlock()
if !n.nopts.SnapshotEnabled() {
n.metrics.snapshot.ReportSuccess(n.nowFn().Sub(callStart))
return nil
}
multiErr := xerrors.NewMultiError()
shards := n.GetOwnedShards()
for _, shard := range shards {
isSnapshotting, _ := shard.SnapshotState()
if isSnapshotting {
// Should never happen because snapshots should never overlap
// each other (controlled by loop in flush manager)
n.log.
WithFields(xlog.NewField("shard", shard.ID())).
Errorf("[invariant violated] tried to snapshot shard that is already snapshotting")
continue
}
// We don't need to perform this check for correctness, but we apply the same logic
// here as we do in the Flush() method so that we don't end up snapshotting a bunch
// of shards/blocks that would have been flushed after the next tick.
shardBootstrapStateBeforeTick, ok := shardBootstrapStatesAtTickStart[shard.ID()]
if !ok || shardBootstrapStateBeforeTick != Bootstrapped {
n.log.
WithFields(xlog.NewField("shard", shard.ID())).
WithFields(xlog.NewField("bootstrapStateBeforeTick", shardBootstrapStateBeforeTick)).
WithFields(xlog.NewField("bootstrapStateExists", ok)).
Debug("skipping snapshot due to shard bootstrap state before tick")
continue
}
err := shard.Snapshot(blockStart, snapshotTime, flush)
if err != nil {
detailedErr := fmt.Errorf("shard %d failed to snapshot: %v", shard.ID(), err)
multiErr = multiErr.Add(detailedErr)
// Continue with remaining shards
}
}
res := multiErr.FinalError()
n.metrics.snapshot.ReportSuccessOrError(res, n.nowFn().Sub(callStart))
return res
}
func (n *dbNamespace) NeedsFlush(
alignedInclusiveStart time.Time, alignedInclusiveEnd time.Time) bool {
// NB(r): Essentially if all are success, we don't need to flush, if any
// are failed with the minimum num failures less than max retries then
// we need to flush - otherwise if any in progress we can't flush and if
// any not started then we need to flush.
n.RLock()
defer n.RUnlock()
return n.needsFlushWithLock(alignedInclusiveStart, alignedInclusiveEnd)
}
func (n *dbNamespace) IsCapturedBySnapshot(
alignedInclusiveStart, alignedInclusiveEnd, capturedUpTo time.Time) (bool, error) {
var (
blockSize = n.nopts.RetentionOptions().BlockSize()
blockStarts = timesInRange(alignedInclusiveStart, alignedInclusiveEnd, blockSize)
filePathPrefix = n.opts.CommitLogOptions().FilesystemOptions().FilePathPrefix()
)
n.RLock()
defer n.RUnlock()
for _, shard := range n.shards {
if shard == nil {
continue
}
for _, blockStart := range blockStarts {
snapshotFiles, err := n.snapshotFilesFn(filePathPrefix, n.ID(), shard.ID())
if err != nil {
return false, err
}
snapshot, ok := snapshotFiles.LatestVolumeForBlock(blockStart)
if !ok {
// If a single shard is missing a snapshot for the blockStart then
// the entire namespace is not covered by snapshots up to time t.
return false, nil
}
snapshotTime, _, err := snapshot.SnapshotTimeAndID()
if err != nil {
return false, err
}
if snapshotTime.Before(capturedUpTo) {
// If a single shard's most recent snapshot has a snapshot time before
// capturedUpTo then we can't be sure that the entire namespace is
// covered by snapshots.
return false, nil
}
}