github.com/m3db/m3@v1.5.1-0.20231129193456-75a402aa583b/src/dbnode/storage/index/mutable_segments.go (about) 1 // Copyright (c) 2020 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 index 22 23 import ( 24 "errors" 25 "fmt" 26 "io" 27 "math" 28 "runtime" 29 "sync" 30 "time" 31 32 "github.com/m3db/m3/src/dbnode/namespace" 33 "github.com/m3db/m3/src/dbnode/storage/index/compaction" 34 "github.com/m3db/m3/src/dbnode/storage/index/segments" 35 "github.com/m3db/m3/src/m3ninx/doc" 36 "github.com/m3db/m3/src/m3ninx/generated/proto/querypb" 37 m3ninxindex "github.com/m3db/m3/src/m3ninx/index" 38 "github.com/m3db/m3/src/m3ninx/index/segment" 39 "github.com/m3db/m3/src/m3ninx/index/segment/builder" 40 "github.com/m3db/m3/src/m3ninx/index/segment/fst" 41 "github.com/m3db/m3/src/m3ninx/postings" 42 "github.com/m3db/m3/src/m3ninx/postings/pilosa" 43 "github.com/m3db/m3/src/m3ninx/postings/roaring" 44 "github.com/m3db/m3/src/m3ninx/search" 45 "github.com/m3db/m3/src/m3ninx/search/query" 46 "github.com/m3db/m3/src/m3ninx/x" 47 "github.com/m3db/m3/src/x/context" 48 xerrors "github.com/m3db/m3/src/x/errors" 49 "github.com/m3db/m3/src/x/instrument" 50 "github.com/m3db/m3/src/x/mmap" 51 xresource "github.com/m3db/m3/src/x/resource" 52 xsync "github.com/m3db/m3/src/x/sync" 53 xtime "github.com/m3db/m3/src/x/time" 54 55 bitmap "github.com/m3dbx/pilosa/roaring" 56 "github.com/uber-go/tally" 57 "go.uber.org/zap" 58 "golang.org/x/sync/errgroup" 59 ) 60 61 var ( 62 errUnableToWriteBlockConcurrent = errors.New("unable to write, index block is being written to already") 63 errMutableSegmentsAlreadyClosed = errors.New("mutable segments already closed") 64 errForegroundCompactorNoPlan = errors.New("index foreground compactor failed to generate a plan") 65 errForegroundCompactorBadPlanFirstTask = errors.New("index foreground compactor generated plan without mutable segment in first task") 66 errForegroundCompactorBadPlanSecondaryTask = errors.New("index foreground compactor generated plan with mutable segment a secondary task") 67 68 numBackgroundCompactorsStandard = 1 69 numBackgroundCompactorsGarbageCollect = 1 70 ) 71 72 type mutableSegmentsState uint 73 74 const ( 75 mutableSegmentsStateOpen mutableSegmentsState = iota 76 mutableSegmentsStateClosed mutableSegmentsState = iota 77 78 segmentCheckInactiveSeriesMinInterval = 5 * time.Minute 79 80 maxForegroundCompactorAge = time.Hour * 1 81 ) 82 83 var _ segment.DocumentsFilter = (*mutableSegments)(nil) 84 85 // nolint: maligned 86 type mutableSegments struct { 87 sync.RWMutex 88 89 state mutableSegmentsState 90 91 foregroundSegments []*readableSeg 92 backgroundSegments []*readableSeg 93 94 compact mutableSegmentsCompact 95 blockStart xtime.UnixNano 96 blockSize time.Duration 97 blockOpts BlockOptions 98 opts Options 99 iopts instrument.Options 100 optsListener xresource.SimpleCloser 101 writeIndexingConcurrency int 102 cachedSearchesWorkers xsync.WorkerPool 103 104 metrics mutableSegmentsMetrics 105 logger *zap.Logger 106 107 // For testing purposes. 108 backgroundCompactDisable bool 109 } 110 111 type mutableSegmentsMetrics struct { 112 foregroundCompactionPlanRunLatency tally.Timer 113 foregroundCompactionTaskRunLatency tally.Timer 114 backgroundCompactionPlanRunLatency tally.Timer 115 backgroundCompactionTaskRunLatency tally.Timer 116 activeBlockIndexNew tally.Counter 117 activeBlockGarbageCollectSegment tally.Counter 118 activeBlockGarbageCollectSeries tally.Counter 119 activeBlockGarbageCollectDuplicate tally.Counter 120 activeBlockGarbageCollectEmptySegment tally.Counter 121 activeBlockGarbageCollectCachedSearchesDisabled tally.Counter 122 activeBlockGarbageCollectCachedSearchesInRegistry tally.Counter 123 activeBlockGarbageCollectCachedSearchesNotInRegistry tally.Counter 124 activeBlockGarbageCollectCachedSearchesTotal tally.Histogram 125 activeBlockGarbageCollectCachedSearchesMatched tally.Histogram 126 activeBlockGarbageCollectReconstructCachedSearchEvalSkip tally.Counter 127 activeBlockGarbageCollectReconstructCachedSearchEvalAttempt tally.Counter 128 activeBlockGarbageCollectReconstructCachedSearchCacheHit tally.Counter 129 activeBlockGarbageCollectReconstructCachedSearchCacheMiss tally.Counter 130 activeBlockGarbageCollectReconstructCachedSearchExecSuccess tally.Counter 131 activeBlockGarbageCollectReconstructCachedSearchExecError tally.Counter 132 } 133 134 func newMutableSegmentsMetrics(s tally.Scope) mutableSegmentsMetrics { 135 foregroundScope := s.Tagged(map[string]string{"compaction-type": "foreground"}) 136 backgroundScope := s.Tagged(map[string]string{"compaction-type": "background"}) 137 activeBlockScope := s.SubScope("active-block") 138 return mutableSegmentsMetrics{ 139 foregroundCompactionPlanRunLatency: foregroundScope.Timer("compaction-plan-run-latency"), 140 foregroundCompactionTaskRunLatency: foregroundScope.Timer("compaction-task-run-latency"), 141 backgroundCompactionPlanRunLatency: backgroundScope.Timer("compaction-plan-run-latency"), 142 backgroundCompactionTaskRunLatency: backgroundScope.Timer("compaction-task-run-latency"), 143 activeBlockIndexNew: activeBlockScope.Tagged(map[string]string{ 144 "result_type": "new", 145 }).Counter("index-result"), 146 activeBlockGarbageCollectSegment: activeBlockScope.Counter("gc-segment"), 147 activeBlockGarbageCollectSeries: activeBlockScope.Counter("gc-series"), 148 activeBlockGarbageCollectDuplicate: activeBlockScope.Counter("gc-duplicate"), 149 activeBlockGarbageCollectEmptySegment: backgroundScope.Counter("gc-empty-segment"), 150 activeBlockGarbageCollectCachedSearchesDisabled: backgroundScope.Counter("gc-cached-searches-disabled"), 151 activeBlockGarbageCollectCachedSearchesInRegistry: backgroundScope.Tagged(map[string]string{ 152 "found": "true", 153 }).Counter("gc-cached-searches-in-registry"), 154 activeBlockGarbageCollectCachedSearchesNotInRegistry: backgroundScope.Tagged(map[string]string{ 155 "found": "false", 156 }).Counter("gc-cached-searches-in-registry"), 157 activeBlockGarbageCollectCachedSearchesTotal: backgroundScope.Histogram("gc-cached-searches-total", 158 append(tally.ValueBuckets{0, 1}, tally.MustMakeExponentialValueBuckets(2, 2, 12)...)), 159 activeBlockGarbageCollectCachedSearchesMatched: backgroundScope.Histogram("gc-cached-searches-matched", 160 append(tally.ValueBuckets{0, 1}, tally.MustMakeExponentialValueBuckets(2, 2, 12)...)), 161 activeBlockGarbageCollectReconstructCachedSearchEvalSkip: backgroundScope.Tagged(map[string]string{ 162 "eval_type": "skip", 163 }).Counter("gc-reconstruct-cached-search-eval"), 164 activeBlockGarbageCollectReconstructCachedSearchEvalAttempt: backgroundScope.Tagged(map[string]string{ 165 "eval_type": "attempt", 166 }).Counter("gc-reconstruct-cached-search-eval"), 167 activeBlockGarbageCollectReconstructCachedSearchCacheHit: backgroundScope.Tagged(map[string]string{ 168 "result_type": "cache_hit", 169 }).Counter("gc-reconstruct-cached-search-cache-result"), 170 activeBlockGarbageCollectReconstructCachedSearchCacheMiss: backgroundScope.Tagged(map[string]string{ 171 "result_type": "cache_miss", 172 }).Counter("gc-reconstruct-cached-search-cache-result"), 173 activeBlockGarbageCollectReconstructCachedSearchExecSuccess: backgroundScope.Tagged(map[string]string{ 174 "result_type": "success", 175 }).Counter("gc-reconstruct-cached-search-exec-result"), 176 activeBlockGarbageCollectReconstructCachedSearchExecError: backgroundScope.Tagged(map[string]string{ 177 "result_type": "error", 178 }).Counter("gc-reconstruct-cached-search-exec-result"), 179 } 180 } 181 182 // newMutableSegments returns a new Block, representing a complete reverse index 183 // for the duration of time specified. It is backed by one or more segments. 184 func newMutableSegments( 185 md namespace.Metadata, 186 blockStart xtime.UnixNano, 187 opts Options, 188 blockOpts BlockOptions, 189 cachedSearchesWorkers xsync.WorkerPool, 190 namespaceRuntimeOptsMgr namespace.RuntimeOptionsManager, 191 iopts instrument.Options, 192 ) *mutableSegments { 193 m := &mutableSegments{ 194 blockStart: blockStart, 195 blockSize: md.Options().IndexOptions().BlockSize(), 196 opts: opts, 197 blockOpts: blockOpts, 198 compact: mutableSegmentsCompact{opts: opts, blockOpts: blockOpts}, 199 cachedSearchesWorkers: cachedSearchesWorkers, 200 iopts: iopts, 201 metrics: newMutableSegmentsMetrics(iopts.MetricsScope()), 202 logger: iopts.Logger(), 203 } 204 m.optsListener = namespaceRuntimeOptsMgr.RegisterListener(m) 205 return m 206 } 207 208 func (m *mutableSegments) SetNamespaceRuntimeOptions(opts namespace.RuntimeOptions) { 209 m.Lock() 210 // Update current runtime opts for segment builders created in future. 211 perCPUFraction := opts.WriteIndexingPerCPUConcurrencyOrDefault() 212 cpus := math.Ceil(perCPUFraction * float64(runtime.GOMAXPROCS(0))) 213 m.writeIndexingConcurrency = int(math.Max(1, cpus)) 214 segmentBuilder := m.compact.segmentBuilder 215 m.Unlock() 216 217 // Reset any existing segment builder to new concurrency, do this 218 // out of the lock since builder can be used for foreground compaction 219 // outside the lock and does it's own locking. 220 if segmentBuilder != nil { 221 segmentBuilder.SetIndexConcurrency(m.writeIndexingConcurrency) 222 } 223 224 // Set the global concurrency control we have (we may need to fork 225 // github.com/twotwotwo/sorts to control this on a per segment builder 226 // basis). 227 builder.SetSortConcurrency(m.writeIndexingConcurrency) 228 } 229 230 func (m *mutableSegments) ContainsDoc(d doc.Metadata) bool { 231 // Filter out any documents that only were indexed for 232 // sealed blocks. 233 if d.OnIndexSeries == nil { 234 instrument.EmitAndLogInvariantViolation(m.iopts, func(l *zap.Logger) { 235 l.Error("unexpected nil for document index entry for background compact") 236 }) 237 return true 238 } 239 240 gc := d.OnIndexSeries.TryMarkIndexGarbageCollected() 241 if gc { 242 // Track expired series filtered out from new index segment during compaction. 243 m.metrics.activeBlockGarbageCollectSeries.Inc(1) 244 } 245 246 // We only want the new segment to contain the doc if we didn't need to GC it. 247 return !gc 248 } 249 250 func (m *mutableSegments) OnDuplicateDoc(d doc.Metadata) { 251 // NB: it is important to ensure duplicate entries get reconciled, as 252 // an entry being duplicated here may indicate that it is not the same 253 // entry as that stored in the shard's index map. Without this step, 254 // situations can arise when an entry may not be correctly indexed in 255 // all blocks, as the full index range for this entry may be split 256 // between the entry in the shard index map that would be persited, 257 // and this duplicated entry which will eventually expire and never 258 // get written to disk. Reconciling merges the full index ranges into 259 // the entry persisted in the shard index map. 260 if d.OnIndexSeries == nil { 261 instrument.EmitAndLogInvariantViolation(m.iopts, func(l *zap.Logger) { 262 l.Error("unexpected nil for document index entry for background compact") 263 }) 264 return 265 } 266 267 d.OnIndexSeries.TryReconcileDuplicates() 268 269 // Track duplicate filtered out from new index segment during compaction. 270 m.metrics.activeBlockGarbageCollectDuplicate.Inc(1) 271 } 272 273 func (m *mutableSegments) WriteBatch(inserts *WriteBatch) (MutableSegmentsStats, error) { 274 m.Lock() 275 if m.state == mutableSegmentsStateClosed { 276 m.Unlock() 277 return MutableSegmentsStats{}, errMutableSegmentsAlreadyClosed 278 } 279 280 if m.compact.compactingForeground { 281 m.Unlock() 282 return MutableSegmentsStats{}, errUnableToWriteBlockConcurrent 283 } 284 285 // Lazily allocate the segment builder and compactors. 286 err := m.compact.allocLazyBuilderAndCompactorsWithLock(m.writeIndexingConcurrency) 287 if err != nil { 288 m.Unlock() 289 return MutableSegmentsStats{}, err 290 } 291 292 m.compact.compactingForeground = true 293 segmentBuilder := m.compact.segmentBuilder 294 m.Unlock() 295 296 defer func() { 297 m.Lock() 298 m.compact.compactingForeground = false 299 m.cleanupForegroundCompactWithLock() 300 m.Unlock() 301 }() 302 303 docs := inserts.PendingDocs() 304 entries := inserts.PendingEntries() 305 306 // Set the doc ref for later recall. 307 for i := range entries { 308 docs[i].OnIndexSeries = entries[i].OnIndexSeries 309 } 310 311 segmentBuilder.Reset() 312 insertResultErr := segmentBuilder.InsertBatch(m3ninxindex.Batch{ 313 Docs: docs, 314 AllowPartialUpdates: true, 315 }) 316 n := len(segmentBuilder.Docs()) 317 if n == 0 { 318 // No inserts, no need to compact. 319 return MutableSegmentsStats{}, insertResultErr 320 } 321 322 // We inserted some documents, need to compact immediately into a 323 // foreground segment from the segment builder before we can serve reads 324 // from an FST segment. 325 result, err := m.foregroundCompactWithBuilder(segmentBuilder) 326 if err != nil { 327 return MutableSegmentsStats{}, err 328 } 329 330 m.metrics.activeBlockIndexNew.Inc(int64(n)) 331 332 // Return result from the original insertion since compaction was successful. 333 return result, insertResultErr 334 } 335 336 func (m *mutableSegments) AddReaders(readers []segment.Reader) ([]segment.Reader, error) { 337 m.RLock() 338 defer m.RUnlock() 339 340 var err error 341 readers, err = m.addReadersWithLock(m.foregroundSegments, readers) 342 if err != nil { 343 return nil, err 344 } 345 346 readers, err = m.addReadersWithLock(m.backgroundSegments, readers) 347 if err != nil { 348 return nil, err 349 } 350 351 return readers, nil 352 } 353 354 func (m *mutableSegments) addReadersWithLock(src []*readableSeg, dst []segment.Reader) ([]segment.Reader, error) { 355 for _, seg := range src { 356 reader, err := seg.Segment().Reader() 357 if err != nil { 358 return nil, err 359 } 360 dst = append(dst, reader) 361 } 362 return dst, nil 363 } 364 365 func (m *mutableSegments) Len() int { 366 m.RLock() 367 defer m.RUnlock() 368 369 return len(m.foregroundSegments) + len(m.backgroundSegments) 370 } 371 372 func (m *mutableSegments) MemorySegmentsData(ctx context.Context) ([]fst.SegmentData, error) { 373 m.RLock() 374 defer m.RUnlock() 375 376 // NB(r): This is for debug operations, do not bother about allocations. 377 var results []fst.SegmentData 378 for _, segs := range [][]*readableSeg{ 379 m.foregroundSegments, 380 m.backgroundSegments, 381 } { 382 for _, seg := range segs { 383 fstSegment, ok := seg.Segment().(fst.Segment) 384 if !ok { 385 return nil, fmt.Errorf("segment not fst segment: created=%v", seg.createdAt) 386 } 387 388 segmentData, err := fstSegment.SegmentData(ctx) 389 if err != nil { 390 return nil, err 391 } 392 393 results = append(results, segmentData) 394 } 395 } 396 return results, nil 397 } 398 399 func (m *mutableSegments) NeedsEviction() bool { 400 m.RLock() 401 defer m.RUnlock() 402 403 var needsEviction bool 404 for _, seg := range m.foregroundSegments { 405 needsEviction = needsEviction || seg.Segment().Size() > 0 406 } 407 for _, seg := range m.backgroundSegments { 408 needsEviction = needsEviction || seg.Segment().Size() > 0 409 } 410 return needsEviction 411 } 412 413 func (m *mutableSegments) NumSegmentsAndDocs() (int64, int64) { 414 m.RLock() 415 defer m.RUnlock() 416 417 foregroundNumSegments, foregroundNumDocs := numSegmentsAndDocs(m.foregroundSegments) 418 backgroundNumSegments, backgroundNumDocs := numSegmentsAndDocs(m.backgroundSegments) 419 numSegments := foregroundNumSegments + backgroundNumSegments 420 numDocs := foregroundNumDocs + backgroundNumDocs 421 return numSegments, numDocs 422 } 423 424 func numSegmentsAndDocs(segs []*readableSeg) (int64, int64) { 425 var numSegments, numDocs int64 426 for _, seg := range segs { 427 numSegments++ 428 numDocs += seg.Segment().Size() 429 } 430 return numSegments, numDocs 431 } 432 433 func (m *mutableSegments) Stats(reporter BlockStatsReporter) { 434 m.RLock() 435 defer m.RUnlock() 436 437 for _, seg := range m.foregroundSegments { 438 _, mutable := seg.Segment().(segment.MutableSegment) 439 reporter.ReportSegmentStats(BlockSegmentStats{ 440 Type: ActiveForegroundSegment, 441 Mutable: mutable, 442 Age: seg.Age(), 443 Size: seg.Segment().Size(), 444 }) 445 } 446 for _, seg := range m.backgroundSegments { 447 _, mutable := seg.Segment().(segment.MutableSegment) 448 reporter.ReportSegmentStats(BlockSegmentStats{ 449 Type: ActiveBackgroundSegment, 450 Mutable: mutable, 451 Age: seg.Age(), 452 Size: seg.Segment().Size(), 453 }) 454 } 455 456 reporter.ReportIndexingStats(BlockIndexingStats{ 457 IndexConcurrency: m.writeIndexingConcurrency, 458 }) 459 } 460 461 func (m *mutableSegments) Close() { 462 m.Lock() 463 defer m.Unlock() 464 m.state = mutableSegmentsStateClosed 465 m.cleanupCompactWithLock() 466 m.optsListener.Close() 467 } 468 469 func (m *mutableSegments) maybeBackgroundCompactWithLock() { 470 if m.compact.compactingBackgroundStandard || m.backgroundCompactDisable { 471 return 472 } 473 474 m.backgroundCompactWithLock(false) 475 } 476 477 // BackgroundCompact background compacts eligible segments. 478 func (m *mutableSegments) BackgroundCompact() { 479 m.Lock() 480 defer m.Unlock() 481 482 m.backgroundCompactWithLock(true) 483 } 484 485 func (m *mutableSegments) backgroundCompactWithLock(force bool) { 486 // Create a logical plan. 487 segs := make([]compaction.Segment, 0, len(m.backgroundSegments)) 488 for _, seg := range m.backgroundSegments { 489 if seg.garbageCollecting { 490 // Do not try to compact something that we are background 491 // garbage collecting documents from (that have been phased out). 492 continue 493 } 494 segs = append(segs, compaction.Segment{ 495 Age: seg.Age(), 496 Size: seg.Segment().Size(), 497 Type: segments.FSTType, 498 Segment: seg.Segment(), 499 }) 500 } 501 502 plan, err := compaction.NewPlan(segs, m.opts.BackgroundCompactionPlannerOptions()) 503 if err != nil { 504 instrument.EmitAndLogInvariantViolation(m.iopts, func(l *zap.Logger) { 505 l.Error("index background compaction plan error", zap.Error(err)) 506 }) 507 return 508 } 509 510 var ( 511 gcRequired = false 512 gcPlan = &compaction.Plan{} 513 gcAlreadyRunning = m.compact.compactingBackgroundGarbageCollect 514 nowFn = m.opts.ClockOptions().NowFn() 515 now = nowFn() 516 ) 517 if !gcAlreadyRunning { 518 gcRequired = true 519 520 for _, seg := range m.backgroundSegments { 521 sinceLastInactiveSeriesCheck := now.Sub(seg.garbageCollectLastCheck) 522 seg.garbageCollectLastCheck = now 523 if !force && sinceLastInactiveSeriesCheck < segmentCheckInactiveSeriesMinInterval { 524 // Only consider for compaction every so often. 525 continue 526 } 527 528 alreadyHasTask := false 529 for _, task := range plan.Tasks { 530 for _, taskSegment := range task.Segments { 531 if taskSegment.Segment == seg.Segment() { 532 alreadyHasTask = true 533 break 534 } 535 } 536 } 537 if alreadyHasTask { 538 // Skip needing to check if segment needs filtering. 539 continue 540 } 541 542 // Ensure that segment has some series that need to be GC'd. 543 hasAnyInactiveSeries, err := m.segmentAnyInactiveSeries(seg.Segment()) 544 if err != nil { 545 instrument.EmitAndLogInvariantViolation(m.iopts, func(l *zap.Logger) { 546 l.Error("error detecting needs background gc segment", zap.Error(err)) 547 }) 548 continue 549 } 550 if !hasAnyInactiveSeries { 551 // Skip background GC since all series are still active and no 552 // series need to be removed. 553 continue 554 } 555 556 // The active block starts are outdated, need to compact 557 // and remove any old data from the segment. 558 var task compaction.Task 559 if len(gcPlan.Tasks) > 0 { 560 task = gcPlan.Tasks[0] 561 } 562 563 task.Segments = append(task.Segments, compaction.Segment{ 564 Age: seg.Age(), 565 Size: seg.Segment().Size(), 566 Type: segments.FSTType, 567 Segment: seg.Segment(), 568 }) 569 570 if len(gcPlan.Tasks) == 0 { 571 gcPlan.Tasks = make([]compaction.Task, 1) 572 } 573 gcPlan.Tasks[0] = task 574 575 // Mark as not-compactable for standard compactions 576 // since this will be async compacted into a smaller 577 // segment. 578 seg.garbageCollecting = true 579 } 580 } 581 582 if len(plan.Tasks) != 0 { 583 // Kick off compaction. 584 m.compact.compactingBackgroundStandard = true 585 go func() { 586 m.backgroundCompactWithPlan(plan, m.compact.backgroundCompactors, gcRequired) 587 588 m.Lock() 589 m.compact.compactingBackgroundStandard = false 590 m.cleanupBackgroundCompactWithLock() 591 m.Unlock() 592 }() 593 } 594 595 if len(gcPlan.Tasks) != 0 { 596 // Run non-GC tasks separately so the standard loop is not blocked. 597 m.compact.compactingBackgroundGarbageCollect = true 598 go func() { 599 compactors, err := m.compact.allocBackgroundCompactorsGarbageCollect() 600 if err != nil { 601 instrument.EmitAndLogInvariantViolation(m.iopts, func(l *zap.Logger) { 602 l.Error("error background gc segments", zap.Error(err)) 603 }) 604 } else { 605 m.backgroundCompactWithPlan(gcPlan, compactors, gcRequired) 606 m.closeCompactors(compactors) 607 } 608 609 m.Lock() 610 m.compact.compactingBackgroundGarbageCollect = false 611 m.cleanupBackgroundCompactWithLock() 612 m.Unlock() 613 }() 614 } 615 } 616 617 func (m *mutableSegments) segmentAnyInactiveSeries(seg segment.Segment) (bool, error) { 618 reader, err := seg.Reader() 619 if err != nil { 620 return false, err 621 } 622 623 defer func() { 624 _ = reader.Close() 625 }() 626 627 docs, err := reader.AllDocs() 628 if err != nil { 629 return false, err 630 } 631 632 docsCloser := x.NewSafeCloser(docs) 633 defer func() { 634 // In case of early return cleanup 635 _ = docsCloser.Close() 636 }() 637 638 var result bool 639 for docs.Next() { 640 d := docs.Current() 641 indexEntry := d.OnIndexSeries 642 if indexEntry == nil { 643 return false, fmt.Errorf("document has no index entry: %s", d.ID) 644 } 645 if indexEntry.NeedsIndexGarbageCollected() { 646 result = true 647 break 648 } 649 } 650 651 if err := docs.Err(); err != nil { 652 return false, err 653 } 654 655 return result, docsCloser.Close() 656 } 657 658 func (m *mutableSegments) shouldEvictCompactedSegmentsWithLock() bool { 659 return m.state == mutableSegmentsStateClosed 660 } 661 662 func (m *mutableSegments) cleanupBackgroundCompactWithLock() { 663 if m.state == mutableSegmentsStateOpen { 664 // See if we need to trigger another compaction. 665 m.maybeBackgroundCompactWithLock() 666 return 667 } 668 669 // Check if need to close all the compacted segments due to 670 // mutableSegments being closed. 671 if !m.shouldEvictCompactedSegmentsWithLock() { 672 return 673 } 674 675 // Close compacted segments. 676 m.closeCompactedSegmentsWithLock(m.backgroundSegments) 677 m.backgroundSegments = nil 678 679 // Free compactor resources. 680 if m.compact.backgroundCompactors == nil { 681 return 682 } 683 684 m.closeCompactors(m.compact.backgroundCompactors) 685 m.compact.backgroundCompactors = nil 686 } 687 688 func (m *mutableSegments) closeCompactors(compactors chan *compaction.Compactor) { 689 close(compactors) 690 for compactor := range compactors { 691 err := compactor.Close() 692 if err == nil { 693 continue 694 } 695 696 instrument.EmitAndLogInvariantViolation(m.iopts, func(l *zap.Logger) { 697 l.Error("error closing index block background compactor", zap.Error(err)) 698 }) 699 } 700 } 701 702 func (m *mutableSegments) closeCompactedSegmentsWithLock(segments []*readableSeg) { 703 for _, seg := range segments { 704 err := seg.Segment().Close() 705 if err != nil { 706 instrument.EmitAndLogInvariantViolation(m.iopts, func(l *zap.Logger) { 707 l.Error("could not close compacted segment", zap.Error(err)) 708 }) 709 } 710 } 711 } 712 713 func (m *mutableSegments) backgroundCompactWithPlan( 714 plan *compaction.Plan, 715 compactors chan *compaction.Compactor, 716 gcRequired bool, 717 ) { 718 sw := m.metrics.backgroundCompactionPlanRunLatency.Start() 719 defer sw.Stop() 720 721 n := m.compact.numBackground 722 m.compact.numBackground++ 723 724 logger := m.logger.With( 725 zap.Time("blockStart", m.blockStart.ToTime()), 726 zap.Int("numBackgroundCompaction", n), 727 ) 728 log := n%compactDebugLogEvery == 0 729 if log { 730 for i, task := range plan.Tasks { 731 summary := task.Summary() 732 logger.Debug("planned background compaction task", 733 zap.Int("task", i), 734 zap.Int("numMutable", summary.NumMutable), 735 zap.Int("numFST", summary.NumFST), 736 zap.Stringer("cumulativeMutableAge", summary.CumulativeMutableAge), 737 zap.Int64("cumulativeSize", summary.CumulativeSize), 738 ) 739 } 740 } 741 742 var wg sync.WaitGroup 743 for i, task := range plan.Tasks { 744 i, task := i, task 745 wg.Add(1) 746 compactor := <-compactors 747 go func() { 748 defer func() { 749 compactors <- compactor 750 wg.Done() 751 }() 752 err := m.backgroundCompactWithTask(task, compactor, gcRequired, 753 log, logger.With(zap.Int("task", i))) 754 if err != nil { 755 instrument.EmitAndLogInvariantViolation(m.iopts, func(l *zap.Logger) { 756 l.Error("error compacting segments", zap.Error(err)) 757 }) 758 } 759 }() 760 } 761 762 wg.Wait() 763 } 764 765 func (m *mutableSegments) newReadThroughSegment(seg fst.Segment) *ReadThroughSegment { 766 var ( 767 plCaches = ReadThroughSegmentCaches{ 768 SegmentPostingsListCache: m.opts.PostingsListCache(), 769 SearchPostingsListCache: m.opts.SearchPostingsListCache(), 770 } 771 readThroughOpts = m.opts.ReadThroughSegmentOptions() 772 ) 773 return NewReadThroughSegment(seg, plCaches, readThroughOpts) 774 } 775 776 func (m *mutableSegments) backgroundCompactWithTask( 777 task compaction.Task, 778 compactor *compaction.Compactor, 779 gcRequired bool, 780 log bool, 781 logger *zap.Logger, 782 ) error { 783 if log { 784 logger.Debug("start compaction task") 785 } 786 787 segments := make([]segment.Segment, 0, len(task.Segments)) 788 for _, seg := range task.Segments { 789 segments = append(segments, seg.Segment) 790 } 791 792 var documentsFilter segment.DocumentsFilter 793 if gcRequired { 794 // Only actively filter out documents if GC is required. 795 documentsFilter = segment.DocumentsFilter(m) 796 } 797 798 start := time.Now() 799 compactResult, err := compactor.Compact(segments, documentsFilter, 800 mmap.ReporterOptions{ 801 Context: mmap.Context{ 802 Name: mmapIndexBlockName, 803 }, 804 Reporter: m.opts.MmapReporter(), 805 }) 806 took := time.Since(start) 807 m.metrics.backgroundCompactionTaskRunLatency.Record(took) 808 809 if log { 810 logger.Debug("done compaction task", zap.Duration("took", took)) 811 } 812 813 // Check if result would have resulted in an empty segment. 814 empty := errors.Is(err, compaction.ErrCompactorBuilderEmpty) 815 if empty { 816 // Don't return the error since we need to remove the old segments 817 // by calling addCompactedSegmentFromSegmentsWithLock. 818 err = nil 819 } 820 if err != nil { 821 return err 822 } 823 824 var ( 825 compacted = compactResult.Compacted 826 segMetas = compactResult.SegmentMetadatas 827 replaceSeg segment.Segment 828 ) 829 if empty { 830 m.metrics.activeBlockGarbageCollectEmptySegment.Inc(1) 831 } else { 832 m.metrics.activeBlockGarbageCollectSegment.Inc(1) 833 834 // Add a read through cache for repeated expensive queries against 835 // background compacted segments since they can live for quite some 836 // time and accrue a large set of documents. 837 readThroughSeg := m.newReadThroughSegment(compacted) 838 replaceSeg = readThroughSeg 839 840 // NB(r): Before replacing the old segments with the compacted segment 841 // we rebuild all the cached postings lists that the previous segment had 842 // to avoid latency spikes during segment rotation. 843 // Note: There was very obvious peaks of latency (p99 of <500ms spiking 844 // to 8 times that at first replace of large segments after a block 845 // rotation) without this optimization. 846 if err := m.populateCachedSearches(readThroughSeg, segMetas); err != nil { 847 instrument.EmitAndLogInvariantViolation(m.iopts, func(l *zap.Logger) { 848 l.Error("failed to populate cached searches", zap.Error(err)) 849 }) 850 } 851 } 852 853 // Rotate out the replaced frozen segments and add the compacted one. 854 m.Lock() 855 defer m.Unlock() 856 857 result := m.addCompactedSegmentFromSegmentsWithLock(m.backgroundSegments, 858 segments, replaceSeg) 859 m.backgroundSegments = result 860 861 return nil 862 } 863 864 type cachedPatternForCompactedSegment struct { 865 field string 866 searchQuery *querypb.Query 867 patterns []cachedPatternFromSegment 868 } 869 870 type cachedPatternFromSegment struct { 871 prevSeg prevSegment 872 hasCachedPattern bool 873 cachedPattern CachedPattern 874 } 875 876 type prevSegment struct { 877 segment *ReadThroughSegment 878 meta segment.SegmentsBuilderSegmentMetadata 879 } 880 881 const ( 882 defaultBitmapContainerPooling = 128 883 ) 884 885 type populateCachedSearchesWorker struct { 886 postings postings.MutableList 887 encoder *pilosa.Encoder 888 closers []x.SafeCloser 889 cachedClosers []x.SafeCloser 890 } 891 892 func newPopulateCachedSearchesWorker() *populateCachedSearchesWorker { 893 b := bitmap.NewBitmapWithDefaultPooling(defaultBitmapContainerPooling) 894 return &populateCachedSearchesWorker{ 895 postings: roaring.NewPostingsListFromBitmap(b), 896 encoder: pilosa.NewEncoder(), 897 } 898 } 899 900 func (w *populateCachedSearchesWorker) addCloser(c io.Closer) { 901 if n := len(w.cachedClosers); n > 0 { 902 last := w.cachedClosers[n-1] 903 last.Reset(c) 904 w.cachedClosers[n-1] = nil 905 w.cachedClosers = w.cachedClosers[:n-1] 906 w.closers = append(w.closers, last) 907 return 908 } 909 w.closers = append(w.closers, x.NewSafeCloser(c)) 910 } 911 912 func (w *populateCachedSearchesWorker) close() error { 913 multiErr := xerrors.NewMultiError() 914 for i, c := range w.closers { 915 multiErr = multiErr.Add(c.Close()) 916 w.closers[i] = nil 917 c.Reset(nil) 918 w.cachedClosers = append(w.cachedClosers, c) 919 } 920 w.closers = w.closers[:0] 921 return multiErr.FinalError() 922 } 923 924 func (m *mutableSegments) populateCachedSearches( 925 compactedSeg *ReadThroughSegment, 926 prevSegsMetas []segment.SegmentsBuilderSegmentMetadata, 927 ) error { 928 prevSegs := make([]prevSegment, 0, len(prevSegsMetas)) 929 for _, segMeta := range prevSegsMetas { 930 prevReadThroughSeg, ok := segMeta.Segment.(*ReadThroughSegment) 931 if !ok { 932 return fmt.Errorf("failed to cast compacted segment to read through segment") 933 } 934 prevSegs = append(prevSegs, prevSegment{ 935 segment: prevReadThroughSeg, 936 meta: segMeta, 937 }) 938 } 939 940 searches := make(map[PostingsListCacheKey]cachedPatternForCompactedSegment) 941 for i, seg := range prevSegs { 942 i := i 943 result := seg.segment.CachedSearchPatterns(func(p CachedPattern) { 944 pattern, ok := searches[p.CacheKey] 945 if !ok { 946 pattern = cachedPatternForCompactedSegment{ 947 searchQuery: p.SearchQuery, 948 patterns: make([]cachedPatternFromSegment, len(prevSegs)), 949 } 950 for j, prevSeg := range prevSegs { 951 pattern.patterns[j] = cachedPatternFromSegment{ 952 prevSeg: prevSeg, 953 } 954 } 955 searches[p.CacheKey] = pattern 956 } 957 // Mark this segment with the cached pattern. 958 pattern.patterns[i].hasCachedPattern = true 959 pattern.patterns[i].cachedPattern = p 960 }) 961 if result.CacheSearchesDisabled { 962 m.metrics.activeBlockGarbageCollectCachedSearchesDisabled.Inc(1) 963 } 964 if result.CachedPatternsResult.InRegistry { 965 m.metrics.activeBlockGarbageCollectCachedSearchesInRegistry.Inc(1) 966 } else { 967 m.metrics.activeBlockGarbageCollectCachedSearchesNotInRegistry.Inc(1) 968 } 969 total := float64(result.CachedPatternsResult.TotalPatterns) 970 m.metrics.activeBlockGarbageCollectCachedSearchesTotal.RecordValue(total) 971 matched := float64(result.CachedPatternsResult.MatchedPatterns) 972 m.metrics.activeBlockGarbageCollectCachedSearchesMatched.RecordValue(matched) 973 } 974 975 var totalSegmentsSize int64 976 for _, seg := range prevSegs { 977 totalSegmentsSize += seg.segment.Size() 978 } 979 980 var ( 981 numWorkers = m.cachedSearchesWorkers.Size() 982 workers = make(chan *populateCachedSearchesWorker, numWorkers) 983 instrumentedExec = func(fn func() error) func() error { 984 return func() error { 985 e := fn() 986 if e != nil { 987 m.metrics.activeBlockGarbageCollectReconstructCachedSearchExecError.Inc(1) 988 return e 989 } 990 m.metrics.activeBlockGarbageCollectReconstructCachedSearchExecSuccess.Inc(1) 991 return nil 992 } 993 } 994 group errgroup.Group 995 ) 996 for i := 0; i < numWorkers; i++ { 997 workers <- newPopulateCachedSearchesWorker() 998 } 999 for _, s := range searches { 1000 s := s // Capture for loop. 1001 1002 var totalSegmentsHasPatternSize int64 1003 for i := range s.patterns { 1004 if !s.patterns[i].hasCachedPattern { 1005 continue 1006 } 1007 totalSegmentsHasPatternSize += s.patterns[i].prevSeg.segment.Size() 1008 } 1009 1010 if totalSegmentsHasPatternSize < totalSegmentsSize/2 { 1011 // If segments that represent less than half total size of all 1012 // segments we compacted together do not have the cached pattern 1013 // then don't bother caching this since it could result in an 1014 // expensive computation and we're not getting the benefit from 1015 // running the computation anyway since these aren't searches 1016 // that were cached in the largest segments we just compacted. 1017 m.metrics.activeBlockGarbageCollectReconstructCachedSearchEvalSkip.Inc(1) 1018 continue 1019 } 1020 1021 m.metrics.activeBlockGarbageCollectReconstructCachedSearchEvalAttempt.Inc(1) 1022 1023 // Control concurrency by taking and returning token from worker pool. 1024 w := <-workers 1025 group.Go(instrumentedExec(func() error { 1026 defer func() { 1027 // Close anything needed to be closed. 1028 _ = w.close() 1029 // Return worker. 1030 workers <- w 1031 }() 1032 1033 if s.searchQuery == nil { 1034 return fmt.Errorf("no search query for cached search pattern") 1035 } 1036 1037 var searcher search.Searcher 1038 search, err := query.UnmarshalProto(s.searchQuery) 1039 if err != nil { 1040 return fmt.Errorf("failed to unmarshal search for cached search pattern: %w", err) 1041 } 1042 1043 // Reset reused postings. 1044 w.postings.Reset() 1045 1046 for i := range s.patterns { 1047 var iter postings.Iterator 1048 if s.patterns[i].hasCachedPattern { 1049 // If has the cached pattern, no need to execute a search, can reuse 1050 // the postings iterator from the cache. 1051 iter = s.patterns[i].cachedPattern.Postings.Iterator() 1052 w.addCloser(iter) 1053 1054 // Track hit. 1055 m.metrics.activeBlockGarbageCollectReconstructCachedSearchCacheHit.Inc(1) 1056 } else { 1057 // Does not have this pattern cached, need to execute the search 1058 // against this segment. 1059 if searcher == nil { 1060 searcher, err = search.Searcher() 1061 if err != nil { 1062 return fmt.Errorf("failed to create searcher for cached search pattern: %w", err) 1063 } 1064 } 1065 1066 reader, err := s.patterns[i].prevSeg.segment.Reader() 1067 if err != nil { 1068 return fmt.Errorf("failed to create prev seg reader: %w", err) 1069 } 1070 1071 w.addCloser(reader) 1072 1073 pl, err := searcher.Search(reader) 1074 if err != nil { 1075 return fmt.Errorf("failed to search prev seg reader: %w", err) 1076 } 1077 1078 iter = pl.Iterator() 1079 w.addCloser(iter) 1080 1081 // Track miss. 1082 m.metrics.activeBlockGarbageCollectReconstructCachedSearchCacheMiss.Inc(1) 1083 } 1084 1085 if s.patterns[i].prevSeg.meta.Offset == 0 && s.patterns[i].prevSeg.meta.Skips == 0 { 1086 // No offset and no postings to skip, can copy into the reused postings. 1087 if err := w.postings.AddIterator(iter); err != nil { 1088 return fmt.Errorf("could not copy cached postings: %w", err) 1089 } 1090 continue 1091 } 1092 1093 // We have to take into account offset and duplicates/skips. 1094 negativeOffsets := s.patterns[i].prevSeg.meta.NegativeOffsets 1095 for iter.Next() { 1096 curr := iter.Current() 1097 negativeOffset := negativeOffsets[curr] 1098 // Then skip the individual if matches. 1099 if negativeOffset == -1 { 1100 // Skip this value, as itself is a duplicate. 1101 continue 1102 } 1103 value := curr + s.patterns[i].prevSeg.meta.Offset - postings.ID(negativeOffset) 1104 if err := w.postings.Insert(value); err != nil { 1105 if closeErr := iter.Close(); closeErr != nil { 1106 err = xerrors.NewMultiError().Add(err).Add(closeErr) 1107 } 1108 return fmt.Errorf("could not insert from cached postings: %w", err) 1109 } 1110 } 1111 1112 err := iter.Err() 1113 if closeErr := iter.Close(); closeErr != nil { 1114 err = xerrors.NewMultiError().Add(err).Add(closeErr) 1115 } 1116 if err != nil { 1117 return fmt.Errorf("could not close cached postings: %w", err) 1118 } 1119 } 1120 1121 // Encode the result and create a read only copy as we reuse the postings 1122 // list in the worker to build consequent postings. 1123 bytes, err := w.encoder.Encode(w.postings) 1124 if err != nil { 1125 return fmt.Errorf("could not encode result cached search postings: %w", err) 1126 } 1127 1128 bitmap.NewBitmap() 1129 1130 // Need to copy bytes since encoder owns the bytes. 1131 copied := append(make([]byte, 0, len(bytes)), bytes...) 1132 bm := bitmap.NewBitmap() 1133 if err := bm.UnmarshalBinary(copied); err != nil { 1134 return fmt.Errorf("could not create result cached search postings: %w", err) 1135 } 1136 1137 pl := roaring.NewPostingsListFromBitmap(bm) 1138 compactedSeg.PutCachedSearchPattern(s.field, search, pl) 1139 return nil 1140 })) 1141 } 1142 1143 return group.Wait() 1144 } 1145 1146 func (m *mutableSegments) addCompactedSegmentFromSegmentsWithLock( 1147 current []*readableSeg, 1148 segmentsJustCompacted []segment.Segment, 1149 compacted segment.Segment, 1150 ) []*readableSeg { 1151 result := make([]*readableSeg, 0, len(current)) 1152 for _, existing := range current { 1153 keepCurr := true 1154 for _, seg := range segmentsJustCompacted { 1155 if existing.Segment() == seg { 1156 // Do not keep this one, it was compacted just then. 1157 keepCurr = false 1158 break 1159 } 1160 } 1161 1162 if keepCurr { 1163 result = append(result, existing) 1164 continue 1165 } 1166 1167 err := existing.Segment().Close() 1168 if err != nil { 1169 // Already compacted, not much we can do about not closing it. 1170 instrument.EmitAndLogInvariantViolation(m.iopts, func(l *zap.Logger) { 1171 l.Error("unable to close compacted block", zap.Error(err)) 1172 }) 1173 } 1174 } 1175 1176 if compacted == nil { 1177 return result 1178 } 1179 1180 // Return all the ones we kept plus the new compacted segment 1181 return append(result, newReadableSeg(compacted, m.opts)) 1182 } 1183 1184 func (m *mutableSegments) foregroundCompactWithBuilder( 1185 builder segment.DocumentsBuilder, 1186 ) (MutableSegmentsStats, error) { 1187 // We inserted some documents, need to compact immediately into a 1188 // foreground segment. 1189 m.Lock() 1190 foregroundSegments := m.foregroundSegments 1191 m.Unlock() 1192 1193 segs := make([]compaction.Segment, 0, len(foregroundSegments)+1) 1194 segs = append(segs, compaction.Segment{ 1195 Age: 0, 1196 Size: int64(len(builder.Docs())), 1197 Type: segments.MutableType, 1198 Builder: builder, 1199 }) 1200 for _, seg := range foregroundSegments { 1201 segs = append(segs, compaction.Segment{ 1202 Age: seg.Age(), 1203 Size: seg.Segment().Size(), 1204 Type: segments.FSTType, 1205 Segment: seg.Segment(), 1206 }) 1207 } 1208 1209 plan, err := compaction.NewPlan(segs, m.opts.ForegroundCompactionPlannerOptions()) 1210 if err != nil { 1211 return MutableSegmentsStats{}, err 1212 } 1213 1214 // Check plan 1215 if len(plan.Tasks) == 0 { 1216 // Should always generate a task when a mutable builder is passed to planner 1217 return MutableSegmentsStats{}, errForegroundCompactorNoPlan 1218 } 1219 if taskNumBuilders(plan.Tasks[0]) != 1 { 1220 // First task of plan must include the builder, so we can avoid resetting it 1221 // for the first task, but then safely reset it in consequent tasks 1222 return MutableSegmentsStats{}, errForegroundCompactorBadPlanFirstTask 1223 } 1224 1225 // Move any unused segments to the background. 1226 m.Lock() 1227 m.maybeMoveForegroundSegmentsToBackgroundWithLock(plan.UnusedSegments) 1228 m.Unlock() 1229 1230 n := m.compact.numForeground 1231 m.compact.numForeground++ 1232 1233 logger := m.logger.With( 1234 zap.Time("blockStart", m.blockStart.ToTime()), 1235 zap.Int("numForegroundCompaction", n), 1236 ) 1237 log := n%compactDebugLogEvery == 0 1238 if log { 1239 for i, task := range plan.Tasks { 1240 summary := task.Summary() 1241 logger.Debug("planned foreground compaction task", 1242 zap.Int("task", i), 1243 zap.Int("numMutable", summary.NumMutable), 1244 zap.Int("numFST", summary.NumFST), 1245 zap.Duration("cumulativeMutableAge", summary.CumulativeMutableAge), 1246 zap.Int64("cumulativeSize", summary.CumulativeSize), 1247 ) 1248 } 1249 } 1250 1251 // Run the plan. 1252 sw := m.metrics.foregroundCompactionPlanRunLatency.Start() 1253 defer sw.Stop() 1254 1255 // Run the first task, without resetting the builder. 1256 result, err := m.foregroundCompactWithTask(builder, plan.Tasks[0], 1257 log, logger.With(zap.Int("task", 0))) 1258 if err != nil { 1259 return result, err 1260 } 1261 1262 // Now run each consequent task, resetting the builder each time since 1263 // the results from the builder have already been compacted in the first 1264 // task. 1265 for i := 1; i < len(plan.Tasks); i++ { 1266 task := plan.Tasks[i] 1267 if taskNumBuilders(task) > 0 { 1268 // Only the first task should compact the builder 1269 return result, errForegroundCompactorBadPlanSecondaryTask 1270 } 1271 // Now use the builder after resetting it. 1272 builder.Reset() 1273 result, err = m.foregroundCompactWithTask(builder, task, 1274 log, logger.With(zap.Int("task", i))) 1275 if err != nil { 1276 return result, err 1277 } 1278 } 1279 1280 return result, nil 1281 } 1282 1283 func (m *mutableSegments) maybeMoveForegroundSegmentsToBackgroundWithLock( 1284 segments []compaction.Segment, 1285 ) { 1286 if len(segments) == 0 { 1287 return 1288 } 1289 if m.compact.backgroundCompactors == nil { 1290 // No longer performing background compaction due to evict/close. 1291 return 1292 } 1293 1294 m.logger.Debug("moving segments from foreground to background", 1295 zap.Int("numSegments", len(segments))) 1296 1297 // If background compaction is still active, then we move any unused 1298 // foreground segments into the background so that they might be 1299 // compacted by the background compactor at some point. 1300 i := 0 1301 for _, currForeground := range m.foregroundSegments { 1302 movedToBackground := false 1303 for _, seg := range segments { 1304 if currForeground.Segment() == seg.Segment { 1305 m.backgroundSegments = append(m.backgroundSegments, currForeground) 1306 movedToBackground = true 1307 break 1308 } 1309 } 1310 if movedToBackground { 1311 continue // No need to keep this segment, we moved it. 1312 } 1313 1314 m.foregroundSegments[i] = currForeground 1315 i++ 1316 } 1317 1318 m.foregroundSegments = m.foregroundSegments[:i] 1319 1320 // Potentially kick off a background compaction. 1321 m.maybeBackgroundCompactWithLock() 1322 } 1323 1324 func (m *mutableSegments) foregroundCompactWithTask( 1325 builder segment.DocumentsBuilder, 1326 task compaction.Task, 1327 log bool, 1328 logger *zap.Logger, 1329 ) (MutableSegmentsStats, error) { 1330 if log { 1331 logger.Debug("start compaction task") 1332 } 1333 1334 segments := make([]segment.Segment, 0, len(task.Segments)) 1335 for _, seg := range task.Segments { 1336 if seg.Segment == nil { 1337 continue // This means the builder is being used. 1338 } 1339 segments = append(segments, seg.Segment) 1340 } 1341 1342 start := time.Now() 1343 compacted, err := m.compact.foregroundCompactor.CompactUsingBuilder(builder, segments, mmap.ReporterOptions{ 1344 Context: mmap.Context{ 1345 Name: mmapIndexBlockName, 1346 }, 1347 Reporter: m.opts.MmapReporter(), 1348 }) 1349 took := time.Since(start) 1350 m.metrics.foregroundCompactionTaskRunLatency.Record(took) 1351 1352 if log { 1353 logger.Debug("done compaction task", zap.Duration("took", took)) 1354 } 1355 1356 if err != nil { 1357 return MutableSegmentsStats{}, err 1358 } 1359 1360 // Add a read through cache for repeated expensive queries against 1361 // compacted segments since they can live for quite some time during 1362 // block rotations while a burst of segments are created. 1363 segment := m.newReadThroughSegment(compacted) 1364 1365 // Rotate in the ones we just compacted. 1366 m.Lock() 1367 defer m.Unlock() 1368 1369 result := m.addCompactedSegmentFromSegmentsWithLock(m.foregroundSegments, 1370 segments, segment) 1371 m.foregroundSegments = result 1372 1373 foregroundNumSegments, foregroundNumDocs := numSegmentsAndDocs(m.foregroundSegments) 1374 backgroundNumSegments, backgroundNumDocs := numSegmentsAndDocs(m.backgroundSegments) 1375 return MutableSegmentsStats{ 1376 Foreground: MutableSegmentsSegmentStats{ 1377 NumSegments: foregroundNumSegments, 1378 NumDocs: foregroundNumDocs, 1379 }, 1380 Background: MutableSegmentsSegmentStats{ 1381 NumSegments: backgroundNumSegments, 1382 NumDocs: backgroundNumDocs, 1383 }, 1384 }, nil 1385 } 1386 1387 func (m *mutableSegments) cleanupForegroundCompactWithLock() { 1388 // Check if need to close all the compacted segments due to 1389 // mutableSegments being closed. 1390 if !m.shouldEvictCompactedSegmentsWithLock() { 1391 return 1392 } 1393 1394 // Close compacted segments. 1395 m.closeCompactedSegmentsWithLock(m.foregroundSegments) 1396 m.foregroundSegments = nil 1397 1398 // Free compactor resources. 1399 if m.compact.foregroundCompactor != nil { 1400 if err := m.compact.foregroundCompactor.Close(); err != nil { 1401 instrument.EmitAndLogInvariantViolation(m.iopts, func(l *zap.Logger) { 1402 l.Error("error closing index block foreground compactor", zap.Error(err)) 1403 }) 1404 } 1405 m.compact.foregroundCompactor = nil 1406 } 1407 1408 // Free segment builder resources. 1409 if m.compact.segmentBuilder != nil { 1410 if err := m.compact.segmentBuilder.Close(); err != nil { 1411 instrument.EmitAndLogInvariantViolation(m.iopts, func(l *zap.Logger) { 1412 l.Error("error closing index block segment builder", zap.Error(err)) 1413 }) 1414 } 1415 m.compact.segmentBuilder = nil 1416 } 1417 } 1418 1419 func (m *mutableSegments) cleanupCompactWithLock() { 1420 // If not compacting, trigger a cleanup so that all frozen segments get 1421 // closed, otherwise after the current running compaction the compacted 1422 // segments will get closed. 1423 if !m.compact.compactingForeground { 1424 m.cleanupForegroundCompactWithLock() 1425 } 1426 if !m.compact.compactingBackgroundStandard && !m.compact.compactingBackgroundGarbageCollect { 1427 m.cleanupBackgroundCompactWithLock() 1428 } 1429 } 1430 1431 // mutableSegmentsCompact has several lazily allocated compaction components. 1432 type mutableSegmentsCompact struct { 1433 opts Options 1434 blockOpts BlockOptions 1435 1436 segmentBuilder segment.CloseableDocumentsBuilder 1437 foregroundCompactor *compaction.Compactor 1438 backgroundCompactors chan *compaction.Compactor 1439 compactingForeground bool 1440 compactingBackgroundStandard bool 1441 compactingBackgroundGarbageCollect bool 1442 numForeground int 1443 numBackground int 1444 1445 foregroundCompactorCreatedAt time.Time 1446 } 1447 1448 func (m *mutableSegmentsCompact) allocLazyBuilderAndCompactorsWithLock( 1449 concurrency int, 1450 ) error { 1451 var ( 1452 err error 1453 metadataPool = m.opts.MetadataArrayPool() 1454 ) 1455 if m.segmentBuilder == nil { 1456 builderOpts := m.opts.SegmentBuilderOptions(). 1457 SetConcurrency(concurrency) 1458 1459 m.segmentBuilder, err = builder.NewBuilderFromDocuments(builderOpts) 1460 if err != nil { 1461 return err 1462 } 1463 } 1464 1465 // Compactors are not meant to be long-lived because of the pooling and accumulation of allocs 1466 // that occur over time. Prior to active block change, these compactors were closed regularly per 1467 // block rotations since the ownership is block->mutableSegments->compactor->fstWriter->builder. 1468 // To account for the active block being long-lived, we now periodically GC the compactor and create anew. 1469 now := m.opts.ClockOptions().NowFn()() 1470 if m.foregroundCompactor != nil && now.Sub(m.foregroundCompactorCreatedAt) > maxForegroundCompactorAge { 1471 if err := m.foregroundCompactor.Close(); err != nil { 1472 m.opts.InstrumentOptions().Logger().Error("error closing foreground compactor", zap.Error(err)) 1473 } 1474 m.foregroundCompactor = nil 1475 } 1476 1477 if m.foregroundCompactor == nil { 1478 m.foregroundCompactorCreatedAt = now 1479 m.foregroundCompactor, err = compaction.NewCompactor(metadataPool, 1480 MetadataArrayPoolCapacity, 1481 m.opts.SegmentBuilderOptions(), 1482 m.opts.FSTSegmentOptions(), 1483 compaction.CompactorOptions{ 1484 FSTWriterOptions: &fst.WriterOptions{ 1485 // DisableRegistry is set to true to trade a larger FST size 1486 // for a faster FST compaction since we want to reduce the end 1487 // to end latency for time to first index a metric. 1488 DisableRegistry: true, 1489 }, 1490 MmapDocsData: m.blockOpts.ForegroundCompactorMmapDocsData, 1491 }) 1492 if err != nil { 1493 return err 1494 } 1495 } 1496 1497 if m.backgroundCompactors == nil { 1498 n := numBackgroundCompactorsStandard 1499 m.backgroundCompactors = make(chan *compaction.Compactor, n) 1500 for i := 0; i < n; i++ { 1501 backgroundCompactor, err := compaction.NewCompactor(metadataPool, 1502 MetadataArrayPoolCapacity, 1503 m.opts.SegmentBuilderOptions(), 1504 m.opts.FSTSegmentOptions(), 1505 compaction.CompactorOptions{ 1506 MmapDocsData: m.blockOpts.BackgroundCompactorMmapDocsData, 1507 }) 1508 if err != nil { 1509 return err 1510 } 1511 m.backgroundCompactors <- backgroundCompactor 1512 } 1513 } 1514 1515 return nil 1516 } 1517 1518 func (m *mutableSegmentsCompact) allocBackgroundCompactorsGarbageCollect() ( 1519 chan *compaction.Compactor, 1520 error, 1521 ) { 1522 metadataPool := m.opts.MetadataArrayPool() 1523 n := numBackgroundCompactorsGarbageCollect 1524 compactors := make(chan *compaction.Compactor, n) 1525 for i := 0; i < n; i++ { 1526 backgroundCompactor, err := compaction.NewCompactor(metadataPool, 1527 MetadataArrayPoolCapacity, 1528 m.opts.SegmentBuilderOptions(), 1529 m.opts.FSTSegmentOptions(), 1530 compaction.CompactorOptions{ 1531 MmapDocsData: m.blockOpts.BackgroundCompactorMmapDocsData, 1532 }) 1533 if err != nil { 1534 return nil, err 1535 } 1536 compactors <- backgroundCompactor 1537 } 1538 return compactors, nil 1539 } 1540 1541 func taskNumBuilders(task compaction.Task) int { 1542 builders := 0 1543 for _, seg := range task.Segments { 1544 if seg.Builder != nil { 1545 builders++ 1546 continue 1547 } 1548 } 1549 return builders 1550 }