github.com/m3db/m3@v1.5.0/src/dbnode/storage/namespace.go (about) 1 // Copyright (c) 2016 Uber Technologies, Inc. 2 // 3 // Permission is hereby granted, free of charge, to any person obtaining a copy 4 // of this software and associated documentation files (the "Software"), to deal 5 // in the Software without restriction, including without limitation the rights 6 // to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 7 // copies of the Software, and to permit persons to whom the Software is 8 // furnished to do so, subject to the following conditions: 9 // 10 // The above copyright notice and this permission notice shall be included in 11 // all copies or substantial portions of the Software. 12 // 13 // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 14 // IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 15 // FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 16 // AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 17 // LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 18 // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN 19 // THE SOFTWARE. 20 21 package storage 22 23 import ( 24 "errors" 25 "fmt" 26 iofs "io/fs" 27 "math" 28 "runtime" 29 "sync" 30 "time" 31 32 opentracinglog "github.com/opentracing/opentracing-go/log" 33 "github.com/uber-go/tally" 34 "go.uber.org/zap" 35 36 "github.com/m3db/m3/src/dbnode/namespace" 37 "github.com/m3db/m3/src/dbnode/persist" 38 "github.com/m3db/m3/src/dbnode/persist/fs" 39 "github.com/m3db/m3/src/dbnode/sharding" 40 "github.com/m3db/m3/src/dbnode/storage/block" 41 "github.com/m3db/m3/src/dbnode/storage/bootstrap" 42 "github.com/m3db/m3/src/dbnode/storage/bootstrap/result" 43 "github.com/m3db/m3/src/dbnode/storage/index" 44 "github.com/m3db/m3/src/dbnode/storage/index/convert" 45 "github.com/m3db/m3/src/dbnode/storage/repair" 46 "github.com/m3db/m3/src/dbnode/storage/series" 47 "github.com/m3db/m3/src/dbnode/tracepoint" 48 "github.com/m3db/m3/src/dbnode/ts" 49 "github.com/m3db/m3/src/dbnode/ts/writes" 50 "github.com/m3db/m3/src/m3ninx/doc" 51 idxpersist "github.com/m3db/m3/src/m3ninx/persist" 52 "github.com/m3db/m3/src/x/clock" 53 "github.com/m3db/m3/src/x/context" 54 xerrors "github.com/m3db/m3/src/x/errors" 55 "github.com/m3db/m3/src/x/ident" 56 "github.com/m3db/m3/src/x/instrument" 57 xopentracing "github.com/m3db/m3/src/x/opentracing" 58 xresource "github.com/m3db/m3/src/x/resource" 59 xsync "github.com/m3db/m3/src/x/sync" 60 xtime "github.com/m3db/m3/src/x/time" 61 ) 62 63 var ( 64 errNamespaceAlreadyClosed = errors.New("namespace already closed") 65 errNamespaceIndexingDisabled = errors.New("namespace indexing is disabled") 66 errNamespaceReadOnly = errors.New("cannot write to a read only namespace") 67 ) 68 69 type commitLogWriter interface { 70 Write( 71 ctx context.Context, 72 series ts.Series, 73 datapoint ts.Datapoint, 74 unit xtime.Unit, 75 annotation ts.Annotation, 76 ) error 77 } 78 79 type commitLogWriterFn func( 80 ctx context.Context, 81 series ts.Series, 82 datapoint ts.Datapoint, 83 unit xtime.Unit, 84 annotation ts.Annotation, 85 ) error 86 87 func (fn commitLogWriterFn) Write( 88 ctx context.Context, 89 series ts.Series, 90 datapoint ts.Datapoint, 91 unit xtime.Unit, 92 annotation ts.Annotation, 93 ) error { 94 return fn(ctx, series, datapoint, unit, annotation) 95 } 96 97 var commitLogWriteNoOp = commitLogWriter(commitLogWriterFn(func( 98 ctx context.Context, 99 series ts.Series, 100 datapoint ts.Datapoint, 101 unit xtime.Unit, 102 annotation ts.Annotation, 103 ) error { 104 return nil 105 })) 106 107 type createEmptyWarmIndexIfNotExistsFn func(blockStart xtime.UnixNano) error 108 109 type dbNamespace struct { 110 sync.RWMutex 111 112 closed bool 113 readOnly bool 114 shutdownCh chan struct{} 115 id ident.ID 116 shardSet sharding.ShardSet 117 blockRetriever block.DatabaseBlockRetriever 118 namespaceReaderMgr databaseNamespaceReaderManager 119 opts Options 120 metadata namespace.Metadata 121 nopts namespace.Options 122 seriesOpts series.Options 123 nowFn clock.NowFn 124 snapshotFilesFn snapshotFilesFn 125 log *zap.Logger 126 bootstrapState BootstrapState 127 repairsAny bool 128 129 // schemaDescr caches the latest schema for the namespace. 130 // schemaDescr is updated whenever schema registry is updated. 131 schemaListener xresource.SimpleCloser 132 schemaDescr namespace.SchemaDescr 133 134 // Contains an entry to all shards for fast shard lookup, an 135 // entry will be nil when this shard does not belong to current database 136 shards []databaseShard 137 138 increasingIndex increasingIndex 139 commitLogWriter commitLogWriter 140 reverseIndex NamespaceIndex 141 142 createEmptyWarmIndexIfNotExistsFn createEmptyWarmIndexIfNotExistsFn 143 144 tickWorkers xsync.WorkerPool 145 tickWorkersConcurrency int 146 statsLastTick databaseNamespaceStatsLastTick 147 148 metrics databaseNamespaceMetrics 149 } 150 151 type databaseNamespaceStatsLastTick struct { 152 sync.RWMutex 153 activeSeries int64 154 activeBlocks int64 155 index databaseNamespaceIndexStatsLastTick 156 } 157 158 type databaseNamespaceIndexStatsLastTick struct { 159 numDocs int64 160 numBlocks int64 161 numSegments int64 162 } 163 164 type databaseNamespaceMetrics struct { 165 bootstrap instrument.MethodMetrics 166 flushWarmData instrument.MethodMetrics 167 flushColdData instrument.MethodMetrics 168 flushIndex instrument.MethodMetrics 169 snapshot instrument.MethodMetrics 170 write instrument.MethodMetrics 171 writeTagged instrument.MethodMetrics 172 read instrument.MethodMetrics 173 fetchBlocks instrument.MethodMetrics 174 fetchBlocksMetadata instrument.MethodMetrics 175 queryIDs instrument.MethodMetrics 176 aggregateQuery instrument.MethodMetrics 177 178 unfulfilled tally.Counter 179 bootstrapStart tally.Counter 180 bootstrapEnd tally.Counter 181 snapshotSeriesPersist tally.Counter 182 writesWithoutAnnotation tally.Counter 183 184 shards databaseNamespaceShardMetrics 185 tick databaseNamespaceTickMetrics 186 status databaseNamespaceStatusMetrics 187 188 repairDifferingPercent tally.Gauge 189 repairComparedBlocks tally.Counter 190 repairDifferingBlocks tally.Counter 191 repairMismatchBlocks tally.Counter 192 repairMissingBlocks tally.Counter 193 repairExtraBlocks tally.Counter 194 } 195 196 type databaseNamespaceShardMetrics struct { 197 add tally.Counter 198 close tally.Counter 199 closeErrors tally.Counter 200 } 201 202 type databaseNamespaceTickMetrics struct { 203 activeSeries tally.Gauge 204 expiredSeries tally.Counter 205 activeBlocks tally.Gauge 206 wiredBlocks tally.Gauge 207 unwiredBlocks tally.Gauge 208 pendingMergeBlocks tally.Gauge 209 madeUnwiredBlocks tally.Counter 210 madeExpiredBlocks tally.Counter 211 mergedOutOfOrderBlocks tally.Counter 212 errors tally.Counter 213 index databaseNamespaceIndexTickMetrics 214 evictedBuckets tally.Counter 215 } 216 217 type databaseNamespaceIndexTickMetrics struct { 218 numBlocks tally.Gauge 219 numDocs tally.Gauge 220 numSegments tally.Gauge 221 numBlocksSealed tally.Counter 222 numBlocksEvicted tally.Counter 223 } 224 225 // databaseNamespaceStatusMetrics are metrics emitted at a fixed interval 226 // so that summing the value of gauges across hosts when graphed summarizing 227 // values at the same fixed intervals can show meaningful results (vs variably 228 // emitted values that can be aggregated across hosts to see a snapshot). 229 type databaseNamespaceStatusMetrics struct { 230 activeSeries tally.Gauge 231 activeBlocks tally.Gauge 232 index databaseNamespaceIndexStatusMetrics 233 } 234 235 type databaseNamespaceIndexStatusMetrics struct { 236 numDocs tally.Gauge 237 numBlocks tally.Gauge 238 numSegments tally.Gauge 239 } 240 241 func newDatabaseNamespaceMetrics( 242 scope tally.Scope, 243 opts instrument.TimerOptions, 244 ) databaseNamespaceMetrics { 245 shardsScope := scope.SubScope("dbnamespace").SubScope("shards") 246 tickScope := scope.SubScope("tick") 247 indexTickScope := tickScope.SubScope("index") 248 statusScope := scope.SubScope("status") 249 indexStatusScope := statusScope.SubScope("index") 250 bootstrapScope := scope.SubScope("bootstrap") 251 snapshotScope := scope.SubScope("snapshot") 252 repairScope := scope.SubScope("repair") 253 return databaseNamespaceMetrics{ 254 bootstrap: instrument.NewMethodMetrics(scope, "bootstrap", opts), 255 flushWarmData: instrument.NewMethodMetrics(scope, "flushWarmData", opts), 256 flushColdData: instrument.NewMethodMetrics(scope, "flushColdData", opts), 257 flushIndex: instrument.NewMethodMetrics(scope, "flushIndex", opts), 258 snapshot: instrument.NewMethodMetrics(scope, "snapshot", opts), 259 write: instrument.NewMethodMetrics(scope, "write", opts), 260 writeTagged: instrument.NewMethodMetrics(scope, "write-tagged", opts), 261 read: instrument.NewMethodMetrics(scope, "read", opts), 262 fetchBlocks: instrument.NewMethodMetrics(scope, "fetchBlocks", opts), 263 fetchBlocksMetadata: instrument.NewMethodMetrics(scope, "fetchBlocksMetadata", opts), 264 queryIDs: instrument.NewMethodMetrics(scope, "queryIDs", opts), 265 aggregateQuery: instrument.NewMethodMetrics(scope, "aggregateQuery", opts), 266 267 unfulfilled: bootstrapScope.Counter("unfulfilled"), 268 bootstrapStart: bootstrapScope.Counter("start"), 269 bootstrapEnd: bootstrapScope.Counter("end"), 270 snapshotSeriesPersist: snapshotScope.Counter("series-persist"), 271 writesWithoutAnnotation: scope.Counter("writes-without-annotation"), 272 273 shards: databaseNamespaceShardMetrics{ 274 add: shardsScope.Counter("add"), 275 close: shardsScope.Counter("close"), 276 closeErrors: shardsScope.Counter("close-errors"), 277 }, 278 tick: databaseNamespaceTickMetrics{ 279 activeSeries: tickScope.Gauge("active-series"), 280 expiredSeries: tickScope.Counter("expired-series"), 281 activeBlocks: tickScope.Gauge("active-blocks"), 282 wiredBlocks: tickScope.Gauge("wired-blocks"), 283 unwiredBlocks: tickScope.Gauge("unwired-blocks"), 284 pendingMergeBlocks: tickScope.Gauge("pending-merge-blocks"), 285 madeUnwiredBlocks: tickScope.Counter("made-unwired-blocks"), 286 madeExpiredBlocks: tickScope.Counter("made-expired-blocks"), 287 mergedOutOfOrderBlocks: tickScope.Counter("merged-out-of-order-blocks"), 288 errors: tickScope.Counter("errors"), 289 index: databaseNamespaceIndexTickMetrics{ 290 numDocs: indexTickScope.Gauge("num-docs"), 291 numBlocks: indexTickScope.Gauge("num-blocks"), 292 numSegments: indexTickScope.Gauge("num-segments"), 293 numBlocksSealed: indexTickScope.Counter("num-blocks-sealed"), 294 numBlocksEvicted: indexTickScope.Counter("num-blocks-evicted"), 295 }, 296 evictedBuckets: tickScope.Counter("evicted-buckets"), 297 }, 298 status: databaseNamespaceStatusMetrics{ 299 activeSeries: statusScope.Gauge("active-series"), 300 activeBlocks: statusScope.Gauge("active-blocks"), 301 index: databaseNamespaceIndexStatusMetrics{ 302 numDocs: indexStatusScope.Gauge("num-docs"), 303 numBlocks: indexStatusScope.Gauge("num-blocks"), 304 numSegments: indexStatusScope.Gauge("num-segments"), 305 }, 306 }, 307 repairDifferingPercent: repairScope.Gauge("differing-percent"), 308 repairComparedBlocks: repairScope.Counter("compared-blocks"), 309 repairDifferingBlocks: repairScope.Counter("differing-blocks"), 310 repairMismatchBlocks: repairScope.Counter("mismatch-blocks"), 311 repairMissingBlocks: repairScope.Counter("missing-blocks"), 312 repairExtraBlocks: repairScope.Counter("extra-blocks"), 313 } 314 } 315 316 func newDatabaseNamespace( 317 metadata namespace.Metadata, 318 namespaceRuntimeOptsMgr namespace.RuntimeOptionsManager, 319 shardSet sharding.ShardSet, 320 blockRetriever block.DatabaseBlockRetriever, 321 increasingIndex increasingIndex, 322 commitLogWriter commitLogWriter, 323 opts Options, 324 ) (databaseNamespace, error) { 325 var ( 326 nopts = metadata.Options() 327 id = metadata.ID() 328 ) 329 if !nopts.WritesToCommitLog() { 330 commitLogWriter = commitLogWriteNoOp 331 } 332 333 iops := opts.InstrumentOptions() 334 logger := iops.Logger().With(zap.String("namespace", id.String())) 335 iops = iops. 336 SetLogger(logger). 337 SetMetricsScope(iops.MetricsScope().Tagged(map[string]string{ 338 "namespace": id.String(), 339 })) 340 opts = opts.SetInstrumentOptions(iops) 341 342 scope := iops.MetricsScope().SubScope("database") 343 344 tickWorkersConcurrency := int(math.Max(1, float64(runtime.GOMAXPROCS(0))/8)) 345 tickWorkers := xsync.NewWorkerPool(tickWorkersConcurrency) 346 tickWorkers.Init() 347 348 seriesOpts := NewSeriesOptionsFromOptions(opts, nopts.RetentionOptions()). 349 SetStats(series.NewStats(scope)). 350 SetColdWritesEnabled(nopts.ColdWritesEnabled()) 351 if err := seriesOpts.Validate(); err != nil { 352 return nil, fmt.Errorf( 353 "unable to create namespace %v, invalid series options: %v", 354 metadata.ID().String(), err) 355 } 356 357 var ( 358 index NamespaceIndex 359 err error 360 ) 361 if metadata.Options().IndexOptions().Enabled() { 362 index, err = newNamespaceIndex(metadata, namespaceRuntimeOptsMgr, 363 shardSet, opts) 364 if err != nil { 365 return nil, err 366 } 367 } 368 369 n := &dbNamespace{ 370 id: id, 371 shutdownCh: make(chan struct{}), 372 shardSet: shardSet, 373 blockRetriever: blockRetriever, 374 namespaceReaderMgr: newNamespaceReaderManager(metadata, scope, opts), 375 opts: opts, 376 metadata: metadata, 377 nopts: nopts, 378 seriesOpts: seriesOpts, 379 nowFn: opts.ClockOptions().NowFn(), 380 snapshotFilesFn: fs.SnapshotFiles, 381 log: logger, 382 increasingIndex: increasingIndex, 383 commitLogWriter: commitLogWriter, 384 reverseIndex: index, 385 tickWorkers: tickWorkers, 386 tickWorkersConcurrency: tickWorkersConcurrency, 387 metrics: newDatabaseNamespaceMetrics(scope, iops.TimerOptions()), 388 } 389 390 n.createEmptyWarmIndexIfNotExistsFn = n.createEmptyWarmIndexIfNotExists 391 392 sl, err := opts.SchemaRegistry().RegisterListener(id, n) 393 // Fail to create namespace is schema listener can not be registered successfully. 394 // If proto is disabled, err will always be nil. 395 if err != nil { 396 return nil, fmt.Errorf( 397 "unable to register schema listener for namespace %v, error: %v", 398 metadata.ID().String(), err) 399 } 400 n.schemaListener = sl 401 n.assignShardSet(shardSet, assignShardSetOptions{ 402 needsBootstrap: nopts.BootstrapEnabled(), 403 initialAssignment: true, 404 }) 405 go n.reportStatusLoop(opts.InstrumentOptions().ReportInterval()) 406 407 return n, nil 408 } 409 410 // SetSchemaHistory implements namespace.SchemaListener. 411 func (n *dbNamespace) SetSchemaHistory(value namespace.SchemaHistory) { 412 n.Lock() 413 defer n.Unlock() 414 415 schema, ok := value.GetLatest() 416 if !ok { 417 n.log.Error("can not update namespace schema to empty", zap.Stringer("namespace", n.ID())) 418 return 419 } 420 421 metadata, err := namespace.NewMetadata(n.ID(), n.nopts.SetSchemaHistory(value)) 422 if err != nil { 423 n.log.Error("can not update namespace metadata with empty schema history", zap.Stringer("namespace", n.ID()), zap.Error(err)) 424 return 425 } 426 427 n.schemaDescr = schema 428 n.metadata = metadata 429 } 430 431 func (n *dbNamespace) reportStatusLoop(reportInterval time.Duration) { 432 ticker := time.NewTicker(reportInterval) 433 defer ticker.Stop() 434 for { 435 select { 436 case <-n.shutdownCh: 437 return 438 case <-ticker.C: 439 n.statsLastTick.RLock() 440 n.metrics.status.activeSeries.Update(float64(n.statsLastTick.activeSeries)) 441 n.metrics.status.activeBlocks.Update(float64(n.statsLastTick.activeBlocks)) 442 n.metrics.status.index.numDocs.Update(float64(n.statsLastTick.index.numDocs)) 443 n.metrics.status.index.numBlocks.Update(float64(n.statsLastTick.index.numBlocks)) 444 n.metrics.status.index.numSegments.Update(float64(n.statsLastTick.index.numSegments)) 445 n.statsLastTick.RUnlock() 446 } 447 } 448 } 449 450 func (n *dbNamespace) Options() namespace.Options { 451 return n.nopts 452 } 453 454 func (n *dbNamespace) StorageOptions() Options { 455 return n.opts 456 } 457 458 func (n *dbNamespace) ID() ident.ID { 459 return n.id 460 } 461 462 func (n *dbNamespace) Metadata() namespace.Metadata { 463 // NB(r): metadata is updated in SetSchemaHistory so requires an RLock. 464 n.RLock() 465 result := n.metadata 466 n.RUnlock() 467 return result 468 } 469 470 func (n *dbNamespace) Schema() namespace.SchemaDescr { 471 n.RLock() 472 schema := n.schemaDescr 473 n.RUnlock() 474 return schema 475 } 476 477 func (n *dbNamespace) NumSeries() int64 { 478 var count int64 479 for _, shard := range n.OwnedShards() { 480 count += shard.NumSeries() 481 } 482 return count 483 } 484 485 func (n *dbNamespace) Shards() []Shard { 486 n.RLock() 487 shards := n.shardSet.AllIDs() 488 databaseShards := make([]Shard, len(shards)) 489 for i, shard := range shards { 490 databaseShards[i] = n.shards[shard] 491 } 492 n.RUnlock() 493 return databaseShards 494 } 495 496 func (n *dbNamespace) AssignShardSet(shardSet sharding.ShardSet) { 497 n.assignShardSet(shardSet, assignShardSetOptions{ 498 needsBootstrap: n.nopts.BootstrapEnabled(), 499 initialAssignment: false, 500 }) 501 } 502 503 type assignShardSetOptions struct { 504 needsBootstrap bool 505 initialAssignment bool 506 } 507 508 func (n *dbNamespace) assignShardSet( 509 shardSet sharding.ShardSet, 510 opts assignShardSetOptions, 511 ) { 512 var ( 513 incoming = make(map[uint32]struct{}, len(shardSet.All())) 514 createdShardIds []uint32 515 existing []databaseShard 516 closing []databaseShard 517 ) 518 for _, shard := range shardSet.AllIDs() { 519 incoming[shard] = struct{}{} 520 } 521 522 n.Lock() 523 metadata := n.metadata 524 existing = n.shards 525 for _, shard := range existing { 526 if shard == nil { 527 continue 528 } 529 if _, ok := incoming[shard.ID()]; !ok { 530 closing = append(closing, shard) 531 } 532 } 533 n.shardSet = shardSet 534 n.shards = make([]databaseShard, n.shardSet.Max()+1) 535 for _, shard := range n.shardSet.AllIDs() { 536 // We create shards if its an initial assignment or if its not an initial assignment 537 // and the shard doesn't already exist. 538 if !opts.initialAssignment && int(shard) < len(existing) && existing[shard] != nil { 539 n.shards[shard] = existing[shard] 540 continue 541 } 542 543 // Otherwise it's the initial assignment or there isn't an existing 544 // shard created for this shard ID. 545 n.shards[shard] = newDatabaseShard(metadata, shard, n.blockRetriever, 546 n.namespaceReaderMgr, n.increasingIndex, n.reverseIndex, 547 opts.needsBootstrap, n.opts, n.seriesOpts) 548 createdShardIds = append(createdShardIds, shard) 549 // NB(bodu): We only record shard add metrics for shards created in non 550 // initial assignments. 551 if !opts.initialAssignment { 552 n.metrics.shards.add.Inc(1) 553 } 554 } 555 556 if len(createdShardIds) > 0 { 557 n.log.Info("created new shards", 558 zap.Stringer("namespace", n.ID()), 559 zap.Uint32s("shards", createdShardIds), 560 zap.Bool("initialAssignment", opts.initialAssignment), 561 zap.Bool("needsBootstrap", opts.needsBootstrap)) 562 } 563 564 if idx := n.reverseIndex; idx != nil { 565 idx.AssignShardSet(shardSet) 566 } 567 if br := n.blockRetriever; br != nil { 568 br.AssignShardSet(shardSet) 569 } 570 if mgr := n.namespaceReaderMgr; mgr != nil { 571 mgr.assignShardSet(shardSet) 572 } 573 574 n.Unlock() 575 n.closeShards(closing, false) 576 } 577 578 func (n *dbNamespace) closeShards(shards []databaseShard, blockUntilClosed bool) { 579 var wg sync.WaitGroup 580 // NB(r): There is a shard close deadline that controls how fast each 581 // shard closes set in the options. To make sure this is the single 582 // point of control for determining how impactful closing shards may 583 // be to performance, we let this be the single gate and simply spin 584 // up a goroutine per shard that we need to close and rely on the self 585 // throttling of each shard as determined by the close shard deadline to 586 // gate the impact. 587 closeFn := func(shard databaseShard) { 588 defer wg.Done() 589 if err := shard.Close(); err != nil { 590 n.log. 591 With(zap.Uint32("shard", shard.ID())). 592 Error("error occurred closing shard", zap.Error(err)) 593 n.metrics.shards.closeErrors.Inc(1) 594 } else { 595 n.metrics.shards.close.Inc(1) 596 } 597 } 598 599 wg.Add(len(shards)) 600 for _, shard := range shards { 601 dbShard := shard 602 if dbShard == nil { 603 continue 604 } 605 go closeFn(dbShard) 606 } 607 608 if blockUntilClosed { 609 wg.Wait() 610 } 611 } 612 613 func (n *dbNamespace) Tick(c context.Cancellable, startTime xtime.UnixNano) error { 614 // Allow the reader cache to tick. 615 n.namespaceReaderMgr.tick() 616 617 // Fetch the owned shards. 618 shards := n.OwnedShards() 619 if len(shards) == 0 { 620 return nil 621 } 622 623 n.RLock() 624 nsCtx := n.nsContextWithRLock() 625 n.RUnlock() 626 627 // Tick through the shards at a capped level of concurrency. 628 var ( 629 r tickResult 630 multiErr xerrors.MultiError 631 l sync.Mutex 632 wg sync.WaitGroup 633 ) 634 for _, shard := range shards { 635 shard := shard 636 wg.Add(1) 637 n.tickWorkers.Go(func() { 638 defer wg.Done() 639 640 if c.IsCancelled() { 641 return 642 } 643 644 shardResult, err := shard.Tick(c, startTime, nsCtx) 645 646 l.Lock() 647 r = r.merge(shardResult) 648 multiErr = multiErr.Add(err) 649 l.Unlock() 650 }) 651 } 652 653 wg.Wait() 654 655 // Tick namespaceIndex if it exists. 656 var ( 657 indexTickResults namespaceIndexTickResult 658 err error 659 ) 660 if idx := n.reverseIndex; idx != nil { 661 indexTickResults, err = idx.Tick(c, startTime) 662 if err != nil { 663 multiErr = multiErr.Add(err) 664 } 665 } 666 667 // NB: we early terminate here to ensure we are not reporting metrics 668 // based on in-accurate/partial tick results. 669 if err := multiErr.FinalError(); err != nil || c.IsCancelled() { 670 return err 671 } 672 673 n.statsLastTick.Lock() 674 n.statsLastTick.activeSeries = int64(r.activeSeries) 675 n.statsLastTick.activeBlocks = int64(r.activeBlocks) 676 n.statsLastTick.index = databaseNamespaceIndexStatsLastTick{ 677 numDocs: indexTickResults.NumTotalDocs, 678 numBlocks: indexTickResults.NumBlocks, 679 numSegments: indexTickResults.NumSegments, 680 } 681 n.statsLastTick.Unlock() 682 683 n.metrics.tick.activeSeries.Update(float64(r.activeSeries)) 684 n.metrics.tick.expiredSeries.Inc(int64(r.expiredSeries)) 685 n.metrics.tick.activeBlocks.Update(float64(r.activeBlocks)) 686 n.metrics.tick.wiredBlocks.Update(float64(r.wiredBlocks)) 687 n.metrics.tick.unwiredBlocks.Update(float64(r.unwiredBlocks)) 688 n.metrics.tick.pendingMergeBlocks.Update(float64(r.pendingMergeBlocks)) 689 n.metrics.tick.madeExpiredBlocks.Inc(int64(r.madeExpiredBlocks)) 690 n.metrics.tick.madeUnwiredBlocks.Inc(int64(r.madeUnwiredBlocks)) 691 n.metrics.tick.mergedOutOfOrderBlocks.Inc(int64(r.mergedOutOfOrderBlocks)) 692 n.metrics.tick.evictedBuckets.Inc(int64(r.evictedBuckets)) 693 n.metrics.tick.index.numDocs.Update(float64(indexTickResults.NumTotalDocs)) 694 n.metrics.tick.index.numBlocks.Update(float64(indexTickResults.NumBlocks)) 695 n.metrics.tick.index.numSegments.Update(float64(indexTickResults.NumSegments)) 696 n.metrics.tick.index.numBlocksEvicted.Inc(indexTickResults.NumBlocksEvicted) 697 n.metrics.tick.index.numBlocksSealed.Inc(indexTickResults.NumBlocksSealed) 698 n.metrics.tick.errors.Inc(int64(r.errors)) 699 700 return nil 701 } 702 703 func (n *dbNamespace) Write( 704 ctx context.Context, 705 id ident.ID, 706 timestamp xtime.UnixNano, 707 value float64, 708 unit xtime.Unit, 709 annotation []byte, 710 ) (SeriesWrite, error) { 711 callStart := n.nowFn() 712 713 if n.ReadOnly() { 714 n.metrics.write.ReportError(n.nowFn().Sub(callStart)) 715 return SeriesWrite{}, errNamespaceReadOnly 716 } 717 718 shard, nsCtx, err := n.shardFor(id) 719 if err != nil { 720 n.metrics.write.ReportError(n.nowFn().Sub(callStart)) 721 return SeriesWrite{}, err 722 } 723 724 opts := series.WriteOptions{ 725 TruncateType: n.opts.TruncateType(), 726 SchemaDesc: nsCtx.Schema, 727 } 728 seriesWrite, err := shard.Write(ctx, id, timestamp, 729 value, unit, annotation, opts) 730 if err == nil && len(annotation) == 0 { 731 n.metrics.writesWithoutAnnotation.Inc(1) 732 } 733 n.metrics.write.ReportSuccessOrError(err, n.nowFn().Sub(callStart)) 734 return seriesWrite, err 735 } 736 737 func (n *dbNamespace) WriteTagged( 738 ctx context.Context, 739 id ident.ID, 740 tagResolver convert.TagMetadataResolver, 741 timestamp xtime.UnixNano, 742 value float64, 743 unit xtime.Unit, 744 annotation []byte, 745 ) (SeriesWrite, error) { 746 callStart := n.nowFn() 747 748 if n.ReadOnly() { 749 n.metrics.writeTagged.ReportError(n.nowFn().Sub(callStart)) 750 return SeriesWrite{}, errNamespaceReadOnly 751 } 752 753 if n.reverseIndex == nil { 754 n.metrics.writeTagged.ReportError(n.nowFn().Sub(callStart)) 755 return SeriesWrite{}, errNamespaceIndexingDisabled 756 } 757 758 shard, nsCtx, err := n.shardFor(id) 759 if err != nil { 760 n.metrics.writeTagged.ReportError(n.nowFn().Sub(callStart)) 761 return SeriesWrite{}, err 762 } 763 764 opts := series.WriteOptions{ 765 TruncateType: n.opts.TruncateType(), 766 SchemaDesc: nsCtx.Schema, 767 } 768 seriesWrite, err := shard.WriteTagged(ctx, id, tagResolver, timestamp, 769 value, unit, annotation, opts) 770 if err == nil && len(annotation) == 0 { 771 n.metrics.writesWithoutAnnotation.Inc(1) 772 } 773 n.metrics.writeTagged.ReportSuccessOrError(err, n.nowFn().Sub(callStart)) 774 return seriesWrite, err 775 } 776 777 func (n *dbNamespace) WritePendingIndexInserts( 778 pending []writes.PendingIndexInsert, 779 ) error { 780 if n.reverseIndex == nil { 781 return errNamespaceIndexingDisabled 782 } 783 return n.reverseIndex.WritePending(pending) 784 } 785 786 func (n *dbNamespace) SeriesRefResolver( 787 shardID uint32, 788 id ident.ID, 789 tags ident.TagIterator, 790 ) (bootstrap.SeriesRefResolver, bool, error) { 791 n.RLock() 792 shard, owned, err := n.shardAtWithRLock(shardID) 793 n.RUnlock() 794 if err != nil { 795 return nil, owned, err 796 } 797 resolver, err := shard.SeriesRefResolver(id, tags) 798 return resolver, true, err 799 } 800 801 func (n *dbNamespace) QueryIDs( 802 ctx context.Context, 803 query index.Query, 804 opts index.QueryOptions, 805 ) (index.QueryResult, error) { 806 ctx, sp, sampled := ctx.StartSampledTraceSpan(tracepoint.NSQueryIDs) 807 if sampled { 808 sp.LogFields( 809 opentracinglog.String("query", query.String()), 810 opentracinglog.String("namespace", n.ID().String()), 811 opentracinglog.Int("seriesLimit", opts.SeriesLimit), 812 opentracinglog.Int("docsLimit", opts.DocsLimit), 813 xopentracing.Time("start", opts.StartInclusive.ToTime()), 814 xopentracing.Time("end", opts.EndExclusive.ToTime()), 815 ) 816 } 817 defer sp.Finish() 818 819 callStart := n.nowFn() 820 if n.reverseIndex == nil { 821 n.metrics.queryIDs.ReportError(n.nowFn().Sub(callStart)) 822 err := errNamespaceIndexingDisabled 823 sp.LogFields(opentracinglog.Error(err)) 824 return index.QueryResult{}, err 825 } 826 827 if !n.reverseIndex.Bootstrapped() { 828 // Similar to reading shard data, return not bootstrapped 829 n.metrics.queryIDs.ReportError(n.nowFn().Sub(callStart)) 830 err := errIndexNotBootstrappedToRead 831 sp.LogFields(opentracinglog.Error(err)) 832 return index.QueryResult{}, 833 xerrors.NewRetryableError(err) 834 } 835 836 res, err := n.reverseIndex.Query(ctx, query, opts) 837 if err != nil { 838 sp.LogFields(opentracinglog.Error(err)) 839 } 840 n.metrics.queryIDs.ReportSuccessOrError(err, n.nowFn().Sub(callStart)) 841 return res, err 842 } 843 844 func (n *dbNamespace) AggregateQuery( 845 ctx context.Context, 846 query index.Query, 847 opts index.AggregationOptions, 848 ) (index.AggregateQueryResult, error) { 849 callStart := n.nowFn() 850 if n.reverseIndex == nil { 851 n.metrics.aggregateQuery.ReportError(n.nowFn().Sub(callStart)) 852 return index.AggregateQueryResult{}, errNamespaceIndexingDisabled 853 } 854 855 if !n.reverseIndex.Bootstrapped() { 856 // Similar to reading shard data, return not bootstrapped 857 n.metrics.aggregateQuery.ReportError(n.nowFn().Sub(callStart)) 858 return index.AggregateQueryResult{}, 859 xerrors.NewRetryableError(errIndexNotBootstrappedToRead) 860 } 861 862 res, err := n.reverseIndex.AggregateQuery(ctx, query, opts) 863 n.metrics.aggregateQuery.ReportSuccessOrError(err, n.nowFn().Sub(callStart)) 864 return res, err 865 } 866 867 func (n *dbNamespace) PrepareBootstrap(ctx context.Context) ([]databaseShard, error) { 868 ctx, span, sampled := ctx.StartSampledTraceSpan(tracepoint.NSPrepareBootstrap) 869 defer span.Finish() 870 871 if sampled { 872 span.LogFields(opentracinglog.String("namespace", n.id.String())) 873 } 874 875 var ( 876 wg sync.WaitGroup 877 multiErrLock sync.Mutex 878 multiErr xerrors.MultiError 879 shards = n.OwnedShards() 880 ) 881 for _, shard := range shards { 882 shard := shard 883 wg.Add(1) 884 go func() { 885 defer wg.Done() 886 887 err := shard.PrepareBootstrap(ctx) 888 if err != nil { 889 multiErrLock.Lock() 890 multiErr = multiErr.Add(err) 891 multiErrLock.Unlock() 892 } 893 }() 894 } 895 896 wg.Wait() 897 898 if err := multiErr.FinalError(); err != nil { 899 return nil, err 900 } 901 902 return shards, nil 903 } 904 905 func (n *dbNamespace) ReadEncoded( 906 ctx context.Context, 907 id ident.ID, 908 start, end xtime.UnixNano, 909 ) (series.BlockReaderIter, error) { 910 callStart := n.nowFn() 911 shard, nsCtx, err := n.readableShardFor(id) 912 if err != nil { 913 n.metrics.read.ReportError(n.nowFn().Sub(callStart)) 914 return nil, err 915 } 916 res, err := shard.ReadEncoded(ctx, id, start, end, nsCtx) 917 n.metrics.read.ReportSuccessOrError(err, n.nowFn().Sub(callStart)) 918 return res, err 919 } 920 921 func (n *dbNamespace) FetchBlocks( 922 ctx context.Context, 923 shardID uint32, 924 id ident.ID, 925 starts []xtime.UnixNano, 926 ) ([]block.FetchBlockResult, error) { 927 callStart := n.nowFn() 928 shard, nsCtx, err := n.ReadableShardAt(shardID) 929 if err != nil { 930 n.metrics.fetchBlocks.ReportError(n.nowFn().Sub(callStart)) 931 return nil, err 932 } 933 934 res, err := shard.FetchBlocks(ctx, id, starts, nsCtx) 935 n.metrics.fetchBlocks.ReportSuccessOrError(err, n.nowFn().Sub(callStart)) 936 return res, err 937 } 938 939 func (n *dbNamespace) FetchBlocksMetadataV2( 940 ctx context.Context, 941 shardID uint32, 942 start, end xtime.UnixNano, 943 limit int64, 944 pageToken PageToken, 945 opts block.FetchBlocksMetadataOptions, 946 ) (block.FetchBlocksMetadataResults, PageToken, error) { 947 callStart := n.nowFn() 948 shard, _, err := n.ReadableShardAt(shardID) 949 if err != nil { 950 n.metrics.fetchBlocksMetadata.ReportError(n.nowFn().Sub(callStart)) 951 return nil, nil, err 952 } 953 954 res, nextPageToken, err := shard.FetchBlocksMetadataV2(ctx, start, end, limit, 955 pageToken, opts) 956 n.metrics.fetchBlocksMetadata.ReportSuccessOrError(err, n.nowFn().Sub(callStart)) 957 return res, nextPageToken, err 958 } 959 960 func (n *dbNamespace) Bootstrap( 961 ctx context.Context, 962 bootstrapResult bootstrap.NamespaceResult, 963 ) error { 964 ctx, span, sampled := ctx.StartSampledTraceSpan(tracepoint.NSBootstrap) 965 defer span.Finish() 966 967 if sampled { 968 span.LogFields(opentracinglog.String("namespace", n.id.String())) 969 } 970 971 callStart := n.nowFn() 972 973 n.Lock() 974 if n.bootstrapState == Bootstrapping { 975 n.Unlock() 976 n.metrics.bootstrap.ReportError(n.nowFn().Sub(callStart)) 977 return errNamespaceIsBootstrapping 978 } 979 n.bootstrapState = Bootstrapping 980 nsCtx := n.nsContextWithRLock() 981 n.Unlock() 982 983 n.metrics.bootstrapStart.Inc(1) 984 985 success := false 986 defer func() { 987 n.Lock() 988 if success { 989 n.bootstrapState = Bootstrapped 990 } else { 991 n.bootstrapState = BootstrapNotStarted 992 } 993 n.Unlock() 994 n.metrics.bootstrapEnd.Inc(1) 995 }() 996 997 if !n.nopts.BootstrapEnabled() { 998 success = true 999 n.metrics.bootstrap.ReportSuccess(n.nowFn().Sub(callStart)) 1000 return nil 1001 } 1002 1003 // Bootstrap shards using at least half the CPUs available 1004 workers := xsync.NewWorkerPool(int(math.Ceil(float64(runtime.GOMAXPROCS(0)) / 2))) 1005 workers.Init() 1006 1007 var ( 1008 bootstrappedShards = bootstrapResult.Shards 1009 multiErr = xerrors.NewMultiError() 1010 mutex sync.Mutex 1011 wg sync.WaitGroup 1012 ) 1013 n.log.Info("bootstrap marking all shards as bootstrapped", 1014 zap.Stringer("namespace", n.id), 1015 zap.Int("numShards", len(bootstrappedShards))) 1016 for _, shard := range n.OwnedShards() { 1017 // Make sure it was bootstrapped during this bootstrap run. 1018 shardID := shard.ID() 1019 bootstrapped := false 1020 for _, elem := range bootstrappedShards { 1021 if elem == shardID { 1022 bootstrapped = true 1023 break 1024 } 1025 } 1026 if !bootstrapped { 1027 // NB(r): Not bootstrapped in this bootstrap run. 1028 n.log.Debug("skipping already bootstrapped shard", 1029 zap.Uint32("shard", shardID), 1030 zap.Stringer("namespace", n.id)) 1031 continue 1032 } 1033 1034 if shard.IsBootstrapped() { 1035 // No concurrent bootstraps, this is an invariant since 1036 // we only select bootstrapping the shard for a run if it's 1037 // not already bootstrapped. 1038 err := instrument.InvariantErrorf( 1039 "bootstrapper already bootstrapped shard: %d", shardID) 1040 mutex.Lock() 1041 multiErr = multiErr.Add(err) 1042 mutex.Unlock() 1043 continue 1044 } 1045 1046 // Check if there are unfulfilled ranges 1047 if ranges, ok := bootstrapResult.DataResult.Unfulfilled().Get(shardID); ok && !ranges.IsEmpty() { 1048 continue 1049 } else if n.reverseIndex != nil { 1050 if ranges, ok := bootstrapResult.IndexResult.Unfulfilled().Get(shardID); ok && !ranges.IsEmpty() { 1051 continue 1052 } 1053 } 1054 1055 wg.Add(1) 1056 shard := shard 1057 workers.Go(func() { 1058 err := shard.Bootstrap(ctx, nsCtx) 1059 1060 mutex.Lock() 1061 multiErr = multiErr.Add(err) 1062 mutex.Unlock() 1063 1064 wg.Done() 1065 }) 1066 } 1067 wg.Wait() 1068 1069 if n.reverseIndex != nil { 1070 indexResults := bootstrapResult.IndexResult.IndexResults() 1071 n.log.Info("bootstrap index with bootstrapped index segments", 1072 zap.Int("numIndexBlocks", len(indexResults))) 1073 err := n.reverseIndex.Bootstrap(indexResults) 1074 multiErr = multiErr.Add(err) 1075 } 1076 1077 markAnyUnfulfilled := func( 1078 bootstrapType string, 1079 unfulfilled result.ShardTimeRanges, 1080 ) error { 1081 shardsUnfulfilled := int64(unfulfilled.Len()) 1082 n.metrics.unfulfilled.Inc(shardsUnfulfilled) 1083 if shardsUnfulfilled == 0 { 1084 return nil 1085 } 1086 1087 errStr := unfulfilled.SummaryString() 1088 errFmt := "bootstrap completed with unfulfilled ranges" 1089 n.log.Error(errFmt, 1090 zap.Error(errors.New(errStr)), 1091 zap.String("namespace", n.id.String()), 1092 zap.String("bootstrapType", bootstrapType)) 1093 return fmt.Errorf("%s: %s", errFmt, errStr) 1094 } 1095 1096 r := bootstrapResult 1097 if err := markAnyUnfulfilled("data", r.DataResult.Unfulfilled()); err != nil { 1098 multiErr = multiErr.Add(err) 1099 } 1100 if n.reverseIndex != nil { 1101 if err := markAnyUnfulfilled("index", r.IndexResult.Unfulfilled()); err != nil { 1102 multiErr = multiErr.Add(err) 1103 } 1104 } 1105 1106 err := multiErr.FinalError() 1107 n.metrics.bootstrap.ReportSuccessOrError(err, n.nowFn().Sub(callStart)) 1108 1109 // NB(r): "success" is read by the defer above and depending on if true/false 1110 // will set the namespace status as bootstrapped or not. 1111 success = err == nil 1112 return err 1113 } 1114 1115 func (n *dbNamespace) WarmFlush( 1116 blockStart xtime.UnixNano, 1117 flushPersist persist.FlushPreparer, 1118 ) error { 1119 // NB(rartoul): This value can be used for emitting metrics, but should not be used 1120 // for business logic. 1121 callStart := n.nowFn() 1122 1123 n.RLock() 1124 if n.bootstrapState != Bootstrapped { 1125 n.RUnlock() 1126 n.metrics.flushWarmData.ReportError(n.nowFn().Sub(callStart)) 1127 return errNamespaceNotBootstrapped 1128 } 1129 nsCtx := n.nsContextWithRLock() 1130 n.RUnlock() 1131 1132 if n.ReadOnly() || !n.nopts.FlushEnabled() { 1133 n.metrics.flushWarmData.ReportSuccess(n.nowFn().Sub(callStart)) 1134 return nil 1135 } 1136 1137 // check if blockStart is aligned with the namespace's retention options 1138 bs := n.nopts.RetentionOptions().BlockSize() 1139 if t := blockStart.Truncate(bs); !blockStart.Equal(t) { 1140 return fmt.Errorf("failed to flush at time %v, not aligned to blockSize", blockStart.String()) 1141 } 1142 1143 multiErr := xerrors.NewMultiError() 1144 shards := n.OwnedShards() 1145 for _, shard := range shards { 1146 if !shard.IsBootstrapped() { 1147 n.log. 1148 With(zap.Uint32("shard", shard.ID())). 1149 Debug("skipping warm flush due to shard not bootstrapped yet") 1150 continue 1151 } 1152 1153 flushState, err := shard.FlushState(blockStart) 1154 if err != nil { 1155 return err 1156 } 1157 // skip flushing if the shard has already flushed data for the `blockStart` 1158 if flushState.WarmStatus.DataFlushed == fileOpSuccess { 1159 continue 1160 } 1161 1162 // NB(xichen): we still want to proceed if a shard fails to flush its data. 1163 // Probably want to emit a counter here, but for now just log it. 1164 if err := shard.WarmFlush(blockStart, flushPersist, nsCtx); err != nil { 1165 detailedErr := fmt.Errorf("shard %d failed to flush data: %v", 1166 shard.ID(), err) 1167 multiErr = multiErr.Add(detailedErr) 1168 } 1169 } 1170 1171 res := multiErr.FinalError() 1172 n.metrics.flushWarmData.ReportSuccessOrError(res, n.nowFn().Sub(callStart)) 1173 return res 1174 } 1175 1176 // idAndBlockStart is the composite key for the genny map used to keep track of 1177 // dirty series that need to be ColdFlushed. 1178 type idAndBlockStart struct { 1179 id []byte 1180 blockStart xtime.UnixNano 1181 } 1182 1183 type coldFlushReusableResources struct { 1184 // dirtySeries is a map from a composite key of <series ID, block start> 1185 // to an element in a list in the dirtySeriesToWrite map. This map is used 1186 // to quickly test whether a series is dirty for a particular block start. 1187 // 1188 // The composite key is deliberately made so that this map stays one level 1189 // deep, making it easier to share between shard loops, minimizing the need 1190 // for allocations. 1191 // 1192 // Having a reference to the element in the dirtySeriesToWrite list enables 1193 // efficient removal of the series that have been read and subsequent 1194 // iterating through remaining series to be read. 1195 dirtySeries *dirtySeriesMap 1196 // dirtySeriesToWrite is a map from block start to a list of dirty series 1197 // that have yet to be written to disk. 1198 dirtySeriesToWrite map[xtime.UnixNano]*idList 1199 // idElementPool is a pool of list elements to be used when constructing 1200 // new lists for the dirtySeriesToWrite map. 1201 idElementPool *idElementPool 1202 fsReader fs.DataFileSetReader 1203 } 1204 1205 func newColdFlushReusableResources(opts Options) coldFlushReusableResources { 1206 fsReader, err := fs.NewReader(opts.BytesPool(), opts.CommitLogOptions().FilesystemOptions()) 1207 if err != nil { 1208 return coldFlushReusableResources{} 1209 } 1210 1211 return coldFlushReusableResources{ 1212 dirtySeries: newDirtySeriesMap(), 1213 dirtySeriesToWrite: make(map[xtime.UnixNano]*idList), 1214 // TODO(juchan): set pool options. 1215 idElementPool: newIDElementPool(nil), 1216 fsReader: fsReader, 1217 } 1218 } 1219 1220 func (r *coldFlushReusableResources) reset() { 1221 for _, seriesList := range r.dirtySeriesToWrite { 1222 if seriesList != nil { 1223 seriesList.Reset() 1224 } 1225 // Don't delete the empty list from the map so that other shards don't 1226 // need to reinitialize the list for these blocks. 1227 } 1228 1229 r.dirtySeries.Reset() 1230 } 1231 1232 func (n *dbNamespace) ColdFlush(flushPersist persist.FlushPreparer) error { 1233 // NB(rartoul): This value can be used for emitting metrics, but should not be used 1234 // for business logic. 1235 callStart := n.nowFn() 1236 1237 n.RLock() 1238 if n.bootstrapState != Bootstrapped { 1239 n.RUnlock() 1240 n.metrics.flushColdData.ReportError(n.nowFn().Sub(callStart)) 1241 return errNamespaceNotBootstrapped 1242 } 1243 nsCtx := n.nsContextWithRLock() 1244 repairsAny := n.repairsAny 1245 n.RUnlock() 1246 1247 // If repair has run we still need cold flush regardless of whether cold writes is 1248 // enabled since repairs are dependent on the cold flushing logic. 1249 enabled := n.nopts.ColdWritesEnabled() || repairsAny 1250 if n.ReadOnly() || !enabled { 1251 n.metrics.flushColdData.ReportSuccess(n.nowFn().Sub(callStart)) 1252 return nil 1253 } 1254 1255 shards := n.OwnedShards() 1256 resources := newColdFlushReusableResources(n.opts) 1257 1258 // NB(bodu): The in-mem index will lag behind the TSDB in terms of new series writes. For a period of 1259 // time between when we rotate out the active cold mutable index segments (happens here) and when 1260 // we actually cold flush the data to disk we will be making writes to the newly active mutable seg. 1261 // This means that some series can live doubly in-mem and loaded from disk until the next cold flush 1262 // where they will be evicted from the in-mem index. 1263 var ( 1264 onColdFlushDone OnColdFlushDone 1265 err error 1266 ) 1267 if n.reverseIndex != nil { 1268 onColdFlushDone, err = n.reverseIndex.ColdFlush(shards) 1269 if err != nil { 1270 n.metrics.flushColdData.ReportError(n.nowFn().Sub(callStart)) 1271 return err 1272 } 1273 } 1274 1275 cfOpts := NewColdFlushNsOpts(true) 1276 onColdFlushNs, err := n.opts.OnColdFlush().ColdFlushNamespace(n, cfOpts) 1277 if err != nil { 1278 n.metrics.flushColdData.ReportError(n.nowFn().Sub(callStart)) 1279 return err 1280 } 1281 1282 // NB(bodu): Deferred shard cold flushes so that we can ensure that cold flush index data is 1283 // persisted before persisting TSDB data to ensure crash consistency. 1284 multiErr := xerrors.NewMultiError() 1285 shardColdFlushes := make([]ShardColdFlush, 0, len(shards)) 1286 for _, shard := range shards { 1287 if !shard.IsBootstrapped() { 1288 n.log. 1289 With(zap.Uint32("shard", shard.ID())). 1290 Debug("skipping cold flush due to shard not bootstrapped yet") 1291 continue 1292 } 1293 shardColdFlush, err := shard.ColdFlush(flushPersist, resources, nsCtx, onColdFlushNs) 1294 if err != nil { 1295 detailedErr := fmt.Errorf("shard %d failed to compact: %v", shard.ID(), err) 1296 multiErr = multiErr.Add(detailedErr) 1297 continue 1298 } 1299 shardColdFlushes = append(shardColdFlushes, shardColdFlush) 1300 } 1301 1302 // We go through this error checking process to allow for partially successful flushes. 1303 indexColdFlushError := onColdFlushNs.Done() 1304 if indexColdFlushError == nil && onColdFlushDone != nil { 1305 // Only evict rotated cold mutable index segments if the index cold flush was successful 1306 // or we will lose queryability of data that's still in mem. 1307 indexColdFlushError = onColdFlushDone() 1308 } 1309 if indexColdFlushError == nil { 1310 // NB(bodu): We only want to complete data cold flushes if the index cold flush 1311 // is successful. If index cold flush is successful, we want to attempt writing 1312 // of checkpoint files to complete the cold data flush lifecycle for successful shards. 1313 for _, shardColdFlush := range shardColdFlushes { 1314 multiErr = multiErr.Add(shardColdFlush.Done()) 1315 } 1316 } 1317 multiErr = multiErr.Add(indexColdFlushError) 1318 1319 res := multiErr.FinalError() 1320 n.metrics.flushColdData.ReportSuccessOrError(res, n.nowFn().Sub(callStart)) 1321 return res 1322 } 1323 1324 func (n *dbNamespace) FlushIndex(flush persist.IndexFlush) error { 1325 callStart := n.nowFn() 1326 n.RLock() 1327 if n.bootstrapState != Bootstrapped { 1328 n.RUnlock() 1329 n.metrics.flushIndex.ReportError(n.nowFn().Sub(callStart)) 1330 return errNamespaceNotBootstrapped 1331 } 1332 n.RUnlock() 1333 1334 if n.ReadOnly() || !n.nopts.FlushEnabled() || !n.nopts.IndexOptions().Enabled() { 1335 n.metrics.flushIndex.ReportSuccess(n.nowFn().Sub(callStart)) 1336 return nil 1337 } 1338 1339 shards := n.OwnedShards() 1340 err := n.reverseIndex.WarmFlush(flush, shards) 1341 n.metrics.flushIndex.ReportSuccessOrError(err, n.nowFn().Sub(callStart)) 1342 return err 1343 } 1344 1345 func (n *dbNamespace) Snapshot( 1346 blockStarts []xtime.UnixNano, 1347 snapshotTime xtime.UnixNano, 1348 snapshotPersist persist.SnapshotPreparer, 1349 ) error { 1350 // NB(rartoul): This value can be used for emitting metrics, but should not be used 1351 // for business logic. 1352 callStart := n.nowFn() 1353 1354 var nsCtx namespace.Context 1355 n.RLock() 1356 if n.bootstrapState != Bootstrapped { 1357 n.RUnlock() 1358 n.metrics.snapshot.ReportError(n.nowFn().Sub(callStart)) 1359 return errNamespaceNotBootstrapped 1360 } 1361 nsCtx = n.nsContextWithRLock() 1362 n.RUnlock() 1363 1364 if !n.nopts.SnapshotEnabled() { 1365 // Note that we keep the ability to disable snapshots at the namespace level around for 1366 // debugging / performance / flexibility reasons, but disabling it can / will cause data 1367 // loss due to the commitlog cleanup logic assuming that a valid snapshot checkpoint file 1368 // means that all namespaces were successfully snapshotted. 1369 n.metrics.snapshot.ReportSuccess(n.nowFn().Sub(callStart)) 1370 return nil 1371 } 1372 1373 var ( 1374 seriesPersist int 1375 multiErr xerrors.MultiError 1376 ) 1377 1378 for _, shard := range n.OwnedShards() { 1379 log := n.log.With(zap.Uint32("shard", shard.ID())) 1380 if !shard.IsBootstrapped() { 1381 log.Debug("skipping snapshot due to shard not bootstrapped yet") 1382 continue 1383 } 1384 snapshotBlockStarts := shard.FilterBlocksNeedSnapshot(blockStarts) 1385 if len(snapshotBlockStarts) == 0 { 1386 log.Debug("skipping shard snapshot since no blocks need it") 1387 continue 1388 } 1389 for _, blockStart := range snapshotBlockStarts { 1390 snapshotResult, err := shard.Snapshot(blockStart, snapshotTime, snapshotPersist, nsCtx) 1391 if err != nil { 1392 detailedErr := fmt.Errorf("shard %d failed to snapshot %v block: %w", shard.ID(), blockStart, err) 1393 multiErr = multiErr.Add(detailedErr) 1394 continue 1395 } 1396 seriesPersist += snapshotResult.SeriesPersist 1397 } 1398 } 1399 1400 n.metrics.snapshotSeriesPersist.Inc(int64(seriesPersist)) 1401 1402 res := multiErr.FinalError() 1403 n.metrics.snapshot.ReportSuccessOrError(res, n.nowFn().Sub(callStart)) 1404 return res 1405 } 1406 1407 func (n *dbNamespace) NeedsFlush( 1408 alignedInclusiveStart xtime.UnixNano, 1409 alignedInclusiveEnd xtime.UnixNano, 1410 ) (bool, error) { 1411 // NB(r): Essentially if all are success, we don't need to flush, if any 1412 // are failed with the minimum num failures less than max retries then 1413 // we need to flush - otherwise if any in progress we can't flush and if 1414 // any not started then we need to flush. 1415 n.RLock() 1416 defer n.RUnlock() 1417 return n.needsFlushWithLock(alignedInclusiveStart, alignedInclusiveEnd) 1418 } 1419 1420 func (n *dbNamespace) needsFlushWithLock( 1421 alignedInclusiveStart xtime.UnixNano, 1422 alignedInclusiveEnd xtime.UnixNano, 1423 ) (bool, error) { 1424 var ( 1425 blockSize = n.nopts.RetentionOptions().BlockSize() 1426 blockStarts = timesInRange(alignedInclusiveStart, alignedInclusiveEnd, blockSize) 1427 ) 1428 1429 // NB(prateek): we do not check if any other flush is in progress in this method, 1430 // instead relying on the databaseFlushManager to ensure atomicity of flushes. 1431 1432 // Check for not started or failed that might need a flush 1433 for _, shard := range n.shards { 1434 if shard == nil { 1435 continue 1436 } 1437 for _, blockStart := range blockStarts { 1438 flushState, err := shard.FlushState(blockStart) 1439 if err != nil { 1440 return false, err 1441 } 1442 if flushState.WarmStatus.DataFlushed != fileOpSuccess { 1443 return true, nil 1444 } 1445 } 1446 } 1447 1448 // All success or failed and reached max retries 1449 return false, nil 1450 } 1451 1452 func (n *dbNamespace) Truncate() (int64, error) { 1453 var totalNumSeries int64 1454 1455 n.RLock() 1456 shards := n.shardSet.AllIDs() 1457 for _, shard := range shards { 1458 totalNumSeries += n.shards[shard].NumSeries() 1459 } 1460 n.RUnlock() 1461 1462 // For now we are simply dropping all the objects (e.g., shards, series, blocks etc) owned by the 1463 // namespace, which means the memory will be reclaimed the next time GC kicks in and returns the 1464 // reclaimed memory to the OS. In the future, we might investigate whether it's worth returning 1465 // the pooled objects to the pools if the pool is low and needs replenishing. 1466 n.assignShardSet(n.shardSet, assignShardSetOptions{ 1467 needsBootstrap: false, 1468 initialAssignment: true, 1469 }) 1470 1471 // NB(xichen): possibly also clean up disk files and force a GC here to reclaim memory immediately 1472 return totalNumSeries, nil 1473 } 1474 1475 func (n *dbNamespace) Repair( 1476 repairer databaseShardRepairer, 1477 tr xtime.Range, 1478 opts NamespaceRepairOptions, 1479 ) error { 1480 shouldRun := opts.Force || n.nopts.RepairEnabled() 1481 if !shouldRun { 1482 return nil 1483 } 1484 1485 n.RLock() 1486 repairsAny := n.repairsAny 1487 n.RUnlock() 1488 if !repairsAny { 1489 // Only acquire write lock if required. 1490 n.Lock() 1491 n.repairsAny = true 1492 n.Unlock() 1493 } 1494 1495 var ( 1496 wg sync.WaitGroup 1497 mutex sync.Mutex 1498 numShardsRepaired int 1499 numTotalSeries int64 1500 numTotalBlocks int64 1501 numSizeDiffSeries int64 1502 numSizeDiffBlocks int64 1503 numChecksumDiffSeries int64 1504 numChecksumDiffBlocks int64 1505 peerMetadataComparisons repair.PeerMetadataComparisonResults 1506 throttlePerShard time.Duration 1507 ) 1508 1509 multiErr := xerrors.NewMultiError() 1510 shards := n.OwnedShards() 1511 numShards := len(shards) 1512 if numShards > 0 { 1513 throttlePerShard = time.Duration( 1514 int64(repairer.Options().RepairThrottle()) / int64(numShards)) 1515 } 1516 1517 workers := xsync.NewWorkerPool(repairer.Options().RepairShardConcurrency()) 1518 workers.Init() 1519 1520 n.RLock() 1521 nsCtx := n.nsContextWithRLock() 1522 nsMeta := n.metadata 1523 n.RUnlock() 1524 1525 for _, shard := range shards { 1526 shard := shard 1527 1528 wg.Add(1) 1529 workers.Go(func() { 1530 defer wg.Done() 1531 1532 ctx := n.opts.ContextPool().Get() 1533 defer ctx.Close() 1534 1535 metadataRes, err := shard.Repair(ctx, nsCtx, nsMeta, tr, repairer) 1536 1537 mutex.Lock() 1538 if err != nil { 1539 multiErr = multiErr.Add(err) 1540 } else { 1541 numShardsRepaired++ 1542 numTotalSeries += metadataRes.NumSeries 1543 numTotalBlocks += metadataRes.NumBlocks 1544 numSizeDiffSeries += metadataRes.SizeDifferences.NumSeries() 1545 numSizeDiffBlocks += metadataRes.SizeDifferences.NumBlocks() 1546 numChecksumDiffSeries += metadataRes.ChecksumDifferences.NumSeries() 1547 numChecksumDiffBlocks += metadataRes.ChecksumDifferences.NumBlocks() 1548 peerMetadataComparisons = append(peerMetadataComparisons, metadataRes.PeerMetadataComparisonResults...) 1549 } 1550 mutex.Unlock() 1551 1552 if throttlePerShard > 0 { 1553 time.Sleep(throttlePerShard) 1554 } 1555 }) 1556 } 1557 1558 wg.Wait() 1559 1560 aggregatePeerComparison := peerMetadataComparisons.Aggregate() 1561 n.metrics.repairDifferingPercent.Update(aggregatePeerComparison.ComparedDifferingPercent) 1562 n.metrics.repairComparedBlocks.Inc(aggregatePeerComparison.ComparedBlocks) 1563 n.metrics.repairDifferingBlocks.Inc(aggregatePeerComparison.ComparedDifferingBlocks) 1564 n.metrics.repairMismatchBlocks.Inc(aggregatePeerComparison.ComparedMismatchBlocks) 1565 n.metrics.repairMissingBlocks.Inc(aggregatePeerComparison.ComparedMissingBlocks) 1566 n.metrics.repairExtraBlocks.Inc(aggregatePeerComparison.ComparedExtraBlocks) 1567 1568 n.log.Info("repair result", 1569 zap.String("repairTimeRange", tr.String()), 1570 zap.Int("numTotalShards", len(shards)), 1571 zap.Int("numShardsRepaired", numShardsRepaired), 1572 zap.Int64("numTotalSeries", numTotalSeries), 1573 zap.Int64("numTotalBlocks", numTotalBlocks), 1574 zap.Int64("numSizeDiffSeries", numSizeDiffSeries), 1575 zap.Int64("numSizeDiffBlocks", numSizeDiffBlocks), 1576 zap.Int64("numChecksumDiffSeries", numChecksumDiffSeries), 1577 zap.Int64("numChecksumDiffBlocks", numChecksumDiffBlocks), 1578 zap.Float64("peerComparisonComparedDifferingPercent", aggregatePeerComparison.ComparedDifferingPercent), 1579 zap.Int64("peerComparisonComparedBlocks", aggregatePeerComparison.ComparedBlocks), 1580 zap.Int64("peerComparisonComparedDifferingBlocks", aggregatePeerComparison.ComparedDifferingBlocks), 1581 zap.Int64("peerComparisonComparedMismatchBlocks", aggregatePeerComparison.ComparedMismatchBlocks), 1582 zap.Int64("peerComparisonComparedMissingBlocks", aggregatePeerComparison.ComparedMissingBlocks), 1583 zap.Int64("peerComparisonComparedExtraBlocks", aggregatePeerComparison.ComparedExtraBlocks), 1584 ) 1585 1586 return multiErr.FinalError() 1587 } 1588 1589 func (n *dbNamespace) OwnedShards() []databaseShard { 1590 n.RLock() 1591 shards := n.shardSet.AllIDs() 1592 databaseShards := make([]databaseShard, len(shards)) 1593 for i, shard := range shards { 1594 databaseShards[i] = n.shards[shard] 1595 } 1596 n.RUnlock() 1597 return databaseShards 1598 } 1599 1600 func (n *dbNamespace) SetIndex(reverseIndex NamespaceIndex) error { 1601 n.Lock() 1602 defer n.Unlock() 1603 1604 if !n.metadata.Options().IndexOptions().Enabled() { 1605 return errNamespaceIndexingDisabled 1606 } 1607 n.reverseIndex = reverseIndex 1608 1609 return nil 1610 } 1611 1612 func (n *dbNamespace) Index() (NamespaceIndex, error) { 1613 n.RLock() 1614 defer n.RUnlock() 1615 if !n.metadata.Options().IndexOptions().Enabled() { 1616 return nil, errNamespaceIndexingDisabled 1617 } 1618 return n.reverseIndex, nil 1619 } 1620 1621 func (n *dbNamespace) ReadOnly() bool { 1622 return n.readOnly 1623 } 1624 1625 func (n *dbNamespace) SetReadOnly(value bool) { 1626 n.readOnly = value 1627 } 1628 1629 func (n *dbNamespace) shardFor(id ident.ID) (databaseShard, namespace.Context, error) { 1630 n.RLock() 1631 nsCtx := n.nsContextWithRLock() 1632 shardID := n.shardSet.Lookup(id) 1633 shard, _, err := n.shardAtWithRLock(shardID) 1634 n.RUnlock() 1635 return shard, nsCtx, err 1636 } 1637 1638 func (n *dbNamespace) readableShardFor(id ident.ID) (databaseShard, namespace.Context, error) { 1639 n.RLock() 1640 nsCtx := n.nsContextWithRLock() 1641 shardID := n.shardSet.Lookup(id) 1642 shard, err := n.readableShardAtWithRLock(shardID) 1643 n.RUnlock() 1644 return shard, nsCtx, err 1645 } 1646 1647 func (n *dbNamespace) ReadableShardAt(shardID uint32) (databaseShard, namespace.Context, error) { 1648 n.RLock() 1649 nsCtx := n.nsContextWithRLock() 1650 shard, err := n.readableShardAtWithRLock(shardID) 1651 n.RUnlock() 1652 return shard, nsCtx, err 1653 } 1654 1655 func (n *dbNamespace) shardAtWithRLock(shardID uint32) (databaseShard, bool, error) { 1656 // NB(r): These errors are retryable as they will occur 1657 // during a topology change and must be retried by the client. 1658 if int(shardID) >= len(n.shards) { 1659 return nil, false, xerrors.NewRetryableError( 1660 fmt.Errorf("not responsible for shard %d", shardID)) 1661 } 1662 shard := n.shards[shardID] 1663 if shard == nil { 1664 return nil, false, xerrors.NewRetryableError( 1665 fmt.Errorf("not responsible for shard %d", shardID)) 1666 } 1667 return shard, true, nil 1668 } 1669 1670 func (n *dbNamespace) readableShardAtWithRLock(shardID uint32) (databaseShard, error) { 1671 shard, _, err := n.shardAtWithRLock(shardID) 1672 if err != nil { 1673 return nil, err 1674 } 1675 if !shard.IsBootstrapped() { 1676 return nil, xerrors.NewRetryableError(errShardNotBootstrappedToRead) 1677 } 1678 return shard, nil 1679 } 1680 1681 func (n *dbNamespace) Close() error { 1682 n.Lock() 1683 if n.closed { 1684 n.Unlock() 1685 return errNamespaceAlreadyClosed 1686 } 1687 n.closed = true 1688 shards := n.shards 1689 n.shards = shards[:0] 1690 n.shardSet = sharding.NewEmptyShardSet(sharding.DefaultHashFn(1)) 1691 n.Unlock() 1692 n.namespaceReaderMgr.close() 1693 n.closeShards(shards, true) 1694 if retriever := n.blockRetriever; retriever != nil { 1695 if err := retriever.Close(); err != nil { 1696 n.log.Error("error when closing blockRetriever", 1697 zap.Error(err), zap.Stringer("namespace", n.id)) 1698 } 1699 } 1700 close(n.shutdownCh) 1701 if n.reverseIndex != nil { 1702 return n.reverseIndex.Close() 1703 } 1704 if n.schemaListener != nil { 1705 n.schemaListener.Close() 1706 } 1707 return nil 1708 } 1709 1710 func (n *dbNamespace) BootstrapState() BootstrapState { 1711 n.RLock() 1712 defer n.RUnlock() 1713 return n.bootstrapState 1714 } 1715 1716 func (n *dbNamespace) ShardBootstrapState() ShardBootstrapStates { 1717 n.RLock() 1718 shardStates := make(ShardBootstrapStates, len(n.shards)) 1719 for _, shard := range n.shards { 1720 if shard == nil { 1721 continue 1722 } 1723 shardStates[shard.ID()] = shard.BootstrapState() 1724 } 1725 n.RUnlock() 1726 return shardStates 1727 } 1728 1729 func (n *dbNamespace) FlushState(shardID uint32, blockStart xtime.UnixNano) (fileOpState, error) { 1730 n.RLock() 1731 defer n.RUnlock() 1732 shard, _, err := n.shardAtWithRLock(shardID) 1733 if err != nil { 1734 return fileOpState{}, err 1735 } 1736 flushState, err := shard.FlushState(blockStart) 1737 if err != nil { 1738 return fileOpState{}, err 1739 } 1740 return flushState, nil 1741 } 1742 1743 func (n *dbNamespace) nsContextWithRLock() namespace.Context { 1744 return namespace.Context{ID: n.id, Schema: n.schemaDescr} 1745 } 1746 1747 func (n *dbNamespace) AggregateTiles( 1748 ctx context.Context, 1749 sourceNs databaseNamespace, 1750 opts AggregateTilesOptions, 1751 ) (int64, error) { 1752 var ( 1753 callStart = n.nowFn() 1754 1755 aggregateTilesScope = opts.InsOptions.MetricsScope() 1756 timerOpts = n.opts.InstrumentOptions().TimerOptions() 1757 methodMetrics = instrument.NewMethodMetrics( 1758 aggregateTilesScope, "aggregate-tiles", timerOpts) 1759 ) 1760 1761 processedTileCount, err := n.aggregateTiles(ctx, sourceNs, opts) 1762 1763 methodMetrics.ReportSuccessOrError(err, n.nowFn().Sub(callStart)) 1764 1765 return processedTileCount, err 1766 } 1767 1768 func (n *dbNamespace) aggregateTiles( 1769 ctx context.Context, 1770 sourceNs databaseNamespace, 1771 opts AggregateTilesOptions, 1772 ) (int64, error) { 1773 var ( 1774 startedAt = time.Now() 1775 targetBlockSize = n.Metadata().Options().RetentionOptions().BlockSize() 1776 targetBlockStart = opts.Start.Truncate(targetBlockSize) 1777 sourceBlockSize = sourceNs.Options().RetentionOptions().BlockSize() 1778 lastSourceBlockEnd = opts.End.Truncate(sourceBlockSize) 1779 1780 scope = opts.InsOptions.MetricsScope() 1781 processedShards = scope.Counter("processed-shards") 1782 ) 1783 1784 if targetBlockStart.Add(targetBlockSize).Before(lastSourceBlockEnd) { 1785 return 0, fmt.Errorf("tile aggregation must be done within a single target block (start=%s, end=%s, blockSize=%s)", 1786 opts.Start, opts.End, targetBlockSize.String()) 1787 } 1788 1789 if n.BootstrapState() != Bootstrapped || sourceNs.BootstrapState() != Bootstrapped { 1790 return 0, errNamespaceNotBootstrapped 1791 } 1792 1793 if len(n.OwnedShards()) == 0 { 1794 // This is possible during a new node join. 1795 n.log.Info("no shards owned, skip AggregateTiles") 1796 return 0, nil 1797 } 1798 1799 // Create an empty warm index fileset because the block 1800 // will not be queryable if it has no warm index. 1801 if err := n.createEmptyWarmIndexIfNotExistsFn(targetBlockStart); err != nil { 1802 return 0, err 1803 } 1804 1805 // Cold flusher builds the reverse index for target (current) ns. 1806 cfOpts := NewColdFlushNsOpts(false) 1807 onColdFlushNs, err := n.opts.OnColdFlush().ColdFlushNamespace(n, cfOpts) 1808 if err != nil { 1809 return 0, err 1810 } 1811 1812 var ( 1813 processedTileCount int64 1814 aggregationSuccess bool 1815 ) 1816 1817 defer func() { 1818 if aggregationSuccess { 1819 return 1820 } 1821 // Abort building reverse index if aggregation fails. 1822 if err := onColdFlushNs.Abort(); err != nil { 1823 n.log.Error("error aborting cold flush", 1824 zap.Stringer("sourceNs", sourceNs.ID()), zap.Error(err)) 1825 } 1826 }() 1827 1828 for _, targetShard := range n.OwnedShards() { 1829 if !targetShard.IsBootstrapped() { 1830 n.log.Debug("skipping aggregateTiles due to shard not bootstrapped", 1831 zap.Uint32("shard", targetShard.ID())) 1832 continue 1833 } 1834 1835 shardProcessedTileCount, err := targetShard.AggregateTiles( 1836 ctx, sourceNs, n, targetShard.ID(), onColdFlushNs, opts) 1837 1838 processedTileCount += shardProcessedTileCount 1839 processedShards.Inc(1) 1840 if err != nil { 1841 return 0, fmt.Errorf("shard %d aggregation failed: %v", targetShard.ID(), err) 1842 } 1843 } 1844 1845 // Aggregation success, mark so we don't abort reverse index building (cold flusher). 1846 aggregationSuccess = true 1847 if err := onColdFlushNs.Done(); err != nil { 1848 return 0, err 1849 } 1850 1851 n.log.Info("finished large tiles aggregation for namespace", 1852 zap.Stringer("sourceNs", sourceNs.ID()), 1853 zap.Stringer("process", opts.Process), 1854 zap.Bool("memorizeMetricTypes", opts.MemorizeMetricTypes), 1855 zap.Bool("backfillMetricTypes", opts.BackfillMetricTypes), 1856 zap.Time("targetBlockStart", targetBlockStart.ToTime()), 1857 zap.Time("lastSourceBlockEnd", lastSourceBlockEnd.ToTime()), 1858 zap.Duration("step", opts.Step), 1859 zap.Int64("processedTiles", processedTileCount), 1860 zap.Duration("took", time.Now().Sub(startedAt))) 1861 1862 return processedTileCount, nil 1863 } 1864 1865 func (n *dbNamespace) DocRef(id ident.ID) (doc.Metadata, bool, error) { 1866 shard, _, err := n.readableShardFor(id) 1867 if err != nil { 1868 return doc.Metadata{}, false, err 1869 } 1870 return shard.DocRef(id) 1871 } 1872 1873 func (n *dbNamespace) createEmptyWarmIndexIfNotExists(blockStart xtime.UnixNano) error { 1874 fsOpts := n.opts.CommitLogOptions().FilesystemOptions() 1875 1876 shardIds := make(map[uint32]struct{}, len(n.OwnedShards())) 1877 for _, shard := range n.OwnedShards() { 1878 if shard.IsBootstrapped() { 1879 shardIds[shard.ID()] = struct{}{} 1880 } 1881 } 1882 1883 fileSetID := fs.FileSetFileIdentifier{ 1884 FileSetContentType: persist.FileSetIndexContentType, 1885 Namespace: n.ID(), 1886 BlockStart: blockStart, 1887 VolumeIndex: 0, 1888 } 1889 1890 warmIndexOpts := fs.IndexWriterOpenOptions{ 1891 BlockSize: n.Metadata().Options().IndexOptions().BlockSize(), 1892 FileSetType: persist.FileSetFlushType, 1893 Identifier: fileSetID, 1894 Shards: shardIds, 1895 IndexVolumeType: idxpersist.DefaultIndexVolumeType, 1896 } 1897 1898 warmIndexWriter, err := fs.NewIndexWriter(fsOpts) 1899 if err != nil { 1900 return err 1901 } 1902 1903 if err = warmIndexWriter.Open(warmIndexOpts); err != nil { 1904 if xerrors.Is(err, iofs.ErrExist) { 1905 n.log.Debug("warm index already exists", 1906 zap.Stringer("namespace", n.id), 1907 zap.Stringer("blockStart", blockStart), 1908 zap.Reflect("shardIds", shardIds)) 1909 return nil 1910 } 1911 return err 1912 } 1913 1914 return warmIndexWriter.Close() 1915 }